You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by ro...@apache.org on 2019/06/12 07:13:07 UTC

[arrow] branch master updated: ARROW-5488: [R] Workaround when C++ lib not available

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

romainfrancois pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/arrow.git


The following commit(s) were added to refs/heads/master by this push:
     new 48ee38f  ARROW-5488: [R] Workaround when C++ lib not available
48ee38f is described below

commit 48ee38f833a51bb46a404da10d89c4b687e952c9
Author: Romain Francois <ro...@rstudio.com>
AuthorDate: Wed Jun 12 09:12:49 2019 +0200

    ARROW-5488: [R] Workaround when C++ lib not available
    
    This is very much a work in progress. The idea is to replace the code generation that is usually done by `Rcpp::compileAttributes()` with something custom. This is driven by the `data-raw/codegen.R` file, which I'll polish. I'm opening this right now for feedback opportunities.
    
    All of the `.cpp` files are guarded by :
    
    ```
    #if defined(ARROW_R_WITH_ARROW)
    ...
    #endif
    ```
    
    And `ARROW_R_WITH_ARROW` is defined via the configure file, if the library was indeed used.
    
    For example, this function:
    
    ```cpp
    // [[arrow::export]]
    std::shared_ptr<arrow::Column> ipc___feather___TableReader__GetColumn(
        const std::unique_ptr<arrow::ipc::feather::TableReader>& reader, int i) {
      std::shared_ptr<arrow::Column> column;
      STOP_IF_NOT_OK(reader->GetColumn(i, &column));
      return column;
    }
    ```
    
    triggers generation of this in the `generated.cpp` file:
    
    ```cpp
    #if defined(ARROW_R_WITH_ARROW)
    std::string ipc___feather___TableReader__GetColumnName(const std::unique_ptr<arrow::ipc::feather::TableReader>& reader, int i);
    SEXP _arrow_ipc___feather___TableReader__GetColumnName(SEXP reader_sexp, SEXP i_sexp){
      BEGIN_RCPP
      Rcpp::traits::input_parameter<const std::unique_ptr<arrow::ipc::feather::TableReader>&>::type reader(reader_sexp);
    Rcpp::traits::input_parameter<int>::type i(i_sexp);Rcpp::Shield<SEXP> rcpp_result_gen(Rcpp::wrap(ipc___feather___TableReader__GetColumnName( reader, i)));
    return rcpp_result_gen;
      END_RCPP
    }
    #else
    SEXP _arrow_ipc___feather___TableReader__GetColumnName(SEXP reader_sexp, SEXP i_sexp){
      BEGIN_RCPP
      Rcpp::stop("arrow C++ library not available");
      END_RCPP
    }
    #endif
    ```
    
    So the generated R api `SEXP`y functions only call the real thing when the c++ library is available, otherwise they just throw an error.
    
    and this in the `generated.R` file:
    
    ```r
    ipc___feather___TableReader__GetColumnName <- function(reader, i) {
        .Call(`_arrow_ipc___feather___TableReader__GetColumnName` , reader, i )
    }
    ```
    
    This also needed some extra care in test functions so that the tests only run if Arrow is available.
    
    @wesm comment from https://issues.apache.org/jira/browse/ARROW-5488 might be more practical and closer to what @jjallaire mentioned about how the `RcppParallel` package does it with Intel tbb:
    https://github.com/RcppCore/RcppParallel/blob/master/R/hooks.R
    
    > One possibility is to bundle the Arrow header files with the CRAN package and build against them, but do not include libarrow and libparquet when linking. When the library is loaded, the libraries must be loaded in-process via dlopen before loading the Rcpp extensions. The C++ libraries can be installed then after the fact
    
    Author: Romain Francois <ro...@rstudio.com>
    Author: Romain François <ro...@rstudio.com>
    
    Closes #4471 from romainfrancois/ARROW-5488/workaround and squashes the following commits:
    
    d13dbfbe <Romain Francois> update error message to mention arrow::install_arrow()
    1b6a737c <Romain Francois> Merge branch 'ARROW-5488/workaround' of https://github.com/romainfrancois/arrow into ARROW-5488/workaround
    418fc2d0 <Romain Francois> mention codegen in the README
    9a1b3e66 <Romain Francois> glue() not needed here
    496eaf52 <Romain Francois> when brew is available but apache-arrow is not installed, install it
    b8a6576b <Romain Francois> no need for stringr
    238fa14d <Romain Francois> rm message about ARROW_R_DEV not being set
    97888116 <Romain François> Merge branch 'master' into ARROW-5488/workaround
    346978d0 <Romain Francois> update docker files.
    bb9eed2d <Romain Francois> not showing diff for generated files
    1df1e395 <Romain Francois> phrase shim message more positively
    3b043696 <Romain Francois> use `\dontrun{}` in examples
    562670df <Romain Francois> RcppExport all the things
    6c06581e <Romain Francois> Using arrow:::arrow_available()
    75b0751a <Romain Francois> Move symbols.h back into arrow_types.h
    bd7f30d7 <Romain Francois> :rat:
    e8ac3ea9 <Romain Francois> using arrowExports.(R,cpp). update configure script
    84fa149f <Romain Francois> lint
    54ee38d4 <Romain Francois> :rat:
    d9e4f944 <Romain Francois> added a second R job to install and check the package on a system without libarrow
    0479a753 <Romain Francois> code generation without rap package, using purrr instead
    008eaf4a <Romain Francois> update generated code, fix merge conflicts
    f9575045 <Romain Francois> update test_that shim
    1170236b <Romain Francois> not necessary
    1ed55966 <Romain Francois> Workaround so that the R package still checks without the C++ library.
    78f69af6 <Romain Francois> move symbols declaration to their own file, might end up be generated automatically later somehow.
---
 .gitattributes                               |    2 +
 .travis.yml                                  |   17 +-
 ci/docker_build_r.sh                         |    2 -
 cpp/build-support/lint_exclusions.txt        |    1 +
 dev/release/rat_exclude_files.txt            |    4 +-
 r/DESCRIPTION                                |    2 +-
 r/Dockerfile                                 |   13 +-
 r/NAMESPACE                                  |    3 +-
 r/R/Field.R                                  |    7 +-
 r/R/RcppExports.R                            |  844 -------
 r/R/arrowExports.R                           |  835 +++++++
 r/R/parquet.R                                |    6 +-
 r/R/zzz.R                                    |    8 +
 r/README.Rmd                                 |    6 +-
 r/README.md                                  |   31 +-
 r/configure                                  |   84 +-
 r/data-raw/codegen.R                         |  172 ++
 r/man/arrow_available.Rd                     |   11 +
 r/man/field.Rd                               |    5 +-
 r/man/read_parquet.Rd                        |    6 +-
 r/man/table.Rd                               |    5 +-
 r/man/threadpool.Rd                          |   27 -
 r/src/RcppExports.cpp                        | 2530 -------------------
 r/src/array.cpp                              |   38 +-
 r/src/array__to_vector.cpp                   |   14 +-
 r/src/array_from_vector.cpp                  |    9 +-
 r/src/arraydata.cpp                          |   14 +-
 r/src/arrowExports.cpp                       | 3428 ++++++++++++++++++++++++++
 r/src/arrow_types.h                          |  118 +-
 r/src/buffer.cpp                             |   14 +-
 r/src/chunkedarray.cpp                       |   20 +-
 r/src/column.cpp                             |   12 +-
 r/src/compression.cpp                        |   10 +-
 r/src/compute.cpp                            |   14 +-
 r/src/csv.cpp                                |   14 +-
 r/src/datatype.cpp                           |  118 +-
 r/src/feather.cpp                            |   34 +-
 r/src/field.cpp                              |   16 +-
 r/src/io.cpp                                 |   50 +-
 r/src/memorypool.cpp                         |    9 +-
 r/src/message.cpp                            |   26 +-
 r/src/parquet.cpp                            |    6 +-
 r/src/recordbatch.cpp                        |   52 +-
 r/src/recordbatchreader.cpp                  |   26 +-
 r/src/recordbatchwriter.cpp                  |   14 +-
 r/src/table.cpp                              |   29 +-
 r/src/threadpool.cpp                         |   13 +-
 r/tests/testthat.R                           |    1 +
 r/{R/zzz.R => tests/testthat/helper-arrow.R} |    9 +-
 49 files changed, 4980 insertions(+), 3749 deletions(-)

diff --git a/.gitattributes b/.gitattributes
index 4a9f90a..fac7bf8 100644
--- a/.gitattributes
+++ b/.gitattributes
@@ -1,4 +1,6 @@
 r/R/RcppExports.R linguist-generated=true
+r/R/arrowExports.R linguist-generated=true
 r/src/RcppExports.cpp linguist-generated=true
+r/src/arrowExports.cpp linguist-generated=true
 r/man/*.Rd linguist-generated=true
 
diff --git a/.travis.yml b/.travis.yml
index 0c0f9fc..74b5bad 100644
--- a/.travis.yml
+++ b/.travis.yml
@@ -365,7 +365,22 @@ matrix:
     - pushd ${TRAVIS_BUILD_DIR}/r
     after_success:
     - Rscript ../ci/travis_upload_r_coverage.R
-
+  - name: R_no_libarrow
+    language: r
+    cache: packages
+    latex: false
+    dist: xenial
+    before_install:
+    # Have to copy-paste this here because of how R's build steps work
+    - eval `python $TRAVIS_BUILD_DIR/ci/detect-changes.py`
+    - if [ $ARROW_CI_R_AFFECTED != "1" ]; then exit; fi
+    - |
+        if [ $TRAVIS_OS_NAME == "linux" ]; then
+          sudo bash -c "echo -e 'Acquire::Retries 10; Acquire::http::Timeout \"20\";' > /etc/apt/apt.conf.d/99-travis-retry"
+          sudo add-apt-repository -y ppa:ubuntu-toolchain-r/test
+          sudo apt-get update -qq
+        fi
+    - pushd ${TRAVIS_BUILD_DIR}/r
 
 after_failure:
 - |
diff --git a/ci/docker_build_r.sh b/ci/docker_build_r.sh
index 7f72cdf..6355afa 100755
--- a/ci/docker_build_r.sh
+++ b/ci/docker_build_r.sh
@@ -23,8 +23,6 @@ export ARROW_HOME=$CONDA_PREFIX
 # Build arrow
 pushd /arrow/r
 
-rm src/RcppExports*
-Rscript -e "Rcpp::compileAttributes()"
 R CMD build --keep-empty-dirs .
 R CMD INSTALL $(ls | grep arrow_*.tar.gz)
 
diff --git a/cpp/build-support/lint_exclusions.txt b/cpp/build-support/lint_exclusions.txt
index 1187beb..448c1b1 100644
--- a/cpp/build-support/lint_exclusions.txt
+++ b/cpp/build-support/lint_exclusions.txt
@@ -8,3 +8,4 @@
 *thirdparty/ae/*
 *vendored/*
 *RcppExports.cpp*
+*arrowExports.cpp*
diff --git a/dev/release/rat_exclude_files.txt b/dev/release/rat_exclude_files.txt
index 4c5aef3..4438917 100644
--- a/dev/release/rat_exclude_files.txt
+++ b/dev/release/rat_exclude_files.txt
@@ -193,8 +193,8 @@ csharp/test/Directory.Build.props
 *.svg
 *.devhelp2
 *.scss
-r/R/RcppExports.R
-r/src/RcppExports.cpp
+r/R/arrowExports.R
+r/src/arrowExports.cpp
 r/DESCRIPTION
 r/LICENSE.md
 r/NAMESPACE
diff --git a/r/DESCRIPTION b/r/DESCRIPTION
index c82bf91..d566f74 100644
--- a/r/DESCRIPTION
+++ b/r/DESCRIPTION
@@ -52,7 +52,6 @@ Collate:
     'Column.R'
     'Field.R'
     'List.R'
-    'RcppExports.R'
     'RecordBatch.R'
     'RecordBatchReader.R'
     'RecordBatchWriter.R'
@@ -60,6 +59,7 @@ Collate:
     'Struct.R'
     'Table.R'
     'array.R'
+    'arrowExports.R'
     'buffer.R'
     'io.R'
     'compression.R'
diff --git a/r/Dockerfile b/r/Dockerfile
index e18aeb9..7c0b087 100644
--- a/r/Dockerfile
+++ b/r/Dockerfile
@@ -48,14 +48,10 @@ RUN apt update && \
     # R CMD CHECK --as-cran needs pdflatex to build the package manual
     apt install -y \
         texlive-latex-base && \
-    # Install vctrs from Github
-    Rscript -e "install.packages('devtools', repos = 'https://cran.rstudio.com')" && \
-    Rscript -e "devtools::install_github('romainfrancois/vctrs@bit64')" && \
-    Rscript -e "devtools::install_github('r-lib/withr')" && \
-    Rscript -e "devtools::install_github('RcppCore/Rcpp')" && \
-    # R is not good at picking up dependencies and installing them automatically
+    Rscript -e "install.packages('devtools', repos = 'http://cran.rstudio.com')" && \
+    Rscript -e "devtools::install_github('romainfrancois/decor')" && \
     Rscript -e "install.packages(c( \
-        'Rcpp', \
+        'Rcpp', 'dplyr', 'stringr', 'glue', 'vctrs', \
         'purrr', \
         'assertthat', \
         'fs', \
@@ -67,6 +63,9 @@ RUN apt update && \
         'lubridate'), \
         repos = 'https://cran.rstudio.com')"
 
+# So that arrowExports.* files are generated
+ENV ARROW_R_DEV=TRUE
+
 # Tell R where it can find the source code for arrow
 ENV PKG_CONFIG_PATH=${PKG_CONFIG_PATH}:/build/cpp/src/arrow:/opt/conda/lib/pkgconfig
 ENV LD_LIBRARY_PATH=/opt/conda/lib/:/build/cpp/src/arrow:/arrow/r/src
diff --git a/r/NAMESPACE b/r/NAMESPACE
index 4b79957..bc4f677 100644
--- a/r/NAMESPACE
+++ b/r/NAMESPACE
@@ -96,7 +96,6 @@ export(FeatherTableWriter)
 export(FileMode)
 export(FileOutputStream)
 export(FixedSizeBufferWriter)
-export(GetCpuThreadPoolCapacity)
 export(MessageReader)
 export(MessageType)
 export(MockOutputStream)
@@ -105,11 +104,11 @@ export(RecordBatchFileReader)
 export(RecordBatchFileWriter)
 export(RecordBatchStreamReader)
 export(RecordBatchStreamWriter)
-export(SetCpuThreadPoolCapacity)
 export(StatusCode)
 export(TimeUnit)
 export(Type)
 export(array)
+export(arrow_available)
 export(as_tibble)
 export(boolean)
 export(buffer)
diff --git a/r/R/Field.R b/r/R/Field.R
index 4f5636f..5175ee5 100644
--- a/r/R/Field.R
+++ b/r/R/Field.R
@@ -64,14 +64,17 @@
 #' @param metadata currently ignored
 #'
 #' @examples
-#' field("x", int32())
+#'
+#' \dontrun{
+#'    field("x", int32())
+#' }
 #'
 #' @export
 field <- function(name, type, metadata) {
   assert_that(inherits(name, "character"), length(name) == 1L)
   assert_that(inherits(type, "arrow::DataType"))
   assert_that(missing(metadata), msg = "metadata= is currently ignored")
-  shared_ptr(`arrow::Field`, Field__initialize(name, type))
+  shared_ptr(`arrow::Field`, Field__initialize(name, type, TRUE))
 }
 
 .fields <- function(.list){
diff --git a/r/R/RcppExports.R b/r/R/RcppExports.R
deleted file mode 100644
index 69745f8..0000000
--- a/r/R/RcppExports.R
+++ /dev/null
@@ -1,844 +0,0 @@
-# Generated by using Rcpp::compileAttributes() -> do not edit by hand
-# Generator token: 10BE3573-1514-4C36-9D1C-5A225CD40393
-
-Array__Slice1 <- function(array, offset) {
-    .Call(`_arrow_Array__Slice1`, array, offset)
-}
-
-Array__Slice2 <- function(array, offset, length) {
-    .Call(`_arrow_Array__Slice2`, array, offset, length)
-}
-
-Array__IsNull <- function(x, i) {
-    .Call(`_arrow_Array__IsNull`, x, i)
-}
-
-Array__IsValid <- function(x, i) {
-    .Call(`_arrow_Array__IsValid`, x, i)
-}
-
-Array__length <- function(x) {
-    .Call(`_arrow_Array__length`, x)
-}
-
-Array__offset <- function(x) {
-    .Call(`_arrow_Array__offset`, x)
-}
-
-Array__null_count <- function(x) {
-    .Call(`_arrow_Array__null_count`, x)
-}
-
-Array__type <- function(x) {
-    .Call(`_arrow_Array__type`, x)
-}
-
-Array__ToString <- function(x) {
-    .Call(`_arrow_Array__ToString`, x)
-}
-
-Array__type_id <- function(x) {
-    .Call(`_arrow_Array__type_id`, x)
-}
-
-Array__Equals <- function(lhs, rhs) {
-    .Call(`_arrow_Array__Equals`, lhs, rhs)
-}
-
-Array__ApproxEquals <- function(lhs, rhs) {
-    .Call(`_arrow_Array__ApproxEquals`, lhs, rhs)
-}
-
-Array__data <- function(array) {
-    .Call(`_arrow_Array__data`, array)
-}
-
-Array__RangeEquals <- function(self, other, start_idx, end_idx, other_start_idx) {
-    .Call(`_arrow_Array__RangeEquals`, self, other, start_idx, end_idx, other_start_idx)
-}
-
-Array__Mask <- function(array) {
-    .Call(`_arrow_Array__Mask`, array)
-}
-
-DictionaryArray__indices <- function(array) {
-    .Call(`_arrow_DictionaryArray__indices`, array)
-}
-
-DictionaryArray__dictionary <- function(array) {
-    .Call(`_arrow_DictionaryArray__dictionary`, array)
-}
-
-Array__as_vector <- function(array) {
-    .Call(`_arrow_Array__as_vector`, array)
-}
-
-ChunkedArray__as_vector <- function(chunked_array) {
-    .Call(`_arrow_ChunkedArray__as_vector`, chunked_array)
-}
-
-RecordBatch__to_dataframe <- function(batch, use_threads) {
-    .Call(`_arrow_RecordBatch__to_dataframe`, batch, use_threads)
-}
-
-Table__to_dataframe <- function(table, use_threads) {
-    .Call(`_arrow_Table__to_dataframe`, table, use_threads)
-}
-
-Array__infer_type <- function(x) {
-    .Call(`_arrow_Array__infer_type`, x)
-}
-
-Array__from_vector <- function(x, s_type) {
-    .Call(`_arrow_Array__from_vector`, x, s_type)
-}
-
-ChunkedArray__from_list <- function(chunks, s_type) {
-    .Call(`_arrow_ChunkedArray__from_list`, chunks, s_type)
-}
-
-ArrayData__get_type <- function(x) {
-    .Call(`_arrow_ArrayData__get_type`, x)
-}
-
-ArrayData__get_length <- function(x) {
-    .Call(`_arrow_ArrayData__get_length`, x)
-}
-
-ArrayData__get_null_count <- function(x) {
-    .Call(`_arrow_ArrayData__get_null_count`, x)
-}
-
-ArrayData__get_offset <- function(x) {
-    .Call(`_arrow_ArrayData__get_offset`, x)
-}
-
-ArrayData__buffers <- function(x) {
-    .Call(`_arrow_ArrayData__buffers`, x)
-}
-
-Buffer__is_mutable <- function(buffer) {
-    .Call(`_arrow_Buffer__is_mutable`, buffer)
-}
-
-Buffer__ZeroPadding <- function(buffer) {
-    invisible(.Call(`_arrow_Buffer__ZeroPadding`, buffer))
-}
-
-Buffer__capacity <- function(buffer) {
-    .Call(`_arrow_Buffer__capacity`, buffer)
-}
-
-Buffer__size <- function(buffer) {
-    .Call(`_arrow_Buffer__size`, buffer)
-}
-
-r___RBuffer__initialize <- function(x) {
-    .Call(`_arrow_r___RBuffer__initialize`, x)
-}
-
-ChunkedArray__length <- function(chunked_array) {
-    .Call(`_arrow_ChunkedArray__length`, chunked_array)
-}
-
-ChunkedArray__null_count <- function(chunked_array) {
-    .Call(`_arrow_ChunkedArray__null_count`, chunked_array)
-}
-
-ChunkedArray__num_chunks <- function(chunked_array) {
-    .Call(`_arrow_ChunkedArray__num_chunks`, chunked_array)
-}
-
-ChunkedArray__chunk <- function(chunked_array, i) {
-    .Call(`_arrow_ChunkedArray__chunk`, chunked_array, i)
-}
-
-ChunkedArray__chunks <- function(chunked_array) {
-    .Call(`_arrow_ChunkedArray__chunks`, chunked_array)
-}
-
-ChunkedArray__type <- function(chunked_array) {
-    .Call(`_arrow_ChunkedArray__type`, chunked_array)
-}
-
-ChunkArray__Slice1 <- function(chunked_array, offset) {
-    .Call(`_arrow_ChunkArray__Slice1`, chunked_array, offset)
-}
-
-ChunkArray__Slice2 <- function(chunked_array, offset, length) {
-    .Call(`_arrow_ChunkArray__Slice2`, chunked_array, offset, length)
-}
-
-Column__length <- function(column) {
-    .Call(`_arrow_Column__length`, column)
-}
-
-Column__null_count <- function(column) {
-    .Call(`_arrow_Column__null_count`, column)
-}
-
-Column__type <- function(column) {
-    .Call(`_arrow_Column__type`, column)
-}
-
-Column__data <- function(column) {
-    .Call(`_arrow_Column__data`, column)
-}
-
-util___Codec__Create <- function(codec) {
-    .Call(`_arrow_util___Codec__Create`, codec)
-}
-
-io___CompressedOutputStream__Make <- function(codec, raw) {
-    .Call(`_arrow_io___CompressedOutputStream__Make`, codec, raw)
-}
-
-io___CompressedInputStream__Make <- function(codec, raw) {
-    .Call(`_arrow_io___CompressedInputStream__Make`, codec, raw)
-}
-
-compute___CastOptions__initialize <- function(allow_int_overflow, allow_time_truncate, allow_float_truncate) {
-    .Call(`_arrow_compute___CastOptions__initialize`, allow_int_overflow, allow_time_truncate, allow_float_truncate)
-}
-
-Array__cast <- function(array, target_type, options) {
-    .Call(`_arrow_Array__cast`, array, target_type, options)
-}
-
-ChunkedArray__cast <- function(chunked_array, target_type, options) {
-    .Call(`_arrow_ChunkedArray__cast`, chunked_array, target_type, options)
-}
-
-RecordBatch__cast <- function(batch, schema, options) {
-    .Call(`_arrow_RecordBatch__cast`, batch, schema, options)
-}
-
-Table__cast <- function(table, schema, options) {
-    .Call(`_arrow_Table__cast`, table, schema, options)
-}
-
-csv___ReadOptions__initialize <- function(options) {
-    .Call(`_arrow_csv___ReadOptions__initialize`, options)
-}
-
-csv___ParseOptions__initialize <- function(options) {
-    .Call(`_arrow_csv___ParseOptions__initialize`, options)
-}
-
-csv___ConvertOptions__initialize <- function(options) {
-    .Call(`_arrow_csv___ConvertOptions__initialize`, options)
-}
-
-csv___TableReader__Make <- function(input, read_options, parse_options, convert_options) {
-    .Call(`_arrow_csv___TableReader__Make`, input, read_options, parse_options, convert_options)
-}
-
-csv___TableReader__Read <- function(table_reader) {
-    .Call(`_arrow_csv___TableReader__Read`, table_reader)
-}
-
-shared_ptr_is_null <- function(xp) {
-    .Call(`_arrow_shared_ptr_is_null`, xp)
-}
-
-unique_ptr_is_null <- function(xp) {
-    .Call(`_arrow_unique_ptr_is_null`, xp)
-}
-
-Int8__initialize <- function() {
-    .Call(`_arrow_Int8__initialize`)
-}
-
-Int16__initialize <- function() {
-    .Call(`_arrow_Int16__initialize`)
-}
-
-Int32__initialize <- function() {
-    .Call(`_arrow_Int32__initialize`)
-}
-
-Int64__initialize <- function() {
-    .Call(`_arrow_Int64__initialize`)
-}
-
-UInt8__initialize <- function() {
-    .Call(`_arrow_UInt8__initialize`)
-}
-
-UInt16__initialize <- function() {
-    .Call(`_arrow_UInt16__initialize`)
-}
-
-UInt32__initialize <- function() {
-    .Call(`_arrow_UInt32__initialize`)
-}
-
-UInt64__initialize <- function() {
-    .Call(`_arrow_UInt64__initialize`)
-}
-
-Float16__initialize <- function() {
-    .Call(`_arrow_Float16__initialize`)
-}
-
-Float32__initialize <- function() {
-    .Call(`_arrow_Float32__initialize`)
-}
-
-Float64__initialize <- function() {
-    .Call(`_arrow_Float64__initialize`)
-}
-
-Boolean__initialize <- function() {
-    .Call(`_arrow_Boolean__initialize`)
-}
-
-Utf8__initialize <- function() {
-    .Call(`_arrow_Utf8__initialize`)
-}
-
-Date32__initialize <- function() {
-    .Call(`_arrow_Date32__initialize`)
-}
-
-Date64__initialize <- function() {
-    .Call(`_arrow_Date64__initialize`)
-}
-
-Null__initialize <- function() {
-    .Call(`_arrow_Null__initialize`)
-}
-
-Decimal128Type__initialize <- function(precision, scale) {
-    .Call(`_arrow_Decimal128Type__initialize`, precision, scale)
-}
-
-FixedSizeBinary__initialize <- function(byte_width) {
-    .Call(`_arrow_FixedSizeBinary__initialize`, byte_width)
-}
-
-Timestamp__initialize1 <- function(unit) {
-    .Call(`_arrow_Timestamp__initialize1`, unit)
-}
-
-Timestamp__initialize2 <- function(unit, timezone) {
-    .Call(`_arrow_Timestamp__initialize2`, unit, timezone)
-}
-
-Time32__initialize <- function(unit) {
-    .Call(`_arrow_Time32__initialize`, unit)
-}
-
-Time64__initialize <- function(unit) {
-    .Call(`_arrow_Time64__initialize`, unit)
-}
-
-list__ <- function(x) {
-    .Call(`_arrow_list__`, x)
-}
-
-struct_ <- function(fields) {
-    .Call(`_arrow_struct_`, fields)
-}
-
-DataType__ToString <- function(type) {
-    .Call(`_arrow_DataType__ToString`, type)
-}
-
-DataType__name <- function(type) {
-    .Call(`_arrow_DataType__name`, type)
-}
-
-DataType__Equals <- function(lhs, rhs) {
-    .Call(`_arrow_DataType__Equals`, lhs, rhs)
-}
-
-DataType__num_children <- function(type) {
-    .Call(`_arrow_DataType__num_children`, type)
-}
-
-DataType__children_pointer <- function(type) {
-    .Call(`_arrow_DataType__children_pointer`, type)
-}
-
-DataType__id <- function(type) {
-    .Call(`_arrow_DataType__id`, type)
-}
-
-schema_ <- function(fields) {
-    .Call(`_arrow_schema_`, fields)
-}
-
-Schema__ToString <- function(s) {
-    .Call(`_arrow_Schema__ToString`, s)
-}
-
-Schema__num_fields <- function(s) {
-    .Call(`_arrow_Schema__num_fields`, s)
-}
-
-Schema__field <- function(s, i) {
-    .Call(`_arrow_Schema__field`, s, i)
-}
-
-Schema__names <- function(schema) {
-    .Call(`_arrow_Schema__names`, schema)
-}
-
-ListType__ToString <- function(type) {
-    .Call(`_arrow_ListType__ToString`, type)
-}
-
-FixedWidthType__bit_width <- function(type) {
-    .Call(`_arrow_FixedWidthType__bit_width`, type)
-}
-
-DateType__unit <- function(type) {
-    .Call(`_arrow_DateType__unit`, type)
-}
-
-TimeType__unit <- function(type) {
-    .Call(`_arrow_TimeType__unit`, type)
-}
-
-DecimalType__precision <- function(type) {
-    .Call(`_arrow_DecimalType__precision`, type)
-}
-
-DecimalType__scale <- function(type) {
-    .Call(`_arrow_DecimalType__scale`, type)
-}
-
-TimestampType__timezone <- function(type) {
-    .Call(`_arrow_TimestampType__timezone`, type)
-}
-
-TimestampType__unit <- function(type) {
-    .Call(`_arrow_TimestampType__unit`, type)
-}
-
-DictionaryType__initialize <- function(index_type, value_type, ordered) {
-    .Call(`_arrow_DictionaryType__initialize`, index_type, value_type, ordered)
-}
-
-DictionaryType__index_type <- function(type) {
-    .Call(`_arrow_DictionaryType__index_type`, type)
-}
-
-DictionaryType__value_type <- function(type) {
-    .Call(`_arrow_DictionaryType__value_type`, type)
-}
-
-DictionaryType__name <- function(type) {
-    .Call(`_arrow_DictionaryType__name`, type)
-}
-
-DictionaryType__ordered <- function(type) {
-    .Call(`_arrow_DictionaryType__ordered`, type)
-}
-
-ipc___feather___TableWriter__SetDescription <- function(writer, description) {
-    invisible(.Call(`_arrow_ipc___feather___TableWriter__SetDescription`, writer, description))
-}
-
-ipc___feather___TableWriter__SetNumRows <- function(writer, num_rows) {
-    invisible(.Call(`_arrow_ipc___feather___TableWriter__SetNumRows`, writer, num_rows))
-}
-
-ipc___feather___TableWriter__Append <- function(writer, name, values) {
-    invisible(.Call(`_arrow_ipc___feather___TableWriter__Append`, writer, name, values))
-}
-
-ipc___feather___TableWriter__Finalize <- function(writer) {
-    invisible(.Call(`_arrow_ipc___feather___TableWriter__Finalize`, writer))
-}
-
-ipc___feather___TableWriter__Open <- function(stream) {
-    .Call(`_arrow_ipc___feather___TableWriter__Open`, stream)
-}
-
-ipc___TableWriter__RecordBatch__WriteFeather <- function(writer, batch) {
-    invisible(.Call(`_arrow_ipc___TableWriter__RecordBatch__WriteFeather`, writer, batch))
-}
-
-ipc___feather___TableReader__GetDescription <- function(reader) {
-    .Call(`_arrow_ipc___feather___TableReader__GetDescription`, reader)
-}
-
-ipc___feather___TableReader__HasDescription <- function(reader) {
-    .Call(`_arrow_ipc___feather___TableReader__HasDescription`, reader)
-}
-
-ipc___feather___TableReader__version <- function(reader) {
-    .Call(`_arrow_ipc___feather___TableReader__version`, reader)
-}
-
-ipc___feather___TableReader__num_rows <- function(reader) {
-    .Call(`_arrow_ipc___feather___TableReader__num_rows`, reader)
-}
-
-ipc___feather___TableReader__num_columns <- function(reader) {
-    .Call(`_arrow_ipc___feather___TableReader__num_columns`, reader)
-}
-
-ipc___feather___TableReader__GetColumnName <- function(reader, i) {
-    .Call(`_arrow_ipc___feather___TableReader__GetColumnName`, reader, i)
-}
-
-ipc___feather___TableReader__GetColumn <- function(reader, i) {
-    .Call(`_arrow_ipc___feather___TableReader__GetColumn`, reader, i)
-}
-
-ipc___feather___TableReader__Read <- function(reader, columns) {
-    .Call(`_arrow_ipc___feather___TableReader__Read`, reader, columns)
-}
-
-ipc___feather___TableReader__Open <- function(stream) {
-    .Call(`_arrow_ipc___feather___TableReader__Open`, stream)
-}
-
-Field__initialize <- function(name, field, nullable = TRUE) {
-    .Call(`_arrow_Field__initialize`, name, field, nullable)
-}
-
-Field__ToString <- function(field) {
-    .Call(`_arrow_Field__ToString`, field)
-}
-
-Field__name <- function(field) {
-    .Call(`_arrow_Field__name`, field)
-}
-
-Field__Equals <- function(field, other) {
-    .Call(`_arrow_Field__Equals`, field, other)
-}
-
-Field__nullable <- function(field) {
-    .Call(`_arrow_Field__nullable`, field)
-}
-
-Field__type <- function(field) {
-    .Call(`_arrow_Field__type`, field)
-}
-
-io___Readable__Read <- function(x, nbytes) {
-    .Call(`_arrow_io___Readable__Read`, x, nbytes)
-}
-
-io___InputStream__Close <- function(x) {
-    invisible(.Call(`_arrow_io___InputStream__Close`, x))
-}
-
-io___OutputStream__Close <- function(x) {
-    invisible(.Call(`_arrow_io___OutputStream__Close`, x))
-}
-
-io___RandomAccessFile__GetSize <- function(x) {
-    .Call(`_arrow_io___RandomAccessFile__GetSize`, x)
-}
-
-io___RandomAccessFile__supports_zero_copy <- function(x) {
-    .Call(`_arrow_io___RandomAccessFile__supports_zero_copy`, x)
-}
-
-io___RandomAccessFile__Seek <- function(x, position) {
-    invisible(.Call(`_arrow_io___RandomAccessFile__Seek`, x, position))
-}
-
-io___RandomAccessFile__Tell <- function(x) {
-    .Call(`_arrow_io___RandomAccessFile__Tell`, x)
-}
-
-io___MemoryMappedFile__Create <- function(path, size) {
-    .Call(`_arrow_io___MemoryMappedFile__Create`, path, size)
-}
-
-io___MemoryMappedFile__Open <- function(path, mode) {
-    .Call(`_arrow_io___MemoryMappedFile__Open`, path, mode)
-}
-
-io___MemoryMappedFile__Resize <- function(x, size) {
-    invisible(.Call(`_arrow_io___MemoryMappedFile__Resize`, x, size))
-}
-
-io___ReadableFile__Open <- function(path) {
-    .Call(`_arrow_io___ReadableFile__Open`, path)
-}
-
-io___BufferReader__initialize <- function(buffer) {
-    .Call(`_arrow_io___BufferReader__initialize`, buffer)
-}
-
-io___Writable__write <- function(stream, buf) {
-    invisible(.Call(`_arrow_io___Writable__write`, stream, buf))
-}
-
-io___OutputStream__Tell <- function(stream) {
-    .Call(`_arrow_io___OutputStream__Tell`, stream)
-}
-
-io___FileOutputStream__Open <- function(path) {
-    .Call(`_arrow_io___FileOutputStream__Open`, path)
-}
-
-io___BufferOutputStream__Create <- function(initial_capacity) {
-    .Call(`_arrow_io___BufferOutputStream__Create`, initial_capacity)
-}
-
-io___BufferOutputStream__capacity <- function(stream) {
-    .Call(`_arrow_io___BufferOutputStream__capacity`, stream)
-}
-
-io___BufferOutputStream__Finish <- function(stream) {
-    .Call(`_arrow_io___BufferOutputStream__Finish`, stream)
-}
-
-io___BufferOutputStream__Tell <- function(stream) {
-    .Call(`_arrow_io___BufferOutputStream__Tell`, stream)
-}
-
-io___BufferOutputStream__Write <- function(stream, bytes) {
-    invisible(.Call(`_arrow_io___BufferOutputStream__Write`, stream, bytes))
-}
-
-io___MockOutputStream__initialize <- function() {
-    .Call(`_arrow_io___MockOutputStream__initialize`)
-}
-
-io___MockOutputStream__GetExtentBytesWritten <- function(stream) {
-    .Call(`_arrow_io___MockOutputStream__GetExtentBytesWritten`, stream)
-}
-
-io___FixedSizeBufferWriter__initialize <- function(buffer) {
-    .Call(`_arrow_io___FixedSizeBufferWriter__initialize`, buffer)
-}
-
-MemoryPool__default <- function() {
-    .Call(`_arrow_MemoryPool__default`)
-}
-
-MemoryPool__bytes_allocated <- function(pool) {
-    .Call(`_arrow_MemoryPool__bytes_allocated`, pool)
-}
-
-MemoryPool__max_memory <- function(pool) {
-    .Call(`_arrow_MemoryPool__max_memory`, pool)
-}
-
-ipc___Message__body_length <- function(message) {
-    .Call(`_arrow_ipc___Message__body_length`, message)
-}
-
-ipc___Message__metadata <- function(message) {
-    .Call(`_arrow_ipc___Message__metadata`, message)
-}
-
-ipc___Message__body <- function(message) {
-    .Call(`_arrow_ipc___Message__body`, message)
-}
-
-ipc___Message__Verify <- function(message) {
-    .Call(`_arrow_ipc___Message__Verify`, message)
-}
-
-ipc___Message__type <- function(message) {
-    .Call(`_arrow_ipc___Message__type`, message)
-}
-
-ipc___Message__Equals <- function(x, y) {
-    .Call(`_arrow_ipc___Message__Equals`, x, y)
-}
-
-ipc___ReadRecordBatch__Message__Schema <- function(message, schema) {
-    .Call(`_arrow_ipc___ReadRecordBatch__Message__Schema`, message, schema)
-}
-
-ipc___ReadSchema_InputStream <- function(stream) {
-    .Call(`_arrow_ipc___ReadSchema_InputStream`, stream)
-}
-
-ipc___MessageReader__Open <- function(stream) {
-    .Call(`_arrow_ipc___MessageReader__Open`, stream)
-}
-
-ipc___MessageReader__ReadNextMessage <- function(reader) {
-    .Call(`_arrow_ipc___MessageReader__ReadNextMessage`, reader)
-}
-
-ipc___ReadMessage <- function(stream) {
-    .Call(`_arrow_ipc___ReadMessage`, stream)
-}
-
-read_parquet_file <- function(filename) {
-    .Call(`_arrow_read_parquet_file`, filename)
-}
-
-RecordBatch__num_columns <- function(x) {
-    .Call(`_arrow_RecordBatch__num_columns`, x)
-}
-
-RecordBatch__num_rows <- function(x) {
-    .Call(`_arrow_RecordBatch__num_rows`, x)
-}
-
-RecordBatch__schema <- function(x) {
-    .Call(`_arrow_RecordBatch__schema`, x)
-}
-
-RecordBatch__columns <- function(batch) {
-    .Call(`_arrow_RecordBatch__columns`, batch)
-}
-
-RecordBatch__column <- function(batch, i) {
-    .Call(`_arrow_RecordBatch__column`, batch, i)
-}
-
-RecordBatch__Equals <- function(self, other) {
-    .Call(`_arrow_RecordBatch__Equals`, self, other)
-}
-
-RecordBatch__RemoveColumn <- function(batch, i) {
-    .Call(`_arrow_RecordBatch__RemoveColumn`, batch, i)
-}
-
-RecordBatch__column_name <- function(batch, i) {
-    .Call(`_arrow_RecordBatch__column_name`, batch, i)
-}
-
-RecordBatch__names <- function(batch) {
-    .Call(`_arrow_RecordBatch__names`, batch)
-}
-
-RecordBatch__Slice1 <- function(self, offset) {
-    .Call(`_arrow_RecordBatch__Slice1`, self, offset)
-}
-
-RecordBatch__Slice2 <- function(self, offset, length) {
-    .Call(`_arrow_RecordBatch__Slice2`, self, offset, length)
-}
-
-ipc___SerializeRecordBatch__Raw <- function(batch) {
-    .Call(`_arrow_ipc___SerializeRecordBatch__Raw`, batch)
-}
-
-ipc___ReadRecordBatch__InputStream__Schema <- function(stream, schema) {
-    .Call(`_arrow_ipc___ReadRecordBatch__InputStream__Schema`, stream, schema)
-}
-
-RecordBatch__from_arrays <- function(schema_sxp, lst) {
-    .Call(`_arrow_RecordBatch__from_arrays`, schema_sxp, lst)
-}
-
-RecordBatchReader__schema <- function(reader) {
-    .Call(`_arrow_RecordBatchReader__schema`, reader)
-}
-
-RecordBatchReader__ReadNext <- function(reader) {
-    .Call(`_arrow_RecordBatchReader__ReadNext`, reader)
-}
-
-ipc___RecordBatchStreamReader__Open <- function(stream) {
-    .Call(`_arrow_ipc___RecordBatchStreamReader__Open`, stream)
-}
-
-ipc___RecordBatchStreamReader__batches <- function(reader) {
-    .Call(`_arrow_ipc___RecordBatchStreamReader__batches`, reader)
-}
-
-ipc___RecordBatchFileReader__schema <- function(reader) {
-    .Call(`_arrow_ipc___RecordBatchFileReader__schema`, reader)
-}
-
-ipc___RecordBatchFileReader__num_record_batches <- function(reader) {
-    .Call(`_arrow_ipc___RecordBatchFileReader__num_record_batches`, reader)
-}
-
-ipc___RecordBatchFileReader__ReadRecordBatch <- function(reader, i) {
-    .Call(`_arrow_ipc___RecordBatchFileReader__ReadRecordBatch`, reader, i)
-}
-
-ipc___RecordBatchFileReader__Open <- function(file) {
-    .Call(`_arrow_ipc___RecordBatchFileReader__Open`, file)
-}
-
-Table__from_RecordBatchFileReader <- function(reader) {
-    .Call(`_arrow_Table__from_RecordBatchFileReader`, reader)
-}
-
-Table__from_RecordBatchStreamReader <- function(reader) {
-    .Call(`_arrow_Table__from_RecordBatchStreamReader`, reader)
-}
-
-ipc___RecordBatchFileReader__batches <- function(reader) {
-    .Call(`_arrow_ipc___RecordBatchFileReader__batches`, reader)
-}
-
-ipc___RecordBatchWriter__WriteRecordBatch <- function(batch_writer, batch) {
-    invisible(.Call(`_arrow_ipc___RecordBatchWriter__WriteRecordBatch`, batch_writer, batch))
-}
-
-ipc___RecordBatchWriter__WriteTable <- function(batch_writer, table) {
-    invisible(.Call(`_arrow_ipc___RecordBatchWriter__WriteTable`, batch_writer, table))
-}
-
-ipc___RecordBatchWriter__Close <- function(batch_writer) {
-    invisible(.Call(`_arrow_ipc___RecordBatchWriter__Close`, batch_writer))
-}
-
-ipc___RecordBatchFileWriter__Open <- function(stream, schema) {
-    .Call(`_arrow_ipc___RecordBatchFileWriter__Open`, stream, schema)
-}
-
-ipc___RecordBatchStreamWriter__Open <- function(stream, schema) {
-    .Call(`_arrow_ipc___RecordBatchStreamWriter__Open`, stream, schema)
-}
-
-Table__num_columns <- function(x) {
-    .Call(`_arrow_Table__num_columns`, x)
-}
-
-Table__num_rows <- function(x) {
-    .Call(`_arrow_Table__num_rows`, x)
-}
-
-Table__schema <- function(x) {
-    .Call(`_arrow_Table__schema`, x)
-}
-
-Table__column <- function(table, i) {
-    .Call(`_arrow_Table__column`, table, i)
-}
-
-Table__columns <- function(table) {
-    .Call(`_arrow_Table__columns`, table)
-}
-
-Table__from_dots <- function(lst, schema_sxp) {
-    .Call(`_arrow_Table__from_dots`, lst, schema_sxp)
-}
-
-#' View and manage the capacity of the global thread pool
-#'
-#' `GetCpuThreadPoolCapacity()` returns the number of worker threads in the
-#' thread pool to which
-#' Arrow dispatches various CPU-bound tasks. This is an ideal number,
-#' not necessarily the exact number of threads at a given point in time.
-#' You can change this number using `SetCpuThreadPoolCapacity()`.
-#'
-#' @param threads the number of worker threads in the thread pool to which
-#' Arrow dispatches various CPU-bound tasks.
-#'
-#' @return `GetCpuThreadPoolCapacity()` returns the number of worker threads.
-#' `SetCpuThreadPoolCapacity()` returns nothing.
-#' @export
-#' @name threadpool
-GetCpuThreadPoolCapacity <- function() {
-    .Call(`_arrow_GetCpuThreadPoolCapacity`)
-}
-
-#' @rdname threadpool
-#' @export
-SetCpuThreadPoolCapacity <- function(threads) {
-    invisible(.Call(`_arrow_SetCpuThreadPoolCapacity`, threads))
-}
-
diff --git a/r/R/arrowExports.R b/r/R/arrowExports.R
new file mode 100644
index 0000000..6359f90
--- /dev/null
+++ b/r/R/arrowExports.R
@@ -0,0 +1,835 @@
+# Generated by using data-raw/codegen.R -> do not edit by hand
+
+Array__Slice1 <- function(array, offset){
+    .Call(`_arrow_Array__Slice1` , array, offset)
+}
+
+Array__Slice2 <- function(array, offset, length){
+    .Call(`_arrow_Array__Slice2` , array, offset, length)
+}
+
+Array__IsNull <- function(x, i){
+    .Call(`_arrow_Array__IsNull` , x, i)
+}
+
+Array__IsValid <- function(x, i){
+    .Call(`_arrow_Array__IsValid` , x, i)
+}
+
+Array__length <- function(x){
+    .Call(`_arrow_Array__length` , x)
+}
+
+Array__offset <- function(x){
+    .Call(`_arrow_Array__offset` , x)
+}
+
+Array__null_count <- function(x){
+    .Call(`_arrow_Array__null_count` , x)
+}
+
+Array__type <- function(x){
+    .Call(`_arrow_Array__type` , x)
+}
+
+Array__ToString <- function(x){
+    .Call(`_arrow_Array__ToString` , x)
+}
+
+Array__type_id <- function(x){
+    .Call(`_arrow_Array__type_id` , x)
+}
+
+Array__Equals <- function(lhs, rhs){
+    .Call(`_arrow_Array__Equals` , lhs, rhs)
+}
+
+Array__ApproxEquals <- function(lhs, rhs){
+    .Call(`_arrow_Array__ApproxEquals` , lhs, rhs)
+}
+
+Array__data <- function(array){
+    .Call(`_arrow_Array__data` , array)
+}
+
+Array__RangeEquals <- function(self, other, start_idx, end_idx, other_start_idx){
+    .Call(`_arrow_Array__RangeEquals` , self, other, start_idx, end_idx, other_start_idx)
+}
+
+Array__Mask <- function(array){
+    .Call(`_arrow_Array__Mask` , array)
+}
+
+DictionaryArray__indices <- function(array){
+    .Call(`_arrow_DictionaryArray__indices` , array)
+}
+
+DictionaryArray__dictionary <- function(array){
+    .Call(`_arrow_DictionaryArray__dictionary` , array)
+}
+
+Array__as_vector <- function(array){
+    .Call(`_arrow_Array__as_vector` , array)
+}
+
+ChunkedArray__as_vector <- function(chunked_array){
+    .Call(`_arrow_ChunkedArray__as_vector` , chunked_array)
+}
+
+RecordBatch__to_dataframe <- function(batch, use_threads){
+    .Call(`_arrow_RecordBatch__to_dataframe` , batch, use_threads)
+}
+
+Table__to_dataframe <- function(table, use_threads){
+    .Call(`_arrow_Table__to_dataframe` , table, use_threads)
+}
+
+Array__infer_type <- function(x){
+    .Call(`_arrow_Array__infer_type` , x)
+}
+
+Array__from_vector <- function(x, s_type){
+    .Call(`_arrow_Array__from_vector` , x, s_type)
+}
+
+ChunkedArray__from_list <- function(chunks, s_type){
+    .Call(`_arrow_ChunkedArray__from_list` , chunks, s_type)
+}
+
+ArrayData__get_type <- function(x){
+    .Call(`_arrow_ArrayData__get_type` , x)
+}
+
+ArrayData__get_length <- function(x){
+    .Call(`_arrow_ArrayData__get_length` , x)
+}
+
+ArrayData__get_null_count <- function(x){
+    .Call(`_arrow_ArrayData__get_null_count` , x)
+}
+
+ArrayData__get_offset <- function(x){
+    .Call(`_arrow_ArrayData__get_offset` , x)
+}
+
+ArrayData__buffers <- function(x){
+    .Call(`_arrow_ArrayData__buffers` , x)
+}
+
+Buffer__is_mutable <- function(buffer){
+    .Call(`_arrow_Buffer__is_mutable` , buffer)
+}
+
+Buffer__ZeroPadding <- function(buffer){
+    invisible(.Call(`_arrow_Buffer__ZeroPadding` , buffer))
+}
+
+Buffer__capacity <- function(buffer){
+    .Call(`_arrow_Buffer__capacity` , buffer)
+}
+
+Buffer__size <- function(buffer){
+    .Call(`_arrow_Buffer__size` , buffer)
+}
+
+r___RBuffer__initialize <- function(x){
+    .Call(`_arrow_r___RBuffer__initialize` , x)
+}
+
+ChunkedArray__length <- function(chunked_array){
+    .Call(`_arrow_ChunkedArray__length` , chunked_array)
+}
+
+ChunkedArray__null_count <- function(chunked_array){
+    .Call(`_arrow_ChunkedArray__null_count` , chunked_array)
+}
+
+ChunkedArray__num_chunks <- function(chunked_array){
+    .Call(`_arrow_ChunkedArray__num_chunks` , chunked_array)
+}
+
+ChunkedArray__chunk <- function(chunked_array, i){
+    .Call(`_arrow_ChunkedArray__chunk` , chunked_array, i)
+}
+
+ChunkedArray__chunks <- function(chunked_array){
+    .Call(`_arrow_ChunkedArray__chunks` , chunked_array)
+}
+
+ChunkedArray__type <- function(chunked_array){
+    .Call(`_arrow_ChunkedArray__type` , chunked_array)
+}
+
+ChunkArray__Slice1 <- function(chunked_array, offset){
+    .Call(`_arrow_ChunkArray__Slice1` , chunked_array, offset)
+}
+
+ChunkArray__Slice2 <- function(chunked_array, offset, length){
+    .Call(`_arrow_ChunkArray__Slice2` , chunked_array, offset, length)
+}
+
+Column__length <- function(column){
+    .Call(`_arrow_Column__length` , column)
+}
+
+Column__null_count <- function(column){
+    .Call(`_arrow_Column__null_count` , column)
+}
+
+Column__type <- function(column){
+    .Call(`_arrow_Column__type` , column)
+}
+
+Column__data <- function(column){
+    .Call(`_arrow_Column__data` , column)
+}
+
+util___Codec__Create <- function(codec){
+    .Call(`_arrow_util___Codec__Create` , codec)
+}
+
+io___CompressedOutputStream__Make <- function(codec, raw){
+    .Call(`_arrow_io___CompressedOutputStream__Make` , codec, raw)
+}
+
+io___CompressedInputStream__Make <- function(codec, raw){
+    .Call(`_arrow_io___CompressedInputStream__Make` , codec, raw)
+}
+
+compute___CastOptions__initialize <- function(allow_int_overflow, allow_time_truncate, allow_float_truncate){
+    .Call(`_arrow_compute___CastOptions__initialize` , allow_int_overflow, allow_time_truncate, allow_float_truncate)
+}
+
+Array__cast <- function(array, target_type, options){
+    .Call(`_arrow_Array__cast` , array, target_type, options)
+}
+
+ChunkedArray__cast <- function(chunked_array, target_type, options){
+    .Call(`_arrow_ChunkedArray__cast` , chunked_array, target_type, options)
+}
+
+RecordBatch__cast <- function(batch, schema, options){
+    .Call(`_arrow_RecordBatch__cast` , batch, schema, options)
+}
+
+Table__cast <- function(table, schema, options){
+    .Call(`_arrow_Table__cast` , table, schema, options)
+}
+
+csv___ReadOptions__initialize <- function(options){
+    .Call(`_arrow_csv___ReadOptions__initialize` , options)
+}
+
+csv___ParseOptions__initialize <- function(options){
+    .Call(`_arrow_csv___ParseOptions__initialize` , options)
+}
+
+csv___ConvertOptions__initialize <- function(options){
+    .Call(`_arrow_csv___ConvertOptions__initialize` , options)
+}
+
+csv___TableReader__Make <- function(input, read_options, parse_options, convert_options){
+    .Call(`_arrow_csv___TableReader__Make` , input, read_options, parse_options, convert_options)
+}
+
+csv___TableReader__Read <- function(table_reader){
+    .Call(`_arrow_csv___TableReader__Read` , table_reader)
+}
+
+shared_ptr_is_null <- function(xp){
+    .Call(`_arrow_shared_ptr_is_null` , xp)
+}
+
+unique_ptr_is_null <- function(xp){
+    .Call(`_arrow_unique_ptr_is_null` , xp)
+}
+
+Int8__initialize <- function(){
+    .Call(`_arrow_Int8__initialize` )
+}
+
+Int16__initialize <- function(){
+    .Call(`_arrow_Int16__initialize` )
+}
+
+Int32__initialize <- function(){
+    .Call(`_arrow_Int32__initialize` )
+}
+
+Int64__initialize <- function(){
+    .Call(`_arrow_Int64__initialize` )
+}
+
+UInt8__initialize <- function(){
+    .Call(`_arrow_UInt8__initialize` )
+}
+
+UInt16__initialize <- function(){
+    .Call(`_arrow_UInt16__initialize` )
+}
+
+UInt32__initialize <- function(){
+    .Call(`_arrow_UInt32__initialize` )
+}
+
+UInt64__initialize <- function(){
+    .Call(`_arrow_UInt64__initialize` )
+}
+
+Float16__initialize <- function(){
+    .Call(`_arrow_Float16__initialize` )
+}
+
+Float32__initialize <- function(){
+    .Call(`_arrow_Float32__initialize` )
+}
+
+Float64__initialize <- function(){
+    .Call(`_arrow_Float64__initialize` )
+}
+
+Boolean__initialize <- function(){
+    .Call(`_arrow_Boolean__initialize` )
+}
+
+Utf8__initialize <- function(){
+    .Call(`_arrow_Utf8__initialize` )
+}
+
+Date32__initialize <- function(){
+    .Call(`_arrow_Date32__initialize` )
+}
+
+Date64__initialize <- function(){
+    .Call(`_arrow_Date64__initialize` )
+}
+
+Null__initialize <- function(){
+    .Call(`_arrow_Null__initialize` )
+}
+
+Decimal128Type__initialize <- function(precision, scale){
+    .Call(`_arrow_Decimal128Type__initialize` , precision, scale)
+}
+
+FixedSizeBinary__initialize <- function(byte_width){
+    .Call(`_arrow_FixedSizeBinary__initialize` , byte_width)
+}
+
+Timestamp__initialize1 <- function(unit){
+    .Call(`_arrow_Timestamp__initialize1` , unit)
+}
+
+Timestamp__initialize2 <- function(unit, timezone){
+    .Call(`_arrow_Timestamp__initialize2` , unit, timezone)
+}
+
+Time32__initialize <- function(unit){
+    .Call(`_arrow_Time32__initialize` , unit)
+}
+
+Time64__initialize <- function(unit){
+    .Call(`_arrow_Time64__initialize` , unit)
+}
+
+list__ <- function(x){
+    .Call(`_arrow_list__` , x)
+}
+
+struct_ <- function(fields){
+    .Call(`_arrow_struct_` , fields)
+}
+
+DataType__ToString <- function(type){
+    .Call(`_arrow_DataType__ToString` , type)
+}
+
+DataType__name <- function(type){
+    .Call(`_arrow_DataType__name` , type)
+}
+
+DataType__Equals <- function(lhs, rhs){
+    .Call(`_arrow_DataType__Equals` , lhs, rhs)
+}
+
+DataType__num_children <- function(type){
+    .Call(`_arrow_DataType__num_children` , type)
+}
+
+DataType__children_pointer <- function(type){
+    .Call(`_arrow_DataType__children_pointer` , type)
+}
+
+DataType__id <- function(type){
+    .Call(`_arrow_DataType__id` , type)
+}
+
+schema_ <- function(fields){
+    .Call(`_arrow_schema_` , fields)
+}
+
+Schema__ToString <- function(s){
+    .Call(`_arrow_Schema__ToString` , s)
+}
+
+Schema__num_fields <- function(s){
+    .Call(`_arrow_Schema__num_fields` , s)
+}
+
+Schema__field <- function(s, i){
+    .Call(`_arrow_Schema__field` , s, i)
+}
+
+Schema__names <- function(schema){
+    .Call(`_arrow_Schema__names` , schema)
+}
+
+ListType__ToString <- function(type){
+    .Call(`_arrow_ListType__ToString` , type)
+}
+
+FixedWidthType__bit_width <- function(type){
+    .Call(`_arrow_FixedWidthType__bit_width` , type)
+}
+
+DateType__unit <- function(type){
+    .Call(`_arrow_DateType__unit` , type)
+}
+
+TimeType__unit <- function(type){
+    .Call(`_arrow_TimeType__unit` , type)
+}
+
+DecimalType__precision <- function(type){
+    .Call(`_arrow_DecimalType__precision` , type)
+}
+
+DecimalType__scale <- function(type){
+    .Call(`_arrow_DecimalType__scale` , type)
+}
+
+TimestampType__timezone <- function(type){
+    .Call(`_arrow_TimestampType__timezone` , type)
+}
+
+TimestampType__unit <- function(type){
+    .Call(`_arrow_TimestampType__unit` , type)
+}
+
+DictionaryType__initialize <- function(index_type, value_type, ordered){
+    .Call(`_arrow_DictionaryType__initialize` , index_type, value_type, ordered)
+}
+
+DictionaryType__index_type <- function(type){
+    .Call(`_arrow_DictionaryType__index_type` , type)
+}
+
+DictionaryType__value_type <- function(type){
+    .Call(`_arrow_DictionaryType__value_type` , type)
+}
+
+DictionaryType__name <- function(type){
+    .Call(`_arrow_DictionaryType__name` , type)
+}
+
+DictionaryType__ordered <- function(type){
+    .Call(`_arrow_DictionaryType__ordered` , type)
+}
+
+ipc___feather___TableWriter__SetDescription <- function(writer, description){
+    invisible(.Call(`_arrow_ipc___feather___TableWriter__SetDescription` , writer, description))
+}
+
+ipc___feather___TableWriter__SetNumRows <- function(writer, num_rows){
+    invisible(.Call(`_arrow_ipc___feather___TableWriter__SetNumRows` , writer, num_rows))
+}
+
+ipc___feather___TableWriter__Append <- function(writer, name, values){
+    invisible(.Call(`_arrow_ipc___feather___TableWriter__Append` , writer, name, values))
+}
+
+ipc___feather___TableWriter__Finalize <- function(writer){
+    invisible(.Call(`_arrow_ipc___feather___TableWriter__Finalize` , writer))
+}
+
+ipc___feather___TableWriter__Open <- function(stream){
+    .Call(`_arrow_ipc___feather___TableWriter__Open` , stream)
+}
+
+ipc___TableWriter__RecordBatch__WriteFeather <- function(writer, batch){
+    invisible(.Call(`_arrow_ipc___TableWriter__RecordBatch__WriteFeather` , writer, batch))
+}
+
+ipc___feather___TableReader__GetDescription <- function(reader){
+    .Call(`_arrow_ipc___feather___TableReader__GetDescription` , reader)
+}
+
+ipc___feather___TableReader__HasDescription <- function(reader){
+    .Call(`_arrow_ipc___feather___TableReader__HasDescription` , reader)
+}
+
+ipc___feather___TableReader__version <- function(reader){
+    .Call(`_arrow_ipc___feather___TableReader__version` , reader)
+}
+
+ipc___feather___TableReader__num_rows <- function(reader){
+    .Call(`_arrow_ipc___feather___TableReader__num_rows` , reader)
+}
+
+ipc___feather___TableReader__num_columns <- function(reader){
+    .Call(`_arrow_ipc___feather___TableReader__num_columns` , reader)
+}
+
+ipc___feather___TableReader__GetColumnName <- function(reader, i){
+    .Call(`_arrow_ipc___feather___TableReader__GetColumnName` , reader, i)
+}
+
+ipc___feather___TableReader__GetColumn <- function(reader, i){
+    .Call(`_arrow_ipc___feather___TableReader__GetColumn` , reader, i)
+}
+
+ipc___feather___TableReader__Read <- function(reader, columns){
+    .Call(`_arrow_ipc___feather___TableReader__Read` , reader, columns)
+}
+
+ipc___feather___TableReader__Open <- function(stream){
+    .Call(`_arrow_ipc___feather___TableReader__Open` , stream)
+}
+
+Field__initialize <- function(name, field, nullable){
+    .Call(`_arrow_Field__initialize` , name, field, nullable)
+}
+
+Field__ToString <- function(field){
+    .Call(`_arrow_Field__ToString` , field)
+}
+
+Field__name <- function(field){
+    .Call(`_arrow_Field__name` , field)
+}
+
+Field__Equals <- function(field, other){
+    .Call(`_arrow_Field__Equals` , field, other)
+}
+
+Field__nullable <- function(field){
+    .Call(`_arrow_Field__nullable` , field)
+}
+
+Field__type <- function(field){
+    .Call(`_arrow_Field__type` , field)
+}
+
+io___Readable__Read <- function(x, nbytes){
+    .Call(`_arrow_io___Readable__Read` , x, nbytes)
+}
+
+io___InputStream__Close <- function(x){
+    invisible(.Call(`_arrow_io___InputStream__Close` , x))
+}
+
+io___OutputStream__Close <- function(x){
+    invisible(.Call(`_arrow_io___OutputStream__Close` , x))
+}
+
+io___RandomAccessFile__GetSize <- function(x){
+    .Call(`_arrow_io___RandomAccessFile__GetSize` , x)
+}
+
+io___RandomAccessFile__supports_zero_copy <- function(x){
+    .Call(`_arrow_io___RandomAccessFile__supports_zero_copy` , x)
+}
+
+io___RandomAccessFile__Seek <- function(x, position){
+    invisible(.Call(`_arrow_io___RandomAccessFile__Seek` , x, position))
+}
+
+io___RandomAccessFile__Tell <- function(x){
+    .Call(`_arrow_io___RandomAccessFile__Tell` , x)
+}
+
+io___MemoryMappedFile__Create <- function(path, size){
+    .Call(`_arrow_io___MemoryMappedFile__Create` , path, size)
+}
+
+io___MemoryMappedFile__Open <- function(path, mode){
+    .Call(`_arrow_io___MemoryMappedFile__Open` , path, mode)
+}
+
+io___MemoryMappedFile__Resize <- function(x, size){
+    invisible(.Call(`_arrow_io___MemoryMappedFile__Resize` , x, size))
+}
+
+io___ReadableFile__Open <- function(path){
+    .Call(`_arrow_io___ReadableFile__Open` , path)
+}
+
+io___BufferReader__initialize <- function(buffer){
+    .Call(`_arrow_io___BufferReader__initialize` , buffer)
+}
+
+io___Writable__write <- function(stream, buf){
+    invisible(.Call(`_arrow_io___Writable__write` , stream, buf))
+}
+
+io___OutputStream__Tell <- function(stream){
+    .Call(`_arrow_io___OutputStream__Tell` , stream)
+}
+
+io___FileOutputStream__Open <- function(path){
+    .Call(`_arrow_io___FileOutputStream__Open` , path)
+}
+
+io___BufferOutputStream__Create <- function(initial_capacity){
+    .Call(`_arrow_io___BufferOutputStream__Create` , initial_capacity)
+}
+
+io___BufferOutputStream__capacity <- function(stream){
+    .Call(`_arrow_io___BufferOutputStream__capacity` , stream)
+}
+
+io___BufferOutputStream__Finish <- function(stream){
+    .Call(`_arrow_io___BufferOutputStream__Finish` , stream)
+}
+
+io___BufferOutputStream__Tell <- function(stream){
+    .Call(`_arrow_io___BufferOutputStream__Tell` , stream)
+}
+
+io___BufferOutputStream__Write <- function(stream, bytes){
+    invisible(.Call(`_arrow_io___BufferOutputStream__Write` , stream, bytes))
+}
+
+io___MockOutputStream__initialize <- function(){
+    .Call(`_arrow_io___MockOutputStream__initialize` )
+}
+
+io___MockOutputStream__GetExtentBytesWritten <- function(stream){
+    .Call(`_arrow_io___MockOutputStream__GetExtentBytesWritten` , stream)
+}
+
+io___FixedSizeBufferWriter__initialize <- function(buffer){
+    .Call(`_arrow_io___FixedSizeBufferWriter__initialize` , buffer)
+}
+
+MemoryPool__default <- function(){
+    .Call(`_arrow_MemoryPool__default` )
+}
+
+MemoryPool__bytes_allocated <- function(pool){
+    .Call(`_arrow_MemoryPool__bytes_allocated` , pool)
+}
+
+MemoryPool__max_memory <- function(pool){
+    .Call(`_arrow_MemoryPool__max_memory` , pool)
+}
+
+ipc___Message__body_length <- function(message){
+    .Call(`_arrow_ipc___Message__body_length` , message)
+}
+
+ipc___Message__metadata <- function(message){
+    .Call(`_arrow_ipc___Message__metadata` , message)
+}
+
+ipc___Message__body <- function(message){
+    .Call(`_arrow_ipc___Message__body` , message)
+}
+
+ipc___Message__Verify <- function(message){
+    .Call(`_arrow_ipc___Message__Verify` , message)
+}
+
+ipc___Message__type <- function(message){
+    .Call(`_arrow_ipc___Message__type` , message)
+}
+
+ipc___Message__Equals <- function(x, y){
+    .Call(`_arrow_ipc___Message__Equals` , x, y)
+}
+
+ipc___ReadRecordBatch__Message__Schema <- function(message, schema){
+    .Call(`_arrow_ipc___ReadRecordBatch__Message__Schema` , message, schema)
+}
+
+ipc___ReadSchema_InputStream <- function(stream){
+    .Call(`_arrow_ipc___ReadSchema_InputStream` , stream)
+}
+
+ipc___MessageReader__Open <- function(stream){
+    .Call(`_arrow_ipc___MessageReader__Open` , stream)
+}
+
+ipc___MessageReader__ReadNextMessage <- function(reader){
+    .Call(`_arrow_ipc___MessageReader__ReadNextMessage` , reader)
+}
+
+ipc___ReadMessage <- function(stream){
+    .Call(`_arrow_ipc___ReadMessage` , stream)
+}
+
+read_parquet_file <- function(filename){
+    .Call(`_arrow_read_parquet_file` , filename)
+}
+
+RecordBatch__num_columns <- function(x){
+    .Call(`_arrow_RecordBatch__num_columns` , x)
+}
+
+RecordBatch__num_rows <- function(x){
+    .Call(`_arrow_RecordBatch__num_rows` , x)
+}
+
+RecordBatch__schema <- function(x){
+    .Call(`_arrow_RecordBatch__schema` , x)
+}
+
+RecordBatch__columns <- function(batch){
+    .Call(`_arrow_RecordBatch__columns` , batch)
+}
+
+RecordBatch__column <- function(batch, i){
+    .Call(`_arrow_RecordBatch__column` , batch, i)
+}
+
+RecordBatch__from_dataframe <- function(tbl){
+    .Call(`_arrow_RecordBatch__from_dataframe` , tbl)
+}
+
+RecordBatch__Equals <- function(self, other){
+    .Call(`_arrow_RecordBatch__Equals` , self, other)
+}
+
+RecordBatch__RemoveColumn <- function(batch, i){
+    .Call(`_arrow_RecordBatch__RemoveColumn` , batch, i)
+}
+
+RecordBatch__column_name <- function(batch, i){
+    .Call(`_arrow_RecordBatch__column_name` , batch, i)
+}
+
+RecordBatch__names <- function(batch){
+    .Call(`_arrow_RecordBatch__names` , batch)
+}
+
+RecordBatch__Slice1 <- function(self, offset){
+    .Call(`_arrow_RecordBatch__Slice1` , self, offset)
+}
+
+RecordBatch__Slice2 <- function(self, offset, length){
+    .Call(`_arrow_RecordBatch__Slice2` , self, offset, length)
+}
+
+ipc___SerializeRecordBatch__Raw <- function(batch){
+    .Call(`_arrow_ipc___SerializeRecordBatch__Raw` , batch)
+}
+
+ipc___ReadRecordBatch__InputStream__Schema <- function(stream, schema){
+    .Call(`_arrow_ipc___ReadRecordBatch__InputStream__Schema` , stream, schema)
+}
+
+RecordBatch__from_arrays <- function(schema_sxp, lst){
+    .Call(`_arrow_RecordBatch__from_arrays` , schema_sxp, lst)
+}
+
+RecordBatchReader__schema <- function(reader){
+    .Call(`_arrow_RecordBatchReader__schema` , reader)
+}
+
+RecordBatchReader__ReadNext <- function(reader){
+    .Call(`_arrow_RecordBatchReader__ReadNext` , reader)
+}
+
+ipc___RecordBatchStreamReader__Open <- function(stream){
+    .Call(`_arrow_ipc___RecordBatchStreamReader__Open` , stream)
+}
+
+ipc___RecordBatchStreamReader__batches <- function(reader){
+    .Call(`_arrow_ipc___RecordBatchStreamReader__batches` , reader)
+}
+
+ipc___RecordBatchFileReader__schema <- function(reader){
+    .Call(`_arrow_ipc___RecordBatchFileReader__schema` , reader)
+}
+
+ipc___RecordBatchFileReader__num_record_batches <- function(reader){
+    .Call(`_arrow_ipc___RecordBatchFileReader__num_record_batches` , reader)
+}
+
+ipc___RecordBatchFileReader__ReadRecordBatch <- function(reader, i){
+    .Call(`_arrow_ipc___RecordBatchFileReader__ReadRecordBatch` , reader, i)
+}
+
+ipc___RecordBatchFileReader__Open <- function(file){
+    .Call(`_arrow_ipc___RecordBatchFileReader__Open` , file)
+}
+
+Table__from_RecordBatchFileReader <- function(reader){
+    .Call(`_arrow_Table__from_RecordBatchFileReader` , reader)
+}
+
+Table__from_RecordBatchStreamReader <- function(reader){
+    .Call(`_arrow_Table__from_RecordBatchStreamReader` , reader)
+}
+
+ipc___RecordBatchFileReader__batches <- function(reader){
+    .Call(`_arrow_ipc___RecordBatchFileReader__batches` , reader)
+}
+
+ipc___RecordBatchWriter__WriteRecordBatch <- function(batch_writer, batch){
+    invisible(.Call(`_arrow_ipc___RecordBatchWriter__WriteRecordBatch` , batch_writer, batch))
+}
+
+ipc___RecordBatchWriter__WriteTable <- function(batch_writer, table){
+    invisible(.Call(`_arrow_ipc___RecordBatchWriter__WriteTable` , batch_writer, table))
+}
+
+ipc___RecordBatchWriter__Close <- function(batch_writer){
+    invisible(.Call(`_arrow_ipc___RecordBatchWriter__Close` , batch_writer))
+}
+
+ipc___RecordBatchFileWriter__Open <- function(stream, schema){
+    .Call(`_arrow_ipc___RecordBatchFileWriter__Open` , stream, schema)
+}
+
+ipc___RecordBatchStreamWriter__Open <- function(stream, schema){
+    .Call(`_arrow_ipc___RecordBatchStreamWriter__Open` , stream, schema)
+}
+
+Table__from_dataframe <- function(tbl){
+    .Call(`_arrow_Table__from_dataframe` , tbl)
+}
+
+Table__num_columns <- function(x){
+    .Call(`_arrow_Table__num_columns` , x)
+}
+
+Table__num_rows <- function(x){
+    .Call(`_arrow_Table__num_rows` , x)
+}
+
+Table__schema <- function(x){
+    .Call(`_arrow_Table__schema` , x)
+}
+
+Table__column <- function(table, i){
+    .Call(`_arrow_Table__column` , table, i)
+}
+
+Table__columns <- function(table){
+    .Call(`_arrow_Table__columns` , table)
+}
+
+Table__from_dots <- function(lst, schema_sxp){
+    .Call(`_arrow_Table__from_dots` , lst, schema_sxp)
+}
+
+GetCpuThreadPoolCapacity <- function(){
+    .Call(`_arrow_GetCpuThreadPoolCapacity` )
+}
+
+SetCpuThreadPoolCapacity <- function(threads){
+    invisible(.Call(`_arrow_SetCpuThreadPoolCapacity` , threads))
+}
+
+
diff --git a/r/R/parquet.R b/r/R/parquet.R
index c126559..d7f389f 100644
--- a/r/R/parquet.R
+++ b/r/R/parquet.R
@@ -30,7 +30,11 @@
 #' @return A [arrow::Table][arrow__Table], or a `tbl_df` if `as_tibble` is
 #' `TRUE`.
 #' @examples
-#' df <- read_parquet(system.file("v0.7.1.parquet", package="arrow"))
+#'
+#' \dontrun{
+#'   df <- read_parquet(system.file("v0.7.1.parquet", package="arrow"))
+#' }
+#'
 #' @export
 read_parquet <- function(file, as_tibble = TRUE, use_threads = TRUE, ...) {
   tab <- shared_ptr(`arrow::Table`, read_parquet_file(file))
diff --git a/r/R/zzz.R b/r/R/zzz.R
index be952a6..eab9ad4 100644
--- a/r/R/zzz.R
+++ b/r/R/zzz.R
@@ -18,3 +18,11 @@
 #' @useDynLib arrow, .registration = TRUE
 #' @importFrom Rcpp sourceCpp
 NULL
+
+#' Is the C++ Arrow library available
+#'
+#' @export
+arrow_available <- function() {
+  .Call(`_arrow_available`)
+}
+
diff --git a/r/README.Rmd b/r/README.Rmd
index 5b837ef..4b6ab14 100644
--- a/r/README.Rmd
+++ b/r/README.Rmd
@@ -88,8 +88,7 @@ For any other build/configuration challenges, see the [C++ developer guide](http
 ```{r}
 library(arrow)
 
-tib <- tibble::tibble(x = 1:10, y = rnorm(10))
-tab <- table(tib)
+tab <- arrow::table(x = 1:10, y = rnorm(10))
 tab$schema
 tab
 as_tibble(tab)
@@ -104,6 +103,9 @@ install.packages("devtools")
 devtools::install_dev_deps()
 ```
 
+If you change C++ code, you need to set the `ARROW_R_DEV` environment variable to `TRUE`, e.g. 
+in your `~/.Renviron` so that the `data-raw/codegen.R` file is used for code generation. 
+
 ### Useful functions
 
 ```r
diff --git a/r/README.md b/r/README.md
index de13465..c87b8c6 100644
--- a/r/README.md
+++ b/r/README.md
@@ -97,8 +97,7 @@ library(arrow)
 #> 
 #>     array, table
 
-tib <- tibble::tibble(x = 1:10, y = rnorm(10))
-tab <- table(tib)
+tab <- arrow::table(x = 1:10, y = rnorm(10))
 tab$schema
 #> arrow::Schema 
 #> x: int32
@@ -107,18 +106,18 @@ tab
 #> arrow::Table
 as_tibble(tab)
 #> # A tibble: 10 x 2
-#>        x       y
-#>    <int>   <dbl>
-#>  1     1  0.912 
-#>  2     2  1.31  
-#>  3     3 -1.47  
-#>  4     4 -0.332 
-#>  5     5 -1.60  
-#>  6     6 -2.51  
-#>  7     7  0.903 
-#>  8     8 -2.44  
-#>  9     9  0.990 
-#> 10    10 -0.0164
+#>        x      y
+#>    <int>  <dbl>
+#>  1     1  0.524
+#>  2     2 -0.606
+#>  3     3 -0.655
+#>  4     4  1.37 
+#>  5     5  1.53 
+#>  6     6  1.96 
+#>  7     7  1.80 
+#>  8     8  1.27 
+#>  9     9  0.698
+#> 10    10 -0.661
 ```
 
 ## Developing
@@ -132,6 +131,10 @@ install.packages("devtools")
 devtools::install_dev_deps()
 ```
 
+If you change C++ code, you need to set the `ARROW_R_DEV` environment
+variable to `TRUE`, e.g.  in your `~/.Renviron` so that the
+`data-raw/codegen.R` file is used for code generation.
+
 ### Useful functions
 
 ``` r
diff --git a/r/configure b/r/configure
index 10045d3..4b3484f 100755
--- a/r/configure
+++ b/r/configure
@@ -33,11 +33,10 @@ PKG_BREW_NAME="apache-arrow"
 PKG_TEST_HEADER="<arrow/api.h>"
 PKG_LIBS="-larrow -lparquet"
 
-# Use pkg-config if available
-pkg-config --version >/dev/null 2>&1
-if [ $? -eq 0 ]; then
-  PKGCONFIG_CFLAGS=`pkg-config --cflags --silence-errors ${PKG_CONFIG_NAME}`
-  PKGCONFIG_LIBS=`pkg-config --libs ${PKG_CONFIG_NAME}`
+# generate code
+if [ "$ARROW_R_DEV" == "TRUE" ]; then
+  echo "*** Generating code with data-raw/codegen.R"
+  ${R_HOME}/bin/Rscript data-raw/codegen.R
 fi
 
 # Note that cflags may be empty in case of success
@@ -45,26 +44,48 @@ if [ "$INCLUDE_DIR" ] || [ "$LIB_DIR" ]; then
   echo "Found INCLUDE_DIR and/or LIB_DIR!"
   PKG_CFLAGS="-I$INCLUDE_DIR $PKG_CFLAGS"
   PKG_LIBS="-L$LIB_DIR $PKG_LIBS"
-elif [ "$PKGCONFIG_CFLAGS" ] || [ "$PKGCONFIG_LIBS" ]; then
-  echo "Found pkg-config cflags and libs!"
-  PKG_CFLAGS=${PKGCONFIG_CFLAGS}
-  PKG_LIBS=${PKGCONFIG_LIBS}
-elif [[ "$OSTYPE" == "darwin"* ]]; then
-  if [ "$(command -v brew)" ]; then
-    BREWDIR=$(brew --prefix)
-  else
-    curl -sfL "https://jeroen.github.io/autobrew/$PKG_BREW_NAME" > autobrew
-    source autobrew
+else
+  # Use pkg-config if available
+  pkg-config --version >/dev/null 2>&1
+  if [ $? -eq 0 ]; then
+    echo "$ pkg-config --cflags --silence-errors ${PKG_CONFIG_NAME}"
+    PKGCONFIG_CFLAGS=`pkg-config --cflags --silence-errors ${PKG_CONFIG_NAME}`
+    echo "PKGCONFIG_CFLAGS = \"${PKGCONFIG_CFLAGS}\""
+
+    echo "$ pkg-config --libs ${PKG_CONFIG_NAME}"
+    PKGCONFIG_LIBS=`pkg-config --libs --silence-errors ${PKG_CONFIG_NAME}`
+    echo "PKGCONFIG_LIBS = \"${PKGCONFIG_LIBS}\""
+    echo ""
   fi
-  PKG_CFLAGS="-I$BREWDIR/opt/$PKG_BREW_NAME/include"
-  PKG_LIBS="-L$BREWDIR/opt/$PKG_BREW_NAME/lib $PKG_LIBS"
-fi
 
-PKG_CFLAGS="$PKG_CFLAGS -DARROW_R_WITH_PARQUET"
+  if [ "$PKGCONFIG_CFLAGS" ] || [ "$PKGCONFIG_LIBS" ]; then
+    echo "Found pkg-config cflags and libs!"
+    PKG_CFLAGS="$PKGCONFIG_CFLAGS"
+    PKG_LIBS=${PKGCONFIG_LIBS}
+  else
+    if [[ "$OSTYPE" == "darwin"* ]]; then
+      if [ "$(command -v brew)" ]; then
+        echo "brew is available"
+        BREWDIR=$(brew --prefix)
+
+        if brew ls --versions apache-arrow > /dev/null; then
+          # right now, we need HEAD version
+          brew install apache-arrow --HEAD
+        fi
+
+        PKG_CFLAGS="-I$BREWDIR/opt/$PKG_BREW_NAME/include"
+        PKG_LIBS="-L$BREWDIR/opt/$PKG_BREW_NAME/lib $PKG_LIBS"
+      else
+        echo "brew is not available, trying autobrew"
+        curl -sfL "https://jeroen.github.io/autobrew/$PKG_BREW_NAME" > autobrew
+        source autobrew
+        PKG_CFLAGS="-I$BREWDIR/opt/$PKG_BREW_NAME/include"
+        PKG_LIBS="-L$BREWDIR/opt/$PKG_BREW_NAME/lib $PKG_LIBS"
+      fi
 
-# For debugging
-echo "PKG_CFLAGS=$PKG_CFLAGS"
-echo "PKG_LIBS=$PKG_LIBS"
+    fi
+  fi
+fi
 
 # Find compiler
 CXXCPP=$("${R_HOME}"/bin/R CMD config CXXCPP)
@@ -82,15 +103,16 @@ echo "#include $PKG_TEST_HEADER" | ${CXXCPP} ${CPPFLAGS} ${PKG_CFLAGS} ${CXX11FL
 
 # Customize the error
 if [ $? -ne 0 ]; then
-  echo "------------------------- ANTICONF ERROR ---------------------------"
-  echo "Configuration failed because $PKG_CONFIG_NAME was not found. Try installing:"
-  echo " * brew: $PKG_BREW_NAME (Mac OSX)"
-  echo "If $PKG_CONFIG_NAME is already installed, check that 'pkg-config' is in your"
-  echo "PATH and PKG_CONFIG_PATH contains a $PKG_CONFIG_NAME.pc file. If pkg-config"
-  echo "is unavailable you can set INCLUDE_DIR and LIB_DIR manually via:"
-  echo "R CMD INSTALL --configure-vars='INCLUDE_DIR=... LIB_DIR=...'"
-  echo "--------------------------------------------------------------------"
-  exit 1;
+  echo "------------------------- NOTE ---------------------------"
+  echo "After installation, please run arrow::install_arrow() to install"
+  echo "required runtime libraries"
+  echo "---------------------------------------------------------"
+  PKG_LIBS=""
+  PKG_CFLAGS=""
+else
+  PKG_CFLAGS="$PKG_CFLAGS -DARROW_R_WITH_ARROW -DARROW_R_WITH_PARQUET"
+  echo "PKG_CFLAGS=$PKG_CFLAGS"
+  echo "PKG_LIBS=$PKG_LIBS"
 fi
 
 # Write to Makevars
diff --git a/r/data-raw/codegen.R b/r/data-raw/codegen.R
new file mode 100644
index 0000000..17fa0f0
--- /dev/null
+++ b/r/data-raw/codegen.R
@@ -0,0 +1,172 @@
+# 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.
+
+# This file is used to generate code in the files
+# src/arrowExports.cpp and R/arrowExports.R
+#
+# This is similar to what Rcpp::compileAttributes() would do,
+# with some arrow specific changes.
+#
+# Functions are decorated with [[arrow::export]] instead of [[Rcpp::export]]
+# and the generated code adds a layer of protection so that
+# the arrow package can be installed even when libarrow is not
+#
+# All the C++ code should be guarded by
+#
+# #if defined(ARROW_R_WITH_ARROW)
+# // [[arrow::export]]
+# std::shared_ptr<arrow::Array> some_function_using_arrow_api(){
+#     ...
+# }
+# #endif
+
+suppressPackageStartupMessages({
+  # pak::pkg_install("romainfrancois/decor")
+  library(decor)
+
+  library(dplyr)
+  library(purrr)
+  library(glue)
+})
+
+decorations <- cpp_decorations() %>%
+  filter(decoration == "arrow::export") %>%
+  # the three lines below can be expressed with rap()
+  # more concisely
+  # rap(            ~ decor:::parse_cpp_function(context))
+  mutate(functions = map(context, decor:::parse_cpp_function)) %>%
+  { bind_cols(., bind_rows(pull(., functions))) } %>%
+  select(-functions)
+
+message(glue("*** > {n} functions decorated with [[arrow::export]]", n = nrow(decorations)))
+
+glue_collapse_data <- function(data, ..., sep = ", ", last = "") {
+  res <- glue_collapse(glue_data(data, ...), sep = sep, last = last)
+  if(length(res) == 0) res <- ""
+  res
+}
+
+wrap_call <- function(name, return_type, args) {
+  call <- glue::glue('{name}({list_params})', list_params = glue_collapse_data(args, "{name}"))
+  if(return_type == "void") {
+    glue::glue("\t{call};\n\treturn R_NilValue;", .trim = FALSE)
+  } else {
+    glue::glue("\treturn Rcpp::wrap({call});")
+  }
+}
+cpp_functions_definitions <- decorations %>%
+  select(name, return_type, args, file, line) %>%
+  pmap_chr(function(name, return_type, args, file, line){
+    glue::glue('
+    // {basename(file)}:{line}
+    #if defined(ARROW_R_WITH_ARROW)
+    {return_type} {name}({real_params});
+    RcppExport SEXP _arrow_{name}({sexp_params}){{
+    BEGIN_RCPP
+    {input_params}{return_line}{wrap_call(name, return_type, args)}
+    END_RCPP
+    }}
+    #else
+    RcppExport SEXP _arrow_{name}({sexp_params}){{
+    \tRf_error("Cannot call {name}(). Please use arrow::install_arrow() to install required runtime libraries. ");
+    }}
+    #endif
+
+    ',
+      sep = "\n",
+      real_params = glue_collapse_data(args, "{type} {name}"),
+      sexp_params = glue_collapse_data(args, "SEXP {name}_sexp"),
+      input_params = glue_collapse_data(args, "\tRcpp::traits::input_parameter<{type}>::type {name}({name}_sexp);", sep = "\n"),
+      return_line = if(nrow(args)) "\n" else ""
+    )
+  }) %>%
+  glue_collapse(sep = "\n")
+
+cpp_functions_registration <- decorations %>%
+  select(name, return_type, args) %>%
+  pmap_chr(function(name, return_type, args){
+    glue('\t\t{{ "_arrow_{name}", (DL_FUNC) &_arrow_{name}, {nrow(args)}}}, ')
+  }) %>%
+  glue_collapse(sep  = "\n")
+
+writeLines(con = "src/arrowExports.cpp", glue::glue('
+// Generated by using data-raw/codegen.R -> do not edit by hand
+#include "./arrow_types.h"
+#include <Rcpp.h>
+
+using namespace Rcpp;
+
+{cpp_functions_definitions}
+
+extern "C" SEXP _arrow_available() {{
+return Rf_ScalarLogical(
+#if defined(ARROW_R_WITH_ARROW)
+  TRUE
+#else
+  FALSE
+#endif
+);
+}}
+
+static const R_CallMethodDef CallEntries[] = {{
+\t\t{{ "_arrow_available", (DL_FUNC)& _arrow_available, 0 }},
+{cpp_functions_registration}
+\t\t{{NULL, NULL, 0}}
+}};
+
+RcppExport void R_init_arrow(DllInfo* dll){{
+  R_registerRoutines(dll, NULL, CallEntries, NULL, NULL);
+  R_useDynamicSymbols(dll, FALSE);
+}}
+
+') )
+
+message("*** > generated file `src/arrowExports.cpp`")
+
+r_functions <- decorations %>%
+  select(name, return_type, args) %>%
+  pmap_chr(function(name, return_type, args) {
+    params <- if (nrow(args)) {
+      paste0(", ", glue_collapse_data(args, "{name}"))
+    } else {
+      ""
+    }
+    call <- if(return_type == "void") {
+      glue::glue('invisible(.Call(`_arrow_{name}` {params}))')
+    } else {
+      glue::glue('.Call(`_arrow_{name}` {params})')
+    }
+
+    glue::glue('
+    {name} <- function({list_params}){{
+        {call}
+    }}
+
+    ',
+      list_params = glue_collapse_data(args, "{name}"),
+      sep = "\n",
+    )
+  }) %>%
+  glue_collapse(sep = "\n")
+
+writeLines(con = "R/arrowExports.R", glue::glue('
+# Generated by using data-raw/codegen.R -> do not edit by hand
+
+{r_functions}
+
+'))
+message("*** > generated file `R/arrowExports.R`")
diff --git a/r/man/arrow_available.Rd b/r/man/arrow_available.Rd
new file mode 100644
index 0000000..af0f938
--- /dev/null
+++ b/r/man/arrow_available.Rd
@@ -0,0 +1,11 @@
+% Generated by roxygen2: do not edit by hand
+% Please edit documentation in R/zzz.R
+\name{arrow_available}
+\alias{arrow_available}
+\title{Is the C++ Arrow library available}
+\usage{
+arrow_available()
+}
+\description{
+Is the C++ Arrow library available
+}
diff --git a/r/man/field.Rd b/r/man/field.Rd
index 5cbd803..52995a1 100644
--- a/r/man/field.Rd
+++ b/r/man/field.Rd
@@ -17,6 +17,9 @@ field(name, type, metadata)
 Factory for a \code{arrow::Field}
 }
 \examples{
-field("x", int32())
+
+\dontrun{
+   field("x", int32())
+}
 
 }
diff --git a/r/man/read_parquet.Rd b/r/man/read_parquet.Rd
index 36e6428..2bce02c 100644
--- a/r/man/read_parquet.Rd
+++ b/r/man/read_parquet.Rd
@@ -26,5 +26,9 @@ A \link[=arrow__Table]{arrow::Table}, or a \code{tbl_df} if \code{as_tibble} is
 This function enables you to read Parquet files into R.
 }
 \examples{
-df <- read_parquet(system.file("v0.7.1.parquet", package="arrow"))
+
+\dontrun{
+  df <- read_parquet(system.file("v0.7.1.parquet", package="arrow"))
+}
+
 }
diff --git a/r/man/table.Rd b/r/man/table.Rd
index 4d93ff3..fbf9632 100644
--- a/r/man/table.Rd
+++ b/r/man/table.Rd
@@ -9,7 +9,10 @@ table(..., schema = NULL)
 \arguments{
 \item{...}{arrays, chunked arrays, or R vectors}
 
-\item{schema}{NULL or a schema}
+\item{schema}{a schema. The default (\code{NULL}) infers the schema from the \code{...}}
+}
+\value{
+an arrow::Table
 }
 \description{
 Create an arrow::Table from a data frame
diff --git a/r/man/threadpool.Rd b/r/man/threadpool.Rd
deleted file mode 100644
index 9cf162f..0000000
--- a/r/man/threadpool.Rd
+++ /dev/null
@@ -1,27 +0,0 @@
-% Generated by roxygen2: do not edit by hand
-% Please edit documentation in R/RcppExports.R
-\name{threadpool}
-\alias{threadpool}
-\alias{GetCpuThreadPoolCapacity}
-\alias{SetCpuThreadPoolCapacity}
-\title{View and manage the capacity of the global thread pool}
-\usage{
-GetCpuThreadPoolCapacity()
-
-SetCpuThreadPoolCapacity(threads)
-}
-\arguments{
-\item{threads}{the number of worker threads in the thread pool to which
-Arrow dispatches various CPU-bound tasks.}
-}
-\value{
-\code{GetCpuThreadPoolCapacity()} returns the number of worker threads.
-\code{SetCpuThreadPoolCapacity()} returns nothing.
-}
-\description{
-\code{GetCpuThreadPoolCapacity()} returns the number of worker threads in the
-thread pool to which
-Arrow dispatches various CPU-bound tasks. This is an ideal number,
-not necessarily the exact number of threads at a given point in time.
-You can change this number using \code{SetCpuThreadPoolCapacity()}.
-}
diff --git a/r/src/RcppExports.cpp b/r/src/RcppExports.cpp
deleted file mode 100644
index a92c4c8..0000000
--- a/r/src/RcppExports.cpp
+++ /dev/null
@@ -1,2530 +0,0 @@
-// Generated by using Rcpp::compileAttributes() -> do not edit by hand
-// Generator token: 10BE3573-1514-4C36-9D1C-5A225CD40393
-
-#include "arrow_types.h"
-#include <Rcpp.h>
-
-using namespace Rcpp;
-
-// Array__Slice1
-std::shared_ptr<arrow::Array> Array__Slice1(const std::shared_ptr<arrow::Array>& array, int offset);
-RcppExport SEXP _arrow_Array__Slice1(SEXP arraySEXP, SEXP offsetSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::Array>& >::type array(arraySEXP);
-    Rcpp::traits::input_parameter< int >::type offset(offsetSEXP);
-    rcpp_result_gen = Rcpp::wrap(Array__Slice1(array, offset));
-    return rcpp_result_gen;
-END_RCPP
-}
-// Array__Slice2
-std::shared_ptr<arrow::Array> Array__Slice2(const std::shared_ptr<arrow::Array>& array, int offset, int length);
-RcppExport SEXP _arrow_Array__Slice2(SEXP arraySEXP, SEXP offsetSEXP, SEXP lengthSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::Array>& >::type array(arraySEXP);
-    Rcpp::traits::input_parameter< int >::type offset(offsetSEXP);
-    Rcpp::traits::input_parameter< int >::type length(lengthSEXP);
-    rcpp_result_gen = Rcpp::wrap(Array__Slice2(array, offset, length));
-    return rcpp_result_gen;
-END_RCPP
-}
-// Array__IsNull
-bool Array__IsNull(const std::shared_ptr<arrow::Array>& x, int i);
-RcppExport SEXP _arrow_Array__IsNull(SEXP xSEXP, SEXP iSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::Array>& >::type x(xSEXP);
-    Rcpp::traits::input_parameter< int >::type i(iSEXP);
-    rcpp_result_gen = Rcpp::wrap(Array__IsNull(x, i));
-    return rcpp_result_gen;
-END_RCPP
-}
-// Array__IsValid
-bool Array__IsValid(const std::shared_ptr<arrow::Array>& x, int i);
-RcppExport SEXP _arrow_Array__IsValid(SEXP xSEXP, SEXP iSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::Array>& >::type x(xSEXP);
-    Rcpp::traits::input_parameter< int >::type i(iSEXP);
-    rcpp_result_gen = Rcpp::wrap(Array__IsValid(x, i));
-    return rcpp_result_gen;
-END_RCPP
-}
-// Array__length
-int Array__length(const std::shared_ptr<arrow::Array>& x);
-RcppExport SEXP _arrow_Array__length(SEXP xSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::Array>& >::type x(xSEXP);
-    rcpp_result_gen = Rcpp::wrap(Array__length(x));
-    return rcpp_result_gen;
-END_RCPP
-}
-// Array__offset
-int Array__offset(const std::shared_ptr<arrow::Array>& x);
-RcppExport SEXP _arrow_Array__offset(SEXP xSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::Array>& >::type x(xSEXP);
-    rcpp_result_gen = Rcpp::wrap(Array__offset(x));
-    return rcpp_result_gen;
-END_RCPP
-}
-// Array__null_count
-int Array__null_count(const std::shared_ptr<arrow::Array>& x);
-RcppExport SEXP _arrow_Array__null_count(SEXP xSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::Array>& >::type x(xSEXP);
-    rcpp_result_gen = Rcpp::wrap(Array__null_count(x));
-    return rcpp_result_gen;
-END_RCPP
-}
-// Array__type
-std::shared_ptr<arrow::DataType> Array__type(const std::shared_ptr<arrow::Array>& x);
-RcppExport SEXP _arrow_Array__type(SEXP xSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::Array>& >::type x(xSEXP);
-    rcpp_result_gen = Rcpp::wrap(Array__type(x));
-    return rcpp_result_gen;
-END_RCPP
-}
-// Array__ToString
-std::string Array__ToString(const std::shared_ptr<arrow::Array>& x);
-RcppExport SEXP _arrow_Array__ToString(SEXP xSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::Array>& >::type x(xSEXP);
-    rcpp_result_gen = Rcpp::wrap(Array__ToString(x));
-    return rcpp_result_gen;
-END_RCPP
-}
-// Array__type_id
-arrow::Type::type Array__type_id(const std::shared_ptr<arrow::Array>& x);
-RcppExport SEXP _arrow_Array__type_id(SEXP xSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::Array>& >::type x(xSEXP);
-    rcpp_result_gen = Rcpp::wrap(Array__type_id(x));
-    return rcpp_result_gen;
-END_RCPP
-}
-// Array__Equals
-bool Array__Equals(const std::shared_ptr<arrow::Array>& lhs, const std::shared_ptr<arrow::Array>& rhs);
-RcppExport SEXP _arrow_Array__Equals(SEXP lhsSEXP, SEXP rhsSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::Array>& >::type lhs(lhsSEXP);
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::Array>& >::type rhs(rhsSEXP);
-    rcpp_result_gen = Rcpp::wrap(Array__Equals(lhs, rhs));
-    return rcpp_result_gen;
-END_RCPP
-}
-// Array__ApproxEquals
-bool Array__ApproxEquals(const std::shared_ptr<arrow::Array>& lhs, const std::shared_ptr<arrow::Array>& rhs);
-RcppExport SEXP _arrow_Array__ApproxEquals(SEXP lhsSEXP, SEXP rhsSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::Array>& >::type lhs(lhsSEXP);
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::Array>& >::type rhs(rhsSEXP);
-    rcpp_result_gen = Rcpp::wrap(Array__ApproxEquals(lhs, rhs));
-    return rcpp_result_gen;
-END_RCPP
-}
-// Array__data
-std::shared_ptr<arrow::ArrayData> Array__data(const std::shared_ptr<arrow::Array>& array);
-RcppExport SEXP _arrow_Array__data(SEXP arraySEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::Array>& >::type array(arraySEXP);
-    rcpp_result_gen = Rcpp::wrap(Array__data(array));
-    return rcpp_result_gen;
-END_RCPP
-}
-// Array__RangeEquals
-bool Array__RangeEquals(const std::shared_ptr<arrow::Array>& self, const std::shared_ptr<arrow::Array>& other, int start_idx, int end_idx, int other_start_idx);
-RcppExport SEXP _arrow_Array__RangeEquals(SEXP selfSEXP, SEXP otherSEXP, SEXP start_idxSEXP, SEXP end_idxSEXP, SEXP other_start_idxSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::Array>& >::type self(selfSEXP);
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::Array>& >::type other(otherSEXP);
-    Rcpp::traits::input_parameter< int >::type start_idx(start_idxSEXP);
-    Rcpp::traits::input_parameter< int >::type end_idx(end_idxSEXP);
-    Rcpp::traits::input_parameter< int >::type other_start_idx(other_start_idxSEXP);
-    rcpp_result_gen = Rcpp::wrap(Array__RangeEquals(self, other, start_idx, end_idx, other_start_idx));
-    return rcpp_result_gen;
-END_RCPP
-}
-// Array__Mask
-LogicalVector Array__Mask(const std::shared_ptr<arrow::Array>& array);
-RcppExport SEXP _arrow_Array__Mask(SEXP arraySEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::Array>& >::type array(arraySEXP);
-    rcpp_result_gen = Rcpp::wrap(Array__Mask(array));
-    return rcpp_result_gen;
-END_RCPP
-}
-// DictionaryArray__indices
-std::shared_ptr<arrow::Array> DictionaryArray__indices(const std::shared_ptr<arrow::DictionaryArray>& array);
-RcppExport SEXP _arrow_DictionaryArray__indices(SEXP arraySEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::DictionaryArray>& >::type array(arraySEXP);
-    rcpp_result_gen = Rcpp::wrap(DictionaryArray__indices(array));
-    return rcpp_result_gen;
-END_RCPP
-}
-// DictionaryArray__dictionary
-std::shared_ptr<arrow::Array> DictionaryArray__dictionary(const std::shared_ptr<arrow::DictionaryArray>& array);
-RcppExport SEXP _arrow_DictionaryArray__dictionary(SEXP arraySEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::DictionaryArray>& >::type array(arraySEXP);
-    rcpp_result_gen = Rcpp::wrap(DictionaryArray__dictionary(array));
-    return rcpp_result_gen;
-END_RCPP
-}
-// Array__as_vector
-SEXP Array__as_vector(const std::shared_ptr<arrow::Array>& array);
-RcppExport SEXP _arrow_Array__as_vector(SEXP arraySEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::Array>& >::type array(arraySEXP);
-    rcpp_result_gen = Rcpp::wrap(Array__as_vector(array));
-    return rcpp_result_gen;
-END_RCPP
-}
-// ChunkedArray__as_vector
-SEXP ChunkedArray__as_vector(const std::shared_ptr<arrow::ChunkedArray>& chunked_array);
-RcppExport SEXP _arrow_ChunkedArray__as_vector(SEXP chunked_arraySEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::ChunkedArray>& >::type chunked_array(chunked_arraySEXP);
-    rcpp_result_gen = Rcpp::wrap(ChunkedArray__as_vector(chunked_array));
-    return rcpp_result_gen;
-END_RCPP
-}
-// RecordBatch__to_dataframe
-Rcpp::List RecordBatch__to_dataframe(const std::shared_ptr<arrow::RecordBatch>& batch, bool use_threads);
-RcppExport SEXP _arrow_RecordBatch__to_dataframe(SEXP batchSEXP, SEXP use_threadsSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::RecordBatch>& >::type batch(batchSEXP);
-    Rcpp::traits::input_parameter< bool >::type use_threads(use_threadsSEXP);
-    rcpp_result_gen = Rcpp::wrap(RecordBatch__to_dataframe(batch, use_threads));
-    return rcpp_result_gen;
-END_RCPP
-}
-// Table__to_dataframe
-Rcpp::List Table__to_dataframe(const std::shared_ptr<arrow::Table>& table, bool use_threads);
-RcppExport SEXP _arrow_Table__to_dataframe(SEXP tableSEXP, SEXP use_threadsSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::Table>& >::type table(tableSEXP);
-    Rcpp::traits::input_parameter< bool >::type use_threads(use_threadsSEXP);
-    rcpp_result_gen = Rcpp::wrap(Table__to_dataframe(table, use_threads));
-    return rcpp_result_gen;
-END_RCPP
-}
-// Array__infer_type
-std::shared_ptr<arrow::DataType> Array__infer_type(SEXP x);
-RcppExport SEXP _arrow_Array__infer_type(SEXP xSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< SEXP >::type x(xSEXP);
-    rcpp_result_gen = Rcpp::wrap(Array__infer_type(x));
-    return rcpp_result_gen;
-END_RCPP
-}
-// Array__from_vector
-std::shared_ptr<arrow::Array> Array__from_vector(SEXP x, SEXP s_type);
-RcppExport SEXP _arrow_Array__from_vector(SEXP xSEXP, SEXP s_typeSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< SEXP >::type x(xSEXP);
-    Rcpp::traits::input_parameter< SEXP >::type s_type(s_typeSEXP);
-    rcpp_result_gen = Rcpp::wrap(Array__from_vector(x, s_type));
-    return rcpp_result_gen;
-END_RCPP
-}
-// ChunkedArray__from_list
-std::shared_ptr<arrow::ChunkedArray> ChunkedArray__from_list(Rcpp::List chunks, SEXP s_type);
-RcppExport SEXP _arrow_ChunkedArray__from_list(SEXP chunksSEXP, SEXP s_typeSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< Rcpp::List >::type chunks(chunksSEXP);
-    Rcpp::traits::input_parameter< SEXP >::type s_type(s_typeSEXP);
-    rcpp_result_gen = Rcpp::wrap(ChunkedArray__from_list(chunks, s_type));
-    return rcpp_result_gen;
-END_RCPP
-}
-// ArrayData__get_type
-std::shared_ptr<arrow::DataType> ArrayData__get_type(const std::shared_ptr<arrow::ArrayData>& x);
-RcppExport SEXP _arrow_ArrayData__get_type(SEXP xSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::ArrayData>& >::type x(xSEXP);
-    rcpp_result_gen = Rcpp::wrap(ArrayData__get_type(x));
-    return rcpp_result_gen;
-END_RCPP
-}
-// ArrayData__get_length
-int ArrayData__get_length(const std::shared_ptr<arrow::ArrayData>& x);
-RcppExport SEXP _arrow_ArrayData__get_length(SEXP xSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::ArrayData>& >::type x(xSEXP);
-    rcpp_result_gen = Rcpp::wrap(ArrayData__get_length(x));
-    return rcpp_result_gen;
-END_RCPP
-}
-// ArrayData__get_null_count
-int ArrayData__get_null_count(const std::shared_ptr<arrow::ArrayData>& x);
-RcppExport SEXP _arrow_ArrayData__get_null_count(SEXP xSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::ArrayData>& >::type x(xSEXP);
-    rcpp_result_gen = Rcpp::wrap(ArrayData__get_null_count(x));
-    return rcpp_result_gen;
-END_RCPP
-}
-// ArrayData__get_offset
-int ArrayData__get_offset(const std::shared_ptr<arrow::ArrayData>& x);
-RcppExport SEXP _arrow_ArrayData__get_offset(SEXP xSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::ArrayData>& >::type x(xSEXP);
-    rcpp_result_gen = Rcpp::wrap(ArrayData__get_offset(x));
-    return rcpp_result_gen;
-END_RCPP
-}
-// ArrayData__buffers
-List ArrayData__buffers(const std::shared_ptr<arrow::ArrayData>& x);
-RcppExport SEXP _arrow_ArrayData__buffers(SEXP xSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::ArrayData>& >::type x(xSEXP);
-    rcpp_result_gen = Rcpp::wrap(ArrayData__buffers(x));
-    return rcpp_result_gen;
-END_RCPP
-}
-// Buffer__is_mutable
-bool Buffer__is_mutable(const std::shared_ptr<arrow::Buffer>& buffer);
-RcppExport SEXP _arrow_Buffer__is_mutable(SEXP bufferSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::Buffer>& >::type buffer(bufferSEXP);
-    rcpp_result_gen = Rcpp::wrap(Buffer__is_mutable(buffer));
-    return rcpp_result_gen;
-END_RCPP
-}
-// Buffer__ZeroPadding
-void Buffer__ZeroPadding(const std::shared_ptr<arrow::Buffer>& buffer);
-RcppExport SEXP _arrow_Buffer__ZeroPadding(SEXP bufferSEXP) {
-BEGIN_RCPP
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::Buffer>& >::type buffer(bufferSEXP);
-    Buffer__ZeroPadding(buffer);
-    return R_NilValue;
-END_RCPP
-}
-// Buffer__capacity
-int64_t Buffer__capacity(const std::shared_ptr<arrow::Buffer>& buffer);
-RcppExport SEXP _arrow_Buffer__capacity(SEXP bufferSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::Buffer>& >::type buffer(bufferSEXP);
-    rcpp_result_gen = Rcpp::wrap(Buffer__capacity(buffer));
-    return rcpp_result_gen;
-END_RCPP
-}
-// Buffer__size
-int64_t Buffer__size(const std::shared_ptr<arrow::Buffer>& buffer);
-RcppExport SEXP _arrow_Buffer__size(SEXP bufferSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::Buffer>& >::type buffer(bufferSEXP);
-    rcpp_result_gen = Rcpp::wrap(Buffer__size(buffer));
-    return rcpp_result_gen;
-END_RCPP
-}
-// r___RBuffer__initialize
-std::shared_ptr<arrow::Buffer> r___RBuffer__initialize(SEXP x);
-RcppExport SEXP _arrow_r___RBuffer__initialize(SEXP xSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< SEXP >::type x(xSEXP);
-    rcpp_result_gen = Rcpp::wrap(r___RBuffer__initialize(x));
-    return rcpp_result_gen;
-END_RCPP
-}
-// ChunkedArray__length
-int ChunkedArray__length(const std::shared_ptr<arrow::ChunkedArray>& chunked_array);
-RcppExport SEXP _arrow_ChunkedArray__length(SEXP chunked_arraySEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::ChunkedArray>& >::type chunked_array(chunked_arraySEXP);
-    rcpp_result_gen = Rcpp::wrap(ChunkedArray__length(chunked_array));
-    return rcpp_result_gen;
-END_RCPP
-}
-// ChunkedArray__null_count
-int ChunkedArray__null_count(const std::shared_ptr<arrow::ChunkedArray>& chunked_array);
-RcppExport SEXP _arrow_ChunkedArray__null_count(SEXP chunked_arraySEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::ChunkedArray>& >::type chunked_array(chunked_arraySEXP);
-    rcpp_result_gen = Rcpp::wrap(ChunkedArray__null_count(chunked_array));
-    return rcpp_result_gen;
-END_RCPP
-}
-// ChunkedArray__num_chunks
-int ChunkedArray__num_chunks(const std::shared_ptr<arrow::ChunkedArray>& chunked_array);
-RcppExport SEXP _arrow_ChunkedArray__num_chunks(SEXP chunked_arraySEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::ChunkedArray>& >::type chunked_array(chunked_arraySEXP);
-    rcpp_result_gen = Rcpp::wrap(ChunkedArray__num_chunks(chunked_array));
-    return rcpp_result_gen;
-END_RCPP
-}
-// ChunkedArray__chunk
-std::shared_ptr<arrow::Array> ChunkedArray__chunk(const std::shared_ptr<arrow::ChunkedArray>& chunked_array, int i);
-RcppExport SEXP _arrow_ChunkedArray__chunk(SEXP chunked_arraySEXP, SEXP iSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::ChunkedArray>& >::type chunked_array(chunked_arraySEXP);
-    Rcpp::traits::input_parameter< int >::type i(iSEXP);
-    rcpp_result_gen = Rcpp::wrap(ChunkedArray__chunk(chunked_array, i));
-    return rcpp_result_gen;
-END_RCPP
-}
-// ChunkedArray__chunks
-List ChunkedArray__chunks(const std::shared_ptr<arrow::ChunkedArray>& chunked_array);
-RcppExport SEXP _arrow_ChunkedArray__chunks(SEXP chunked_arraySEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::ChunkedArray>& >::type chunked_array(chunked_arraySEXP);
-    rcpp_result_gen = Rcpp::wrap(ChunkedArray__chunks(chunked_array));
-    return rcpp_result_gen;
-END_RCPP
-}
-// ChunkedArray__type
-std::shared_ptr<arrow::DataType> ChunkedArray__type(const std::shared_ptr<arrow::ChunkedArray>& chunked_array);
-RcppExport SEXP _arrow_ChunkedArray__type(SEXP chunked_arraySEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::ChunkedArray>& >::type chunked_array(chunked_arraySEXP);
-    rcpp_result_gen = Rcpp::wrap(ChunkedArray__type(chunked_array));
-    return rcpp_result_gen;
-END_RCPP
-}
-// ChunkArray__Slice1
-std::shared_ptr<arrow::ChunkedArray> ChunkArray__Slice1(const std::shared_ptr<arrow::ChunkedArray>& chunked_array, int offset);
-RcppExport SEXP _arrow_ChunkArray__Slice1(SEXP chunked_arraySEXP, SEXP offsetSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::ChunkedArray>& >::type chunked_array(chunked_arraySEXP);
-    Rcpp::traits::input_parameter< int >::type offset(offsetSEXP);
-    rcpp_result_gen = Rcpp::wrap(ChunkArray__Slice1(chunked_array, offset));
-    return rcpp_result_gen;
-END_RCPP
-}
-// ChunkArray__Slice2
-std::shared_ptr<arrow::ChunkedArray> ChunkArray__Slice2(const std::shared_ptr<arrow::ChunkedArray>& chunked_array, int offset, int length);
-RcppExport SEXP _arrow_ChunkArray__Slice2(SEXP chunked_arraySEXP, SEXP offsetSEXP, SEXP lengthSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::ChunkedArray>& >::type chunked_array(chunked_arraySEXP);
-    Rcpp::traits::input_parameter< int >::type offset(offsetSEXP);
-    Rcpp::traits::input_parameter< int >::type length(lengthSEXP);
-    rcpp_result_gen = Rcpp::wrap(ChunkArray__Slice2(chunked_array, offset, length));
-    return rcpp_result_gen;
-END_RCPP
-}
-// Column__length
-int Column__length(const std::shared_ptr<arrow::Column>& column);
-RcppExport SEXP _arrow_Column__length(SEXP columnSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::Column>& >::type column(columnSEXP);
-    rcpp_result_gen = Rcpp::wrap(Column__length(column));
-    return rcpp_result_gen;
-END_RCPP
-}
-// Column__null_count
-int Column__null_count(const std::shared_ptr<arrow::Column>& column);
-RcppExport SEXP _arrow_Column__null_count(SEXP columnSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::Column>& >::type column(columnSEXP);
-    rcpp_result_gen = Rcpp::wrap(Column__null_count(column));
-    return rcpp_result_gen;
-END_RCPP
-}
-// Column__type
-std::shared_ptr<arrow::DataType> Column__type(const std::shared_ptr<arrow::Column>& column);
-RcppExport SEXP _arrow_Column__type(SEXP columnSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::Column>& >::type column(columnSEXP);
-    rcpp_result_gen = Rcpp::wrap(Column__type(column));
-    return rcpp_result_gen;
-END_RCPP
-}
-// Column__data
-std::shared_ptr<arrow::ChunkedArray> Column__data(const std::shared_ptr<arrow::Column>& column);
-RcppExport SEXP _arrow_Column__data(SEXP columnSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::Column>& >::type column(columnSEXP);
-    rcpp_result_gen = Rcpp::wrap(Column__data(column));
-    return rcpp_result_gen;
-END_RCPP
-}
-// util___Codec__Create
-std::unique_ptr<arrow::util::Codec> util___Codec__Create(arrow::Compression::type codec);
-RcppExport SEXP _arrow_util___Codec__Create(SEXP codecSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< arrow::Compression::type >::type codec(codecSEXP);
-    rcpp_result_gen = Rcpp::wrap(util___Codec__Create(codec));
-    return rcpp_result_gen;
-END_RCPP
-}
-// io___CompressedOutputStream__Make
-std::shared_ptr<arrow::io::CompressedOutputStream> io___CompressedOutputStream__Make(const std::unique_ptr<arrow::util::Codec>& codec, const std::shared_ptr<arrow::io::OutputStream>& raw);
-RcppExport SEXP _arrow_io___CompressedOutputStream__Make(SEXP codecSEXP, SEXP rawSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::unique_ptr<arrow::util::Codec>& >::type codec(codecSEXP);
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::io::OutputStream>& >::type raw(rawSEXP);
-    rcpp_result_gen = Rcpp::wrap(io___CompressedOutputStream__Make(codec, raw));
-    return rcpp_result_gen;
-END_RCPP
-}
-// io___CompressedInputStream__Make
-std::shared_ptr<arrow::io::CompressedInputStream> io___CompressedInputStream__Make(const std::unique_ptr<arrow::util::Codec>& codec, const std::shared_ptr<arrow::io::InputStream>& raw);
-RcppExport SEXP _arrow_io___CompressedInputStream__Make(SEXP codecSEXP, SEXP rawSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::unique_ptr<arrow::util::Codec>& >::type codec(codecSEXP);
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::io::InputStream>& >::type raw(rawSEXP);
-    rcpp_result_gen = Rcpp::wrap(io___CompressedInputStream__Make(codec, raw));
-    return rcpp_result_gen;
-END_RCPP
-}
-// compute___CastOptions__initialize
-std::shared_ptr<arrow::compute::CastOptions> compute___CastOptions__initialize(bool allow_int_overflow, bool allow_time_truncate, bool allow_float_truncate);
-RcppExport SEXP _arrow_compute___CastOptions__initialize(SEXP allow_int_overflowSEXP, SEXP allow_time_truncateSEXP, SEXP allow_float_truncateSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< bool >::type allow_int_overflow(allow_int_overflowSEXP);
-    Rcpp::traits::input_parameter< bool >::type allow_time_truncate(allow_time_truncateSEXP);
-    Rcpp::traits::input_parameter< bool >::type allow_float_truncate(allow_float_truncateSEXP);
-    rcpp_result_gen = Rcpp::wrap(compute___CastOptions__initialize(allow_int_overflow, allow_time_truncate, allow_float_truncate));
-    return rcpp_result_gen;
-END_RCPP
-}
-// Array__cast
-std::shared_ptr<arrow::Array> Array__cast(const std::shared_ptr<arrow::Array>& array, const std::shared_ptr<arrow::DataType>& target_type, const std::shared_ptr<arrow::compute::CastOptions>& options);
-RcppExport SEXP _arrow_Array__cast(SEXP arraySEXP, SEXP target_typeSEXP, SEXP optionsSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::Array>& >::type array(arraySEXP);
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::DataType>& >::type target_type(target_typeSEXP);
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::compute::CastOptions>& >::type options(optionsSEXP);
-    rcpp_result_gen = Rcpp::wrap(Array__cast(array, target_type, options));
-    return rcpp_result_gen;
-END_RCPP
-}
-// ChunkedArray__cast
-std::shared_ptr<arrow::ChunkedArray> ChunkedArray__cast(const std::shared_ptr<arrow::ChunkedArray>& chunked_array, const std::shared_ptr<arrow::DataType>& target_type, const std::shared_ptr<arrow::compute::CastOptions>& options);
-RcppExport SEXP _arrow_ChunkedArray__cast(SEXP chunked_arraySEXP, SEXP target_typeSEXP, SEXP optionsSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::ChunkedArray>& >::type chunked_array(chunked_arraySEXP);
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::DataType>& >::type target_type(target_typeSEXP);
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::compute::CastOptions>& >::type options(optionsSEXP);
-    rcpp_result_gen = Rcpp::wrap(ChunkedArray__cast(chunked_array, target_type, options));
-    return rcpp_result_gen;
-END_RCPP
-}
-// RecordBatch__cast
-std::shared_ptr<arrow::RecordBatch> RecordBatch__cast(const std::shared_ptr<arrow::RecordBatch>& batch, const std::shared_ptr<arrow::Schema>& schema, const std::shared_ptr<arrow::compute::CastOptions>& options);
-RcppExport SEXP _arrow_RecordBatch__cast(SEXP batchSEXP, SEXP schemaSEXP, SEXP optionsSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::RecordBatch>& >::type batch(batchSEXP);
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::Schema>& >::type schema(schemaSEXP);
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::compute::CastOptions>& >::type options(optionsSEXP);
-    rcpp_result_gen = Rcpp::wrap(RecordBatch__cast(batch, schema, options));
-    return rcpp_result_gen;
-END_RCPP
-}
-// Table__cast
-std::shared_ptr<arrow::Table> Table__cast(const std::shared_ptr<arrow::Table>& table, const std::shared_ptr<arrow::Schema>& schema, const std::shared_ptr<arrow::compute::CastOptions>& options);
-RcppExport SEXP _arrow_Table__cast(SEXP tableSEXP, SEXP schemaSEXP, SEXP optionsSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::Table>& >::type table(tableSEXP);
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::Schema>& >::type schema(schemaSEXP);
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::compute::CastOptions>& >::type options(optionsSEXP);
-    rcpp_result_gen = Rcpp::wrap(Table__cast(table, schema, options));
-    return rcpp_result_gen;
-END_RCPP
-}
-// csv___ReadOptions__initialize
-std::shared_ptr<arrow::csv::ReadOptions> csv___ReadOptions__initialize(List_ options);
-RcppExport SEXP _arrow_csv___ReadOptions__initialize(SEXP optionsSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< List_ >::type options(optionsSEXP);
-    rcpp_result_gen = Rcpp::wrap(csv___ReadOptions__initialize(options));
-    return rcpp_result_gen;
-END_RCPP
-}
-// csv___ParseOptions__initialize
-std::shared_ptr<arrow::csv::ParseOptions> csv___ParseOptions__initialize(List_ options);
-RcppExport SEXP _arrow_csv___ParseOptions__initialize(SEXP optionsSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< List_ >::type options(optionsSEXP);
-    rcpp_result_gen = Rcpp::wrap(csv___ParseOptions__initialize(options));
-    return rcpp_result_gen;
-END_RCPP
-}
-// csv___ConvertOptions__initialize
-std::shared_ptr<arrow::csv::ConvertOptions> csv___ConvertOptions__initialize(List_ options);
-RcppExport SEXP _arrow_csv___ConvertOptions__initialize(SEXP optionsSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< List_ >::type options(optionsSEXP);
-    rcpp_result_gen = Rcpp::wrap(csv___ConvertOptions__initialize(options));
-    return rcpp_result_gen;
-END_RCPP
-}
-// csv___TableReader__Make
-std::shared_ptr<arrow::csv::TableReader> csv___TableReader__Make(const std::shared_ptr<arrow::io::InputStream>& input, const std::shared_ptr<arrow::csv::ReadOptions>& read_options, const std::shared_ptr<arrow::csv::ParseOptions>& parse_options, const std::shared_ptr<arrow::csv::ConvertOptions>& convert_options);
-RcppExport SEXP _arrow_csv___TableReader__Make(SEXP inputSEXP, SEXP read_optionsSEXP, SEXP parse_optionsSEXP, SEXP convert_optionsSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::io::InputStream>& >::type input(inputSEXP);
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::csv::ReadOptions>& >::type read_options(read_optionsSEXP);
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::csv::ParseOptions>& >::type parse_options(parse_optionsSEXP);
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::csv::ConvertOptions>& >::type convert_options(convert_optionsSEXP);
-    rcpp_result_gen = Rcpp::wrap(csv___TableReader__Make(input, read_options, parse_options, convert_options));
-    return rcpp_result_gen;
-END_RCPP
-}
-// csv___TableReader__Read
-std::shared_ptr<arrow::Table> csv___TableReader__Read(const std::shared_ptr<arrow::csv::TableReader>& table_reader);
-RcppExport SEXP _arrow_csv___TableReader__Read(SEXP table_readerSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::csv::TableReader>& >::type table_reader(table_readerSEXP);
-    rcpp_result_gen = Rcpp::wrap(csv___TableReader__Read(table_reader));
-    return rcpp_result_gen;
-END_RCPP
-}
-// shared_ptr_is_null
-bool shared_ptr_is_null(SEXP xp);
-RcppExport SEXP _arrow_shared_ptr_is_null(SEXP xpSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< SEXP >::type xp(xpSEXP);
-    rcpp_result_gen = Rcpp::wrap(shared_ptr_is_null(xp));
-    return rcpp_result_gen;
-END_RCPP
-}
-// unique_ptr_is_null
-bool unique_ptr_is_null(SEXP xp);
-RcppExport SEXP _arrow_unique_ptr_is_null(SEXP xpSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< SEXP >::type xp(xpSEXP);
-    rcpp_result_gen = Rcpp::wrap(unique_ptr_is_null(xp));
-    return rcpp_result_gen;
-END_RCPP
-}
-// Int8__initialize
-std::shared_ptr<arrow::DataType> Int8__initialize();
-RcppExport SEXP _arrow_Int8__initialize() {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    rcpp_result_gen = Rcpp::wrap(Int8__initialize());
-    return rcpp_result_gen;
-END_RCPP
-}
-// Int16__initialize
-std::shared_ptr<arrow::DataType> Int16__initialize();
-RcppExport SEXP _arrow_Int16__initialize() {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    rcpp_result_gen = Rcpp::wrap(Int16__initialize());
-    return rcpp_result_gen;
-END_RCPP
-}
-// Int32__initialize
-std::shared_ptr<arrow::DataType> Int32__initialize();
-RcppExport SEXP _arrow_Int32__initialize() {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    rcpp_result_gen = Rcpp::wrap(Int32__initialize());
-    return rcpp_result_gen;
-END_RCPP
-}
-// Int64__initialize
-std::shared_ptr<arrow::DataType> Int64__initialize();
-RcppExport SEXP _arrow_Int64__initialize() {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    rcpp_result_gen = Rcpp::wrap(Int64__initialize());
-    return rcpp_result_gen;
-END_RCPP
-}
-// UInt8__initialize
-std::shared_ptr<arrow::DataType> UInt8__initialize();
-RcppExport SEXP _arrow_UInt8__initialize() {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    rcpp_result_gen = Rcpp::wrap(UInt8__initialize());
-    return rcpp_result_gen;
-END_RCPP
-}
-// UInt16__initialize
-std::shared_ptr<arrow::DataType> UInt16__initialize();
-RcppExport SEXP _arrow_UInt16__initialize() {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    rcpp_result_gen = Rcpp::wrap(UInt16__initialize());
-    return rcpp_result_gen;
-END_RCPP
-}
-// UInt32__initialize
-std::shared_ptr<arrow::DataType> UInt32__initialize();
-RcppExport SEXP _arrow_UInt32__initialize() {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    rcpp_result_gen = Rcpp::wrap(UInt32__initialize());
-    return rcpp_result_gen;
-END_RCPP
-}
-// UInt64__initialize
-std::shared_ptr<arrow::DataType> UInt64__initialize();
-RcppExport SEXP _arrow_UInt64__initialize() {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    rcpp_result_gen = Rcpp::wrap(UInt64__initialize());
-    return rcpp_result_gen;
-END_RCPP
-}
-// Float16__initialize
-std::shared_ptr<arrow::DataType> Float16__initialize();
-RcppExport SEXP _arrow_Float16__initialize() {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    rcpp_result_gen = Rcpp::wrap(Float16__initialize());
-    return rcpp_result_gen;
-END_RCPP
-}
-// Float32__initialize
-std::shared_ptr<arrow::DataType> Float32__initialize();
-RcppExport SEXP _arrow_Float32__initialize() {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    rcpp_result_gen = Rcpp::wrap(Float32__initialize());
-    return rcpp_result_gen;
-END_RCPP
-}
-// Float64__initialize
-std::shared_ptr<arrow::DataType> Float64__initialize();
-RcppExport SEXP _arrow_Float64__initialize() {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    rcpp_result_gen = Rcpp::wrap(Float64__initialize());
-    return rcpp_result_gen;
-END_RCPP
-}
-// Boolean__initialize
-std::shared_ptr<arrow::DataType> Boolean__initialize();
-RcppExport SEXP _arrow_Boolean__initialize() {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    rcpp_result_gen = Rcpp::wrap(Boolean__initialize());
-    return rcpp_result_gen;
-END_RCPP
-}
-// Utf8__initialize
-std::shared_ptr<arrow::DataType> Utf8__initialize();
-RcppExport SEXP _arrow_Utf8__initialize() {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    rcpp_result_gen = Rcpp::wrap(Utf8__initialize());
-    return rcpp_result_gen;
-END_RCPP
-}
-// Date32__initialize
-std::shared_ptr<arrow::DataType> Date32__initialize();
-RcppExport SEXP _arrow_Date32__initialize() {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    rcpp_result_gen = Rcpp::wrap(Date32__initialize());
-    return rcpp_result_gen;
-END_RCPP
-}
-// Date64__initialize
-std::shared_ptr<arrow::DataType> Date64__initialize();
-RcppExport SEXP _arrow_Date64__initialize() {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    rcpp_result_gen = Rcpp::wrap(Date64__initialize());
-    return rcpp_result_gen;
-END_RCPP
-}
-// Null__initialize
-std::shared_ptr<arrow::DataType> Null__initialize();
-RcppExport SEXP _arrow_Null__initialize() {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    rcpp_result_gen = Rcpp::wrap(Null__initialize());
-    return rcpp_result_gen;
-END_RCPP
-}
-// Decimal128Type__initialize
-std::shared_ptr<arrow::DataType> Decimal128Type__initialize(int32_t precision, int32_t scale);
-RcppExport SEXP _arrow_Decimal128Type__initialize(SEXP precisionSEXP, SEXP scaleSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< int32_t >::type precision(precisionSEXP);
-    Rcpp::traits::input_parameter< int32_t >::type scale(scaleSEXP);
-    rcpp_result_gen = Rcpp::wrap(Decimal128Type__initialize(precision, scale));
-    return rcpp_result_gen;
-END_RCPP
-}
-// FixedSizeBinary__initialize
-std::shared_ptr<arrow::DataType> FixedSizeBinary__initialize(int32_t byte_width);
-RcppExport SEXP _arrow_FixedSizeBinary__initialize(SEXP byte_widthSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< int32_t >::type byte_width(byte_widthSEXP);
-    rcpp_result_gen = Rcpp::wrap(FixedSizeBinary__initialize(byte_width));
-    return rcpp_result_gen;
-END_RCPP
-}
-// Timestamp__initialize1
-std::shared_ptr<arrow::DataType> Timestamp__initialize1(arrow::TimeUnit::type unit);
-RcppExport SEXP _arrow_Timestamp__initialize1(SEXP unitSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< arrow::TimeUnit::type >::type unit(unitSEXP);
-    rcpp_result_gen = Rcpp::wrap(Timestamp__initialize1(unit));
-    return rcpp_result_gen;
-END_RCPP
-}
-// Timestamp__initialize2
-std::shared_ptr<arrow::DataType> Timestamp__initialize2(arrow::TimeUnit::type unit, const std::string& timezone);
-RcppExport SEXP _arrow_Timestamp__initialize2(SEXP unitSEXP, SEXP timezoneSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< arrow::TimeUnit::type >::type unit(unitSEXP);
-    Rcpp::traits::input_parameter< const std::string& >::type timezone(timezoneSEXP);
-    rcpp_result_gen = Rcpp::wrap(Timestamp__initialize2(unit, timezone));
-    return rcpp_result_gen;
-END_RCPP
-}
-// Time32__initialize
-std::shared_ptr<arrow::DataType> Time32__initialize(arrow::TimeUnit::type unit);
-RcppExport SEXP _arrow_Time32__initialize(SEXP unitSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< arrow::TimeUnit::type >::type unit(unitSEXP);
-    rcpp_result_gen = Rcpp::wrap(Time32__initialize(unit));
-    return rcpp_result_gen;
-END_RCPP
-}
-// Time64__initialize
-std::shared_ptr<arrow::DataType> Time64__initialize(arrow::TimeUnit::type unit);
-RcppExport SEXP _arrow_Time64__initialize(SEXP unitSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< arrow::TimeUnit::type >::type unit(unitSEXP);
-    rcpp_result_gen = Rcpp::wrap(Time64__initialize(unit));
-    return rcpp_result_gen;
-END_RCPP
-}
-// list__
-SEXP list__(SEXP x);
-RcppExport SEXP _arrow_list__(SEXP xSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< SEXP >::type x(xSEXP);
-    rcpp_result_gen = Rcpp::wrap(list__(x));
-    return rcpp_result_gen;
-END_RCPP
-}
-// struct_
-std::shared_ptr<arrow::DataType> struct_(List fields);
-RcppExport SEXP _arrow_struct_(SEXP fieldsSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< List >::type fields(fieldsSEXP);
-    rcpp_result_gen = Rcpp::wrap(struct_(fields));
-    return rcpp_result_gen;
-END_RCPP
-}
-// DataType__ToString
-std::string DataType__ToString(const std::shared_ptr<arrow::DataType>& type);
-RcppExport SEXP _arrow_DataType__ToString(SEXP typeSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::DataType>& >::type type(typeSEXP);
-    rcpp_result_gen = Rcpp::wrap(DataType__ToString(type));
-    return rcpp_result_gen;
-END_RCPP
-}
-// DataType__name
-std::string DataType__name(const std::shared_ptr<arrow::DataType>& type);
-RcppExport SEXP _arrow_DataType__name(SEXP typeSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::DataType>& >::type type(typeSEXP);
-    rcpp_result_gen = Rcpp::wrap(DataType__name(type));
-    return rcpp_result_gen;
-END_RCPP
-}
-// DataType__Equals
-bool DataType__Equals(const std::shared_ptr<arrow::DataType>& lhs, const std::shared_ptr<arrow::DataType>& rhs);
-RcppExport SEXP _arrow_DataType__Equals(SEXP lhsSEXP, SEXP rhsSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::DataType>& >::type lhs(lhsSEXP);
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::DataType>& >::type rhs(rhsSEXP);
-    rcpp_result_gen = Rcpp::wrap(DataType__Equals(lhs, rhs));
-    return rcpp_result_gen;
-END_RCPP
-}
-// DataType__num_children
-int DataType__num_children(const std::shared_ptr<arrow::DataType>& type);
-RcppExport SEXP _arrow_DataType__num_children(SEXP typeSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::DataType>& >::type type(typeSEXP);
-    rcpp_result_gen = Rcpp::wrap(DataType__num_children(type));
-    return rcpp_result_gen;
-END_RCPP
-}
-// DataType__children_pointer
-List DataType__children_pointer(const std::shared_ptr<arrow::DataType>& type);
-RcppExport SEXP _arrow_DataType__children_pointer(SEXP typeSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::DataType>& >::type type(typeSEXP);
-    rcpp_result_gen = Rcpp::wrap(DataType__children_pointer(type));
-    return rcpp_result_gen;
-END_RCPP
-}
-// DataType__id
-arrow::Type::type DataType__id(const std::shared_ptr<arrow::DataType>& type);
-RcppExport SEXP _arrow_DataType__id(SEXP typeSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::DataType>& >::type type(typeSEXP);
-    rcpp_result_gen = Rcpp::wrap(DataType__id(type));
-    return rcpp_result_gen;
-END_RCPP
-}
-// schema_
-std::shared_ptr<arrow::Schema> schema_(List fields);
-RcppExport SEXP _arrow_schema_(SEXP fieldsSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< List >::type fields(fieldsSEXP);
-    rcpp_result_gen = Rcpp::wrap(schema_(fields));
-    return rcpp_result_gen;
-END_RCPP
-}
-// Schema__ToString
-std::string Schema__ToString(const std::shared_ptr<arrow::Schema>& s);
-RcppExport SEXP _arrow_Schema__ToString(SEXP sSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::Schema>& >::type s(sSEXP);
-    rcpp_result_gen = Rcpp::wrap(Schema__ToString(s));
-    return rcpp_result_gen;
-END_RCPP
-}
-// Schema__num_fields
-int Schema__num_fields(const std::shared_ptr<arrow::Schema>& s);
-RcppExport SEXP _arrow_Schema__num_fields(SEXP sSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::Schema>& >::type s(sSEXP);
-    rcpp_result_gen = Rcpp::wrap(Schema__num_fields(s));
-    return rcpp_result_gen;
-END_RCPP
-}
-// Schema__field
-std::shared_ptr<arrow::Field> Schema__field(const std::shared_ptr<arrow::Schema>& s, int i);
-RcppExport SEXP _arrow_Schema__field(SEXP sSEXP, SEXP iSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::Schema>& >::type s(sSEXP);
-    Rcpp::traits::input_parameter< int >::type i(iSEXP);
-    rcpp_result_gen = Rcpp::wrap(Schema__field(s, i));
-    return rcpp_result_gen;
-END_RCPP
-}
-// Schema__names
-CharacterVector Schema__names(const std::shared_ptr<arrow::Schema>& schema);
-RcppExport SEXP _arrow_Schema__names(SEXP schemaSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::Schema>& >::type schema(schemaSEXP);
-    rcpp_result_gen = Rcpp::wrap(Schema__names(schema));
-    return rcpp_result_gen;
-END_RCPP
-}
-// ListType__ToString
-std::string ListType__ToString(const std::shared_ptr<arrow::ListType>& type);
-RcppExport SEXP _arrow_ListType__ToString(SEXP typeSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::ListType>& >::type type(typeSEXP);
-    rcpp_result_gen = Rcpp::wrap(ListType__ToString(type));
-    return rcpp_result_gen;
-END_RCPP
-}
-// FixedWidthType__bit_width
-int FixedWidthType__bit_width(const std::shared_ptr<arrow::FixedWidthType>& type);
-RcppExport SEXP _arrow_FixedWidthType__bit_width(SEXP typeSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::FixedWidthType>& >::type type(typeSEXP);
-    rcpp_result_gen = Rcpp::wrap(FixedWidthType__bit_width(type));
-    return rcpp_result_gen;
-END_RCPP
-}
-// DateType__unit
-arrow::DateUnit DateType__unit(const std::shared_ptr<arrow::DateType>& type);
-RcppExport SEXP _arrow_DateType__unit(SEXP typeSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::DateType>& >::type type(typeSEXP);
-    rcpp_result_gen = Rcpp::wrap(DateType__unit(type));
-    return rcpp_result_gen;
-END_RCPP
-}
-// TimeType__unit
-arrow::TimeUnit::type TimeType__unit(const std::shared_ptr<arrow::TimeType>& type);
-RcppExport SEXP _arrow_TimeType__unit(SEXP typeSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::TimeType>& >::type type(typeSEXP);
-    rcpp_result_gen = Rcpp::wrap(TimeType__unit(type));
-    return rcpp_result_gen;
-END_RCPP
-}
-// DecimalType__precision
-int32_t DecimalType__precision(const std::shared_ptr<arrow::DecimalType>& type);
-RcppExport SEXP _arrow_DecimalType__precision(SEXP typeSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::DecimalType>& >::type type(typeSEXP);
-    rcpp_result_gen = Rcpp::wrap(DecimalType__precision(type));
-    return rcpp_result_gen;
-END_RCPP
-}
-// DecimalType__scale
-int32_t DecimalType__scale(const std::shared_ptr<arrow::DecimalType>& type);
-RcppExport SEXP _arrow_DecimalType__scale(SEXP typeSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::DecimalType>& >::type type(typeSEXP);
-    rcpp_result_gen = Rcpp::wrap(DecimalType__scale(type));
-    return rcpp_result_gen;
-END_RCPP
-}
-// TimestampType__timezone
-std::string TimestampType__timezone(const std::shared_ptr<arrow::TimestampType>& type);
-RcppExport SEXP _arrow_TimestampType__timezone(SEXP typeSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::TimestampType>& >::type type(typeSEXP);
-    rcpp_result_gen = Rcpp::wrap(TimestampType__timezone(type));
-    return rcpp_result_gen;
-END_RCPP
-}
-// TimestampType__unit
-arrow::TimeUnit::type TimestampType__unit(const std::shared_ptr<arrow::TimestampType>& type);
-RcppExport SEXP _arrow_TimestampType__unit(SEXP typeSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::TimestampType>& >::type type(typeSEXP);
-    rcpp_result_gen = Rcpp::wrap(TimestampType__unit(type));
-    return rcpp_result_gen;
-END_RCPP
-}
-// DictionaryType__initialize
-std::shared_ptr<arrow::DataType> DictionaryType__initialize(const std::shared_ptr<arrow::DataType>& index_type, const std::shared_ptr<arrow::DataType>& value_type, bool ordered);
-RcppExport SEXP _arrow_DictionaryType__initialize(SEXP index_typeSEXP, SEXP value_typeSEXP, SEXP orderedSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::DataType>& >::type index_type(index_typeSEXP);
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::DataType>& >::type value_type(value_typeSEXP);
-    Rcpp::traits::input_parameter< bool >::type ordered(orderedSEXP);
-    rcpp_result_gen = Rcpp::wrap(DictionaryType__initialize(index_type, value_type, ordered));
-    return rcpp_result_gen;
-END_RCPP
-}
-// DictionaryType__index_type
-std::shared_ptr<arrow::DataType> DictionaryType__index_type(const std::shared_ptr<arrow::DictionaryType>& type);
-RcppExport SEXP _arrow_DictionaryType__index_type(SEXP typeSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::DictionaryType>& >::type type(typeSEXP);
-    rcpp_result_gen = Rcpp::wrap(DictionaryType__index_type(type));
-    return rcpp_result_gen;
-END_RCPP
-}
-// DictionaryType__value_type
-std::shared_ptr<arrow::DataType> DictionaryType__value_type(const std::shared_ptr<arrow::DictionaryType>& type);
-RcppExport SEXP _arrow_DictionaryType__value_type(SEXP typeSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::DictionaryType>& >::type type(typeSEXP);
-    rcpp_result_gen = Rcpp::wrap(DictionaryType__value_type(type));
-    return rcpp_result_gen;
-END_RCPP
-}
-// DictionaryType__name
-std::string DictionaryType__name(const std::shared_ptr<arrow::DictionaryType>& type);
-RcppExport SEXP _arrow_DictionaryType__name(SEXP typeSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::DictionaryType>& >::type type(typeSEXP);
-    rcpp_result_gen = Rcpp::wrap(DictionaryType__name(type));
-    return rcpp_result_gen;
-END_RCPP
-}
-// DictionaryType__ordered
-bool DictionaryType__ordered(const std::shared_ptr<arrow::DictionaryType>& type);
-RcppExport SEXP _arrow_DictionaryType__ordered(SEXP typeSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::DictionaryType>& >::type type(typeSEXP);
-    rcpp_result_gen = Rcpp::wrap(DictionaryType__ordered(type));
-    return rcpp_result_gen;
-END_RCPP
-}
-// ipc___feather___TableWriter__SetDescription
-void ipc___feather___TableWriter__SetDescription(const std::unique_ptr<arrow::ipc::feather::TableWriter>& writer, const std::string& description);
-RcppExport SEXP _arrow_ipc___feather___TableWriter__SetDescription(SEXP writerSEXP, SEXP descriptionSEXP) {
-BEGIN_RCPP
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::unique_ptr<arrow::ipc::feather::TableWriter>& >::type writer(writerSEXP);
-    Rcpp::traits::input_parameter< const std::string& >::type description(descriptionSEXP);
-    ipc___feather___TableWriter__SetDescription(writer, description);
-    return R_NilValue;
-END_RCPP
-}
-// ipc___feather___TableWriter__SetNumRows
-void ipc___feather___TableWriter__SetNumRows(const std::unique_ptr<arrow::ipc::feather::TableWriter>& writer, int64_t num_rows);
-RcppExport SEXP _arrow_ipc___feather___TableWriter__SetNumRows(SEXP writerSEXP, SEXP num_rowsSEXP) {
-BEGIN_RCPP
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::unique_ptr<arrow::ipc::feather::TableWriter>& >::type writer(writerSEXP);
-    Rcpp::traits::input_parameter< int64_t >::type num_rows(num_rowsSEXP);
-    ipc___feather___TableWriter__SetNumRows(writer, num_rows);
-    return R_NilValue;
-END_RCPP
-}
-// ipc___feather___TableWriter__Append
-void ipc___feather___TableWriter__Append(const std::unique_ptr<arrow::ipc::feather::TableWriter>& writer, const std::string& name, const std::shared_ptr<arrow::Array>& values);
-RcppExport SEXP _arrow_ipc___feather___TableWriter__Append(SEXP writerSEXP, SEXP nameSEXP, SEXP valuesSEXP) {
-BEGIN_RCPP
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::unique_ptr<arrow::ipc::feather::TableWriter>& >::type writer(writerSEXP);
-    Rcpp::traits::input_parameter< const std::string& >::type name(nameSEXP);
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::Array>& >::type values(valuesSEXP);
-    ipc___feather___TableWriter__Append(writer, name, values);
-    return R_NilValue;
-END_RCPP
-}
-// ipc___feather___TableWriter__Finalize
-void ipc___feather___TableWriter__Finalize(const std::unique_ptr<arrow::ipc::feather::TableWriter>& writer);
-RcppExport SEXP _arrow_ipc___feather___TableWriter__Finalize(SEXP writerSEXP) {
-BEGIN_RCPP
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::unique_ptr<arrow::ipc::feather::TableWriter>& >::type writer(writerSEXP);
-    ipc___feather___TableWriter__Finalize(writer);
-    return R_NilValue;
-END_RCPP
-}
-// ipc___feather___TableWriter__Open
-std::unique_ptr<arrow::ipc::feather::TableWriter> ipc___feather___TableWriter__Open(const std::shared_ptr<arrow::io::OutputStream>& stream);
-RcppExport SEXP _arrow_ipc___feather___TableWriter__Open(SEXP streamSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::io::OutputStream>& >::type stream(streamSEXP);
-    rcpp_result_gen = Rcpp::wrap(ipc___feather___TableWriter__Open(stream));
-    return rcpp_result_gen;
-END_RCPP
-}
-// ipc___TableWriter__RecordBatch__WriteFeather
-void ipc___TableWriter__RecordBatch__WriteFeather(const std::unique_ptr<arrow::ipc::feather::TableWriter>& writer, const std::shared_ptr<arrow::RecordBatch>& batch);
-RcppExport SEXP _arrow_ipc___TableWriter__RecordBatch__WriteFeather(SEXP writerSEXP, SEXP batchSEXP) {
-BEGIN_RCPP
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::unique_ptr<arrow::ipc::feather::TableWriter>& >::type writer(writerSEXP);
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::RecordBatch>& >::type batch(batchSEXP);
-    ipc___TableWriter__RecordBatch__WriteFeather(writer, batch);
-    return R_NilValue;
-END_RCPP
-}
-// ipc___feather___TableReader__GetDescription
-std::string ipc___feather___TableReader__GetDescription(const std::unique_ptr<arrow::ipc::feather::TableReader>& reader);
-RcppExport SEXP _arrow_ipc___feather___TableReader__GetDescription(SEXP readerSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::unique_ptr<arrow::ipc::feather::TableReader>& >::type reader(readerSEXP);
-    rcpp_result_gen = Rcpp::wrap(ipc___feather___TableReader__GetDescription(reader));
-    return rcpp_result_gen;
-END_RCPP
-}
-// ipc___feather___TableReader__HasDescription
-bool ipc___feather___TableReader__HasDescription(const std::unique_ptr<arrow::ipc::feather::TableReader>& reader);
-RcppExport SEXP _arrow_ipc___feather___TableReader__HasDescription(SEXP readerSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::unique_ptr<arrow::ipc::feather::TableReader>& >::type reader(readerSEXP);
-    rcpp_result_gen = Rcpp::wrap(ipc___feather___TableReader__HasDescription(reader));
-    return rcpp_result_gen;
-END_RCPP
-}
-// ipc___feather___TableReader__version
-int ipc___feather___TableReader__version(const std::unique_ptr<arrow::ipc::feather::TableReader>& reader);
-RcppExport SEXP _arrow_ipc___feather___TableReader__version(SEXP readerSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::unique_ptr<arrow::ipc::feather::TableReader>& >::type reader(readerSEXP);
-    rcpp_result_gen = Rcpp::wrap(ipc___feather___TableReader__version(reader));
-    return rcpp_result_gen;
-END_RCPP
-}
-// ipc___feather___TableReader__num_rows
-int64_t ipc___feather___TableReader__num_rows(const std::unique_ptr<arrow::ipc::feather::TableReader>& reader);
-RcppExport SEXP _arrow_ipc___feather___TableReader__num_rows(SEXP readerSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::unique_ptr<arrow::ipc::feather::TableReader>& >::type reader(readerSEXP);
-    rcpp_result_gen = Rcpp::wrap(ipc___feather___TableReader__num_rows(reader));
-    return rcpp_result_gen;
-END_RCPP
-}
-// ipc___feather___TableReader__num_columns
-int64_t ipc___feather___TableReader__num_columns(const std::unique_ptr<arrow::ipc::feather::TableReader>& reader);
-RcppExport SEXP _arrow_ipc___feather___TableReader__num_columns(SEXP readerSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::unique_ptr<arrow::ipc::feather::TableReader>& >::type reader(readerSEXP);
-    rcpp_result_gen = Rcpp::wrap(ipc___feather___TableReader__num_columns(reader));
-    return rcpp_result_gen;
-END_RCPP
-}
-// ipc___feather___TableReader__GetColumnName
-std::string ipc___feather___TableReader__GetColumnName(const std::unique_ptr<arrow::ipc::feather::TableReader>& reader, int i);
-RcppExport SEXP _arrow_ipc___feather___TableReader__GetColumnName(SEXP readerSEXP, SEXP iSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::unique_ptr<arrow::ipc::feather::TableReader>& >::type reader(readerSEXP);
-    Rcpp::traits::input_parameter< int >::type i(iSEXP);
-    rcpp_result_gen = Rcpp::wrap(ipc___feather___TableReader__GetColumnName(reader, i));
-    return rcpp_result_gen;
-END_RCPP
-}
-// ipc___feather___TableReader__GetColumn
-std::shared_ptr<arrow::Column> ipc___feather___TableReader__GetColumn(const std::unique_ptr<arrow::ipc::feather::TableReader>& reader, int i);
-RcppExport SEXP _arrow_ipc___feather___TableReader__GetColumn(SEXP readerSEXP, SEXP iSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::unique_ptr<arrow::ipc::feather::TableReader>& >::type reader(readerSEXP);
-    Rcpp::traits::input_parameter< int >::type i(iSEXP);
-    rcpp_result_gen = Rcpp::wrap(ipc___feather___TableReader__GetColumn(reader, i));
-    return rcpp_result_gen;
-END_RCPP
-}
-// ipc___feather___TableReader__Read
-std::shared_ptr<arrow::Table> ipc___feather___TableReader__Read(const std::unique_ptr<arrow::ipc::feather::TableReader>& reader, SEXP columns);
-RcppExport SEXP _arrow_ipc___feather___TableReader__Read(SEXP readerSEXP, SEXP columnsSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::unique_ptr<arrow::ipc::feather::TableReader>& >::type reader(readerSEXP);
-    Rcpp::traits::input_parameter< SEXP >::type columns(columnsSEXP);
-    rcpp_result_gen = Rcpp::wrap(ipc___feather___TableReader__Read(reader, columns));
-    return rcpp_result_gen;
-END_RCPP
-}
-// ipc___feather___TableReader__Open
-std::unique_ptr<arrow::ipc::feather::TableReader> ipc___feather___TableReader__Open(const std::shared_ptr<arrow::io::RandomAccessFile>& stream);
-RcppExport SEXP _arrow_ipc___feather___TableReader__Open(SEXP streamSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::io::RandomAccessFile>& >::type stream(streamSEXP);
-    rcpp_result_gen = Rcpp::wrap(ipc___feather___TableReader__Open(stream));
-    return rcpp_result_gen;
-END_RCPP
-}
-// Field__initialize
-std::shared_ptr<arrow::Field> Field__initialize(const std::string& name, const std::shared_ptr<arrow::DataType>& field, bool nullable);
-RcppExport SEXP _arrow_Field__initialize(SEXP nameSEXP, SEXP fieldSEXP, SEXP nullableSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::string& >::type name(nameSEXP);
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::DataType>& >::type field(fieldSEXP);
-    Rcpp::traits::input_parameter< bool >::type nullable(nullableSEXP);
-    rcpp_result_gen = Rcpp::wrap(Field__initialize(name, field, nullable));
-    return rcpp_result_gen;
-END_RCPP
-}
-// Field__ToString
-std::string Field__ToString(const std::shared_ptr<arrow::Field>& field);
-RcppExport SEXP _arrow_Field__ToString(SEXP fieldSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::Field>& >::type field(fieldSEXP);
-    rcpp_result_gen = Rcpp::wrap(Field__ToString(field));
-    return rcpp_result_gen;
-END_RCPP
-}
-// Field__name
-std::string Field__name(const std::shared_ptr<arrow::Field>& field);
-RcppExport SEXP _arrow_Field__name(SEXP fieldSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::Field>& >::type field(fieldSEXP);
-    rcpp_result_gen = Rcpp::wrap(Field__name(field));
-    return rcpp_result_gen;
-END_RCPP
-}
-// Field__Equals
-bool Field__Equals(const std::shared_ptr<arrow::Field>& field, const std::shared_ptr<arrow::Field>& other);
-RcppExport SEXP _arrow_Field__Equals(SEXP fieldSEXP, SEXP otherSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::Field>& >::type field(fieldSEXP);
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::Field>& >::type other(otherSEXP);
-    rcpp_result_gen = Rcpp::wrap(Field__Equals(field, other));
-    return rcpp_result_gen;
-END_RCPP
-}
-// Field__nullable
-bool Field__nullable(const std::shared_ptr<arrow::Field>& field);
-RcppExport SEXP _arrow_Field__nullable(SEXP fieldSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::Field>& >::type field(fieldSEXP);
-    rcpp_result_gen = Rcpp::wrap(Field__nullable(field));
-    return rcpp_result_gen;
-END_RCPP
-}
-// Field__type
-std::shared_ptr<arrow::DataType> Field__type(const std::shared_ptr<arrow::Field>& field);
-RcppExport SEXP _arrow_Field__type(SEXP fieldSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::Field>& >::type field(fieldSEXP);
-    rcpp_result_gen = Rcpp::wrap(Field__type(field));
-    return rcpp_result_gen;
-END_RCPP
-}
-// io___Readable__Read
-std::shared_ptr<arrow::Buffer> io___Readable__Read(const std::shared_ptr<arrow::io::Readable>& x, int64_t nbytes);
-RcppExport SEXP _arrow_io___Readable__Read(SEXP xSEXP, SEXP nbytesSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::io::Readable>& >::type x(xSEXP);
-    Rcpp::traits::input_parameter< int64_t >::type nbytes(nbytesSEXP);
-    rcpp_result_gen = Rcpp::wrap(io___Readable__Read(x, nbytes));
-    return rcpp_result_gen;
-END_RCPP
-}
-// io___InputStream__Close
-void io___InputStream__Close(const std::shared_ptr<arrow::io::InputStream>& x);
-RcppExport SEXP _arrow_io___InputStream__Close(SEXP xSEXP) {
-BEGIN_RCPP
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::io::InputStream>& >::type x(xSEXP);
-    io___InputStream__Close(x);
-    return R_NilValue;
-END_RCPP
-}
-// io___OutputStream__Close
-void io___OutputStream__Close(const std::shared_ptr<arrow::io::OutputStream>& x);
-RcppExport SEXP _arrow_io___OutputStream__Close(SEXP xSEXP) {
-BEGIN_RCPP
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::io::OutputStream>& >::type x(xSEXP);
-    io___OutputStream__Close(x);
-    return R_NilValue;
-END_RCPP
-}
-// io___RandomAccessFile__GetSize
-int64_t io___RandomAccessFile__GetSize(const std::shared_ptr<arrow::io::RandomAccessFile>& x);
-RcppExport SEXP _arrow_io___RandomAccessFile__GetSize(SEXP xSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::io::RandomAccessFile>& >::type x(xSEXP);
-    rcpp_result_gen = Rcpp::wrap(io___RandomAccessFile__GetSize(x));
-    return rcpp_result_gen;
-END_RCPP
-}
-// io___RandomAccessFile__supports_zero_copy
-bool io___RandomAccessFile__supports_zero_copy(const std::shared_ptr<arrow::io::RandomAccessFile>& x);
-RcppExport SEXP _arrow_io___RandomAccessFile__supports_zero_copy(SEXP xSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::io::RandomAccessFile>& >::type x(xSEXP);
-    rcpp_result_gen = Rcpp::wrap(io___RandomAccessFile__supports_zero_copy(x));
-    return rcpp_result_gen;
-END_RCPP
-}
-// io___RandomAccessFile__Seek
-void io___RandomAccessFile__Seek(const std::shared_ptr<arrow::io::RandomAccessFile>& x, int64_t position);
-RcppExport SEXP _arrow_io___RandomAccessFile__Seek(SEXP xSEXP, SEXP positionSEXP) {
-BEGIN_RCPP
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::io::RandomAccessFile>& >::type x(xSEXP);
-    Rcpp::traits::input_parameter< int64_t >::type position(positionSEXP);
-    io___RandomAccessFile__Seek(x, position);
-    return R_NilValue;
-END_RCPP
-}
-// io___RandomAccessFile__Tell
-int64_t io___RandomAccessFile__Tell(const std::shared_ptr<arrow::io::RandomAccessFile>& x);
-RcppExport SEXP _arrow_io___RandomAccessFile__Tell(SEXP xSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::io::RandomAccessFile>& >::type x(xSEXP);
-    rcpp_result_gen = Rcpp::wrap(io___RandomAccessFile__Tell(x));
-    return rcpp_result_gen;
-END_RCPP
-}
-// io___MemoryMappedFile__Create
-std::shared_ptr<arrow::io::MemoryMappedFile> io___MemoryMappedFile__Create(const std::string& path, int64_t size);
-RcppExport SEXP _arrow_io___MemoryMappedFile__Create(SEXP pathSEXP, SEXP sizeSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::string& >::type path(pathSEXP);
-    Rcpp::traits::input_parameter< int64_t >::type size(sizeSEXP);
-    rcpp_result_gen = Rcpp::wrap(io___MemoryMappedFile__Create(path, size));
-    return rcpp_result_gen;
-END_RCPP
-}
-// io___MemoryMappedFile__Open
-std::shared_ptr<arrow::io::MemoryMappedFile> io___MemoryMappedFile__Open(const std::string& path, arrow::io::FileMode::type mode);
-RcppExport SEXP _arrow_io___MemoryMappedFile__Open(SEXP pathSEXP, SEXP modeSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::string& >::type path(pathSEXP);
-    Rcpp::traits::input_parameter< arrow::io::FileMode::type >::type mode(modeSEXP);
-    rcpp_result_gen = Rcpp::wrap(io___MemoryMappedFile__Open(path, mode));
-    return rcpp_result_gen;
-END_RCPP
-}
-// io___MemoryMappedFile__Resize
-void io___MemoryMappedFile__Resize(const std::shared_ptr<arrow::io::MemoryMappedFile>& x, int64_t size);
-RcppExport SEXP _arrow_io___MemoryMappedFile__Resize(SEXP xSEXP, SEXP sizeSEXP) {
-BEGIN_RCPP
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::io::MemoryMappedFile>& >::type x(xSEXP);
-    Rcpp::traits::input_parameter< int64_t >::type size(sizeSEXP);
-    io___MemoryMappedFile__Resize(x, size);
-    return R_NilValue;
-END_RCPP
-}
-// io___ReadableFile__Open
-std::shared_ptr<arrow::io::ReadableFile> io___ReadableFile__Open(const std::string& path);
-RcppExport SEXP _arrow_io___ReadableFile__Open(SEXP pathSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::string& >::type path(pathSEXP);
-    rcpp_result_gen = Rcpp::wrap(io___ReadableFile__Open(path));
-    return rcpp_result_gen;
-END_RCPP
-}
-// io___BufferReader__initialize
-std::shared_ptr<arrow::io::BufferReader> io___BufferReader__initialize(const std::shared_ptr<arrow::Buffer>& buffer);
-RcppExport SEXP _arrow_io___BufferReader__initialize(SEXP bufferSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::Buffer>& >::type buffer(bufferSEXP);
-    rcpp_result_gen = Rcpp::wrap(io___BufferReader__initialize(buffer));
-    return rcpp_result_gen;
-END_RCPP
-}
-// io___Writable__write
-void io___Writable__write(const std::shared_ptr<arrow::io::Writable>& stream, const std::shared_ptr<arrow::Buffer>& buf);
-RcppExport SEXP _arrow_io___Writable__write(SEXP streamSEXP, SEXP bufSEXP) {
-BEGIN_RCPP
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::io::Writable>& >::type stream(streamSEXP);
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::Buffer>& >::type buf(bufSEXP);
-    io___Writable__write(stream, buf);
-    return R_NilValue;
-END_RCPP
-}
-// io___OutputStream__Tell
-int64_t io___OutputStream__Tell(const std::shared_ptr<arrow::io::OutputStream>& stream);
-RcppExport SEXP _arrow_io___OutputStream__Tell(SEXP streamSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::io::OutputStream>& >::type stream(streamSEXP);
-    rcpp_result_gen = Rcpp::wrap(io___OutputStream__Tell(stream));
-    return rcpp_result_gen;
-END_RCPP
-}
-// io___FileOutputStream__Open
-std::shared_ptr<arrow::io::FileOutputStream> io___FileOutputStream__Open(const std::string& path);
-RcppExport SEXP _arrow_io___FileOutputStream__Open(SEXP pathSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::string& >::type path(pathSEXP);
-    rcpp_result_gen = Rcpp::wrap(io___FileOutputStream__Open(path));
-    return rcpp_result_gen;
-END_RCPP
-}
-// io___BufferOutputStream__Create
-std::shared_ptr<arrow::io::BufferOutputStream> io___BufferOutputStream__Create(int64_t initial_capacity);
-RcppExport SEXP _arrow_io___BufferOutputStream__Create(SEXP initial_capacitySEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< int64_t >::type initial_capacity(initial_capacitySEXP);
-    rcpp_result_gen = Rcpp::wrap(io___BufferOutputStream__Create(initial_capacity));
-    return rcpp_result_gen;
-END_RCPP
-}
-// io___BufferOutputStream__capacity
-int64_t io___BufferOutputStream__capacity(const std::shared_ptr<arrow::io::BufferOutputStream>& stream);
-RcppExport SEXP _arrow_io___BufferOutputStream__capacity(SEXP streamSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::io::BufferOutputStream>& >::type stream(streamSEXP);
-    rcpp_result_gen = Rcpp::wrap(io___BufferOutputStream__capacity(stream));
-    return rcpp_result_gen;
-END_RCPP
-}
-// io___BufferOutputStream__Finish
-std::shared_ptr<arrow::Buffer> io___BufferOutputStream__Finish(const std::shared_ptr<arrow::io::BufferOutputStream>& stream);
-RcppExport SEXP _arrow_io___BufferOutputStream__Finish(SEXP streamSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::io::BufferOutputStream>& >::type stream(streamSEXP);
-    rcpp_result_gen = Rcpp::wrap(io___BufferOutputStream__Finish(stream));
-    return rcpp_result_gen;
-END_RCPP
-}
-// io___BufferOutputStream__Tell
-int64_t io___BufferOutputStream__Tell(const std::shared_ptr<arrow::io::BufferOutputStream>& stream);
-RcppExport SEXP _arrow_io___BufferOutputStream__Tell(SEXP streamSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::io::BufferOutputStream>& >::type stream(streamSEXP);
-    rcpp_result_gen = Rcpp::wrap(io___BufferOutputStream__Tell(stream));
-    return rcpp_result_gen;
-END_RCPP
-}
-// io___BufferOutputStream__Write
-void io___BufferOutputStream__Write(const std::shared_ptr<arrow::io::BufferOutputStream>& stream, RawVector_ bytes);
-RcppExport SEXP _arrow_io___BufferOutputStream__Write(SEXP streamSEXP, SEXP bytesSEXP) {
-BEGIN_RCPP
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::io::BufferOutputStream>& >::type stream(streamSEXP);
-    Rcpp::traits::input_parameter< RawVector_ >::type bytes(bytesSEXP);
-    io___BufferOutputStream__Write(stream, bytes);
-    return R_NilValue;
-END_RCPP
-}
-// io___MockOutputStream__initialize
-std::shared_ptr<arrow::io::MockOutputStream> io___MockOutputStream__initialize();
-RcppExport SEXP _arrow_io___MockOutputStream__initialize() {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    rcpp_result_gen = Rcpp::wrap(io___MockOutputStream__initialize());
-    return rcpp_result_gen;
-END_RCPP
-}
-// io___MockOutputStream__GetExtentBytesWritten
-int64_t io___MockOutputStream__GetExtentBytesWritten(const std::shared_ptr<arrow::io::MockOutputStream>& stream);
-RcppExport SEXP _arrow_io___MockOutputStream__GetExtentBytesWritten(SEXP streamSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::io::MockOutputStream>& >::type stream(streamSEXP);
-    rcpp_result_gen = Rcpp::wrap(io___MockOutputStream__GetExtentBytesWritten(stream));
-    return rcpp_result_gen;
-END_RCPP
-}
-// io___FixedSizeBufferWriter__initialize
-std::shared_ptr<arrow::io::FixedSizeBufferWriter> io___FixedSizeBufferWriter__initialize(const std::shared_ptr<arrow::Buffer>& buffer);
-RcppExport SEXP _arrow_io___FixedSizeBufferWriter__initialize(SEXP bufferSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::Buffer>& >::type buffer(bufferSEXP);
-    rcpp_result_gen = Rcpp::wrap(io___FixedSizeBufferWriter__initialize(buffer));
-    return rcpp_result_gen;
-END_RCPP
-}
-// MemoryPool__default
-std::shared_ptr<arrow::MemoryPool> MemoryPool__default();
-RcppExport SEXP _arrow_MemoryPool__default() {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    rcpp_result_gen = Rcpp::wrap(MemoryPool__default());
-    return rcpp_result_gen;
-END_RCPP
-}
-// MemoryPool__bytes_allocated
-int MemoryPool__bytes_allocated(const std::shared_ptr<arrow::MemoryPool>& pool);
-RcppExport SEXP _arrow_MemoryPool__bytes_allocated(SEXP poolSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::MemoryPool>& >::type pool(poolSEXP);
-    rcpp_result_gen = Rcpp::wrap(MemoryPool__bytes_allocated(pool));
-    return rcpp_result_gen;
-END_RCPP
-}
-// MemoryPool__max_memory
-int MemoryPool__max_memory(const std::shared_ptr<arrow::MemoryPool>& pool);
-RcppExport SEXP _arrow_MemoryPool__max_memory(SEXP poolSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::MemoryPool>& >::type pool(poolSEXP);
-    rcpp_result_gen = Rcpp::wrap(MemoryPool__max_memory(pool));
-    return rcpp_result_gen;
-END_RCPP
-}
-// ipc___Message__body_length
-int64_t ipc___Message__body_length(const std::unique_ptr<arrow::ipc::Message>& message);
-RcppExport SEXP _arrow_ipc___Message__body_length(SEXP messageSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::unique_ptr<arrow::ipc::Message>& >::type message(messageSEXP);
-    rcpp_result_gen = Rcpp::wrap(ipc___Message__body_length(message));
-    return rcpp_result_gen;
-END_RCPP
-}
-// ipc___Message__metadata
-std::shared_ptr<arrow::Buffer> ipc___Message__metadata(const std::unique_ptr<arrow::ipc::Message>& message);
-RcppExport SEXP _arrow_ipc___Message__metadata(SEXP messageSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::unique_ptr<arrow::ipc::Message>& >::type message(messageSEXP);
-    rcpp_result_gen = Rcpp::wrap(ipc___Message__metadata(message));
-    return rcpp_result_gen;
-END_RCPP
-}
-// ipc___Message__body
-std::shared_ptr<arrow::Buffer> ipc___Message__body(const std::unique_ptr<arrow::ipc::Message>& message);
-RcppExport SEXP _arrow_ipc___Message__body(SEXP messageSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::unique_ptr<arrow::ipc::Message>& >::type message(messageSEXP);
-    rcpp_result_gen = Rcpp::wrap(ipc___Message__body(message));
-    return rcpp_result_gen;
-END_RCPP
-}
-// ipc___Message__Verify
-int64_t ipc___Message__Verify(const std::unique_ptr<arrow::ipc::Message>& message);
-RcppExport SEXP _arrow_ipc___Message__Verify(SEXP messageSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::unique_ptr<arrow::ipc::Message>& >::type message(messageSEXP);
-    rcpp_result_gen = Rcpp::wrap(ipc___Message__Verify(message));
-    return rcpp_result_gen;
-END_RCPP
-}
-// ipc___Message__type
-arrow::ipc::Message::Type ipc___Message__type(const std::unique_ptr<arrow::ipc::Message>& message);
-RcppExport SEXP _arrow_ipc___Message__type(SEXP messageSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::unique_ptr<arrow::ipc::Message>& >::type message(messageSEXP);
-    rcpp_result_gen = Rcpp::wrap(ipc___Message__type(message));
-    return rcpp_result_gen;
-END_RCPP
-}
-// ipc___Message__Equals
-bool ipc___Message__Equals(const std::unique_ptr<arrow::ipc::Message>& x, const std::unique_ptr<arrow::ipc::Message>& y);
-RcppExport SEXP _arrow_ipc___Message__Equals(SEXP xSEXP, SEXP ySEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::unique_ptr<arrow::ipc::Message>& >::type x(xSEXP);
-    Rcpp::traits::input_parameter< const std::unique_ptr<arrow::ipc::Message>& >::type y(ySEXP);
-    rcpp_result_gen = Rcpp::wrap(ipc___Message__Equals(x, y));
-    return rcpp_result_gen;
-END_RCPP
-}
-// ipc___ReadRecordBatch__Message__Schema
-std::shared_ptr<arrow::RecordBatch> ipc___ReadRecordBatch__Message__Schema(const std::unique_ptr<arrow::ipc::Message>& message, const std::shared_ptr<arrow::Schema>& schema);
-RcppExport SEXP _arrow_ipc___ReadRecordBatch__Message__Schema(SEXP messageSEXP, SEXP schemaSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::unique_ptr<arrow::ipc::Message>& >::type message(messageSEXP);
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::Schema>& >::type schema(schemaSEXP);
-    rcpp_result_gen = Rcpp::wrap(ipc___ReadRecordBatch__Message__Schema(message, schema));
-    return rcpp_result_gen;
-END_RCPP
-}
-// ipc___ReadSchema_InputStream
-std::shared_ptr<arrow::Schema> ipc___ReadSchema_InputStream(const std::shared_ptr<arrow::io::InputStream>& stream);
-RcppExport SEXP _arrow_ipc___ReadSchema_InputStream(SEXP streamSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::io::InputStream>& >::type stream(streamSEXP);
-    rcpp_result_gen = Rcpp::wrap(ipc___ReadSchema_InputStream(stream));
-    return rcpp_result_gen;
-END_RCPP
-}
-// ipc___MessageReader__Open
-std::unique_ptr<arrow::ipc::MessageReader> ipc___MessageReader__Open(const std::shared_ptr<arrow::io::InputStream>& stream);
-RcppExport SEXP _arrow_ipc___MessageReader__Open(SEXP streamSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::io::InputStream>& >::type stream(streamSEXP);
-    rcpp_result_gen = Rcpp::wrap(ipc___MessageReader__Open(stream));
-    return rcpp_result_gen;
-END_RCPP
-}
-// ipc___MessageReader__ReadNextMessage
-std::unique_ptr<arrow::ipc::Message> ipc___MessageReader__ReadNextMessage(const std::unique_ptr<arrow::ipc::MessageReader>& reader);
-RcppExport SEXP _arrow_ipc___MessageReader__ReadNextMessage(SEXP readerSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::unique_ptr<arrow::ipc::MessageReader>& >::type reader(readerSEXP);
-    rcpp_result_gen = Rcpp::wrap(ipc___MessageReader__ReadNextMessage(reader));
-    return rcpp_result_gen;
-END_RCPP
-}
-// ipc___ReadMessage
-std::unique_ptr<arrow::ipc::Message> ipc___ReadMessage(const std::shared_ptr<arrow::io::InputStream>& stream);
-RcppExport SEXP _arrow_ipc___ReadMessage(SEXP streamSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::io::InputStream>& >::type stream(streamSEXP);
-    rcpp_result_gen = Rcpp::wrap(ipc___ReadMessage(stream));
-    return rcpp_result_gen;
-END_RCPP
-}
-// read_parquet_file
-std::shared_ptr<arrow::Table> read_parquet_file(std::string filename);
-RcppExport SEXP _arrow_read_parquet_file(SEXP filenameSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< std::string >::type filename(filenameSEXP);
-    rcpp_result_gen = Rcpp::wrap(read_parquet_file(filename));
-    return rcpp_result_gen;
-END_RCPP
-}
-// RecordBatch__num_columns
-int RecordBatch__num_columns(const std::shared_ptr<arrow::RecordBatch>& x);
-RcppExport SEXP _arrow_RecordBatch__num_columns(SEXP xSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::RecordBatch>& >::type x(xSEXP);
-    rcpp_result_gen = Rcpp::wrap(RecordBatch__num_columns(x));
-    return rcpp_result_gen;
-END_RCPP
-}
-// RecordBatch__num_rows
-int RecordBatch__num_rows(const std::shared_ptr<arrow::RecordBatch>& x);
-RcppExport SEXP _arrow_RecordBatch__num_rows(SEXP xSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::RecordBatch>& >::type x(xSEXP);
-    rcpp_result_gen = Rcpp::wrap(RecordBatch__num_rows(x));
-    return rcpp_result_gen;
-END_RCPP
-}
-// RecordBatch__schema
-std::shared_ptr<arrow::Schema> RecordBatch__schema(const std::shared_ptr<arrow::RecordBatch>& x);
-RcppExport SEXP _arrow_RecordBatch__schema(SEXP xSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::RecordBatch>& >::type x(xSEXP);
-    rcpp_result_gen = Rcpp::wrap(RecordBatch__schema(x));
-    return rcpp_result_gen;
-END_RCPP
-}
-// RecordBatch__columns
-arrow::ArrayVector RecordBatch__columns(const std::shared_ptr<arrow::RecordBatch>& batch);
-RcppExport SEXP _arrow_RecordBatch__columns(SEXP batchSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::RecordBatch>& >::type batch(batchSEXP);
-    rcpp_result_gen = Rcpp::wrap(RecordBatch__columns(batch));
-    return rcpp_result_gen;
-END_RCPP
-}
-// RecordBatch__column
-std::shared_ptr<arrow::Array> RecordBatch__column(const std::shared_ptr<arrow::RecordBatch>& batch, int i);
-RcppExport SEXP _arrow_RecordBatch__column(SEXP batchSEXP, SEXP iSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::RecordBatch>& >::type batch(batchSEXP);
-    Rcpp::traits::input_parameter< int >::type i(iSEXP);
-    rcpp_result_gen = Rcpp::wrap(RecordBatch__column(batch, i));
-    return rcpp_result_gen;
-END_RCPP
-}
-// RecordBatch__Equals
-bool RecordBatch__Equals(const std::shared_ptr<arrow::RecordBatch>& self, const std::shared_ptr<arrow::RecordBatch>& other);
-RcppExport SEXP _arrow_RecordBatch__Equals(SEXP selfSEXP, SEXP otherSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::RecordBatch>& >::type self(selfSEXP);
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::RecordBatch>& >::type other(otherSEXP);
-    rcpp_result_gen = Rcpp::wrap(RecordBatch__Equals(self, other));
-    return rcpp_result_gen;
-END_RCPP
-}
-// RecordBatch__RemoveColumn
-std::shared_ptr<arrow::RecordBatch> RecordBatch__RemoveColumn(const std::shared_ptr<arrow::RecordBatch>& batch, int i);
-RcppExport SEXP _arrow_RecordBatch__RemoveColumn(SEXP batchSEXP, SEXP iSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::RecordBatch>& >::type batch(batchSEXP);
-    Rcpp::traits::input_parameter< int >::type i(iSEXP);
-    rcpp_result_gen = Rcpp::wrap(RecordBatch__RemoveColumn(batch, i));
-    return rcpp_result_gen;
-END_RCPP
-}
-// RecordBatch__column_name
-std::string RecordBatch__column_name(const std::shared_ptr<arrow::RecordBatch>& batch, int i);
-RcppExport SEXP _arrow_RecordBatch__column_name(SEXP batchSEXP, SEXP iSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::RecordBatch>& >::type batch(batchSEXP);
-    Rcpp::traits::input_parameter< int >::type i(iSEXP);
-    rcpp_result_gen = Rcpp::wrap(RecordBatch__column_name(batch, i));
-    return rcpp_result_gen;
-END_RCPP
-}
-// RecordBatch__names
-Rcpp::CharacterVector RecordBatch__names(const std::shared_ptr<arrow::RecordBatch>& batch);
-RcppExport SEXP _arrow_RecordBatch__names(SEXP batchSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::RecordBatch>& >::type batch(batchSEXP);
-    rcpp_result_gen = Rcpp::wrap(RecordBatch__names(batch));
-    return rcpp_result_gen;
-END_RCPP
-}
-// RecordBatch__Slice1
-std::shared_ptr<arrow::RecordBatch> RecordBatch__Slice1(const std::shared_ptr<arrow::RecordBatch>& self, int offset);
-RcppExport SEXP _arrow_RecordBatch__Slice1(SEXP selfSEXP, SEXP offsetSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::RecordBatch>& >::type self(selfSEXP);
-    Rcpp::traits::input_parameter< int >::type offset(offsetSEXP);
-    rcpp_result_gen = Rcpp::wrap(RecordBatch__Slice1(self, offset));
-    return rcpp_result_gen;
-END_RCPP
-}
-// RecordBatch__Slice2
-std::shared_ptr<arrow::RecordBatch> RecordBatch__Slice2(const std::shared_ptr<arrow::RecordBatch>& self, int offset, int length);
-RcppExport SEXP _arrow_RecordBatch__Slice2(SEXP selfSEXP, SEXP offsetSEXP, SEXP lengthSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::RecordBatch>& >::type self(selfSEXP);
-    Rcpp::traits::input_parameter< int >::type offset(offsetSEXP);
-    Rcpp::traits::input_parameter< int >::type length(lengthSEXP);
-    rcpp_result_gen = Rcpp::wrap(RecordBatch__Slice2(self, offset, length));
-    return rcpp_result_gen;
-END_RCPP
-}
-// ipc___SerializeRecordBatch__Raw
-Rcpp::RawVector ipc___SerializeRecordBatch__Raw(const std::shared_ptr<arrow::RecordBatch>& batch);
-RcppExport SEXP _arrow_ipc___SerializeRecordBatch__Raw(SEXP batchSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::RecordBatch>& >::type batch(batchSEXP);
-    rcpp_result_gen = Rcpp::wrap(ipc___SerializeRecordBatch__Raw(batch));
-    return rcpp_result_gen;
-END_RCPP
-}
-// ipc___ReadRecordBatch__InputStream__Schema
-std::shared_ptr<arrow::RecordBatch> ipc___ReadRecordBatch__InputStream__Schema(const std::shared_ptr<arrow::io::InputStream>& stream, const std::shared_ptr<arrow::Schema>& schema);
-RcppExport SEXP _arrow_ipc___ReadRecordBatch__InputStream__Schema(SEXP streamSEXP, SEXP schemaSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::io::InputStream>& >::type stream(streamSEXP);
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::Schema>& >::type schema(schemaSEXP);
-    rcpp_result_gen = Rcpp::wrap(ipc___ReadRecordBatch__InputStream__Schema(stream, schema));
-    return rcpp_result_gen;
-END_RCPP
-}
-// RecordBatch__from_arrays
-std::shared_ptr<arrow::RecordBatch> RecordBatch__from_arrays(SEXP schema_sxp, SEXP lst);
-RcppExport SEXP _arrow_RecordBatch__from_arrays(SEXP schema_sxpSEXP, SEXP lstSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< SEXP >::type schema_sxp(schema_sxpSEXP);
-    Rcpp::traits::input_parameter< SEXP >::type lst(lstSEXP);
-    rcpp_result_gen = Rcpp::wrap(RecordBatch__from_arrays(schema_sxp, lst));
-    return rcpp_result_gen;
-END_RCPP
-}
-// RecordBatchReader__schema
-std::shared_ptr<arrow::Schema> RecordBatchReader__schema(const std::shared_ptr<arrow::RecordBatchReader>& reader);
-RcppExport SEXP _arrow_RecordBatchReader__schema(SEXP readerSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::RecordBatchReader>& >::type reader(readerSEXP);
-    rcpp_result_gen = Rcpp::wrap(RecordBatchReader__schema(reader));
-    return rcpp_result_gen;
-END_RCPP
-}
-// RecordBatchReader__ReadNext
-std::shared_ptr<arrow::RecordBatch> RecordBatchReader__ReadNext(const std::shared_ptr<arrow::RecordBatchReader>& reader);
-RcppExport SEXP _arrow_RecordBatchReader__ReadNext(SEXP readerSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::RecordBatchReader>& >::type reader(readerSEXP);
-    rcpp_result_gen = Rcpp::wrap(RecordBatchReader__ReadNext(reader));
-    return rcpp_result_gen;
-END_RCPP
-}
-// ipc___RecordBatchStreamReader__Open
-std::shared_ptr<arrow::RecordBatchReader> ipc___RecordBatchStreamReader__Open(const std::shared_ptr<arrow::io::InputStream>& stream);
-RcppExport SEXP _arrow_ipc___RecordBatchStreamReader__Open(SEXP streamSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::io::InputStream>& >::type stream(streamSEXP);
-    rcpp_result_gen = Rcpp::wrap(ipc___RecordBatchStreamReader__Open(stream));
-    return rcpp_result_gen;
-END_RCPP
-}
-// ipc___RecordBatchStreamReader__batches
-std::vector<std::shared_ptr<arrow::RecordBatch>> ipc___RecordBatchStreamReader__batches(const std::shared_ptr<arrow::ipc::RecordBatchStreamReader>& reader);
-RcppExport SEXP _arrow_ipc___RecordBatchStreamReader__batches(SEXP readerSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::ipc::RecordBatchStreamReader>& >::type reader(readerSEXP);
-    rcpp_result_gen = Rcpp::wrap(ipc___RecordBatchStreamReader__batches(reader));
-    return rcpp_result_gen;
-END_RCPP
-}
-// ipc___RecordBatchFileReader__schema
-std::shared_ptr<arrow::Schema> ipc___RecordBatchFileReader__schema(const std::shared_ptr<arrow::ipc::RecordBatchFileReader>& reader);
-RcppExport SEXP _arrow_ipc___RecordBatchFileReader__schema(SEXP readerSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::ipc::RecordBatchFileReader>& >::type reader(readerSEXP);
-    rcpp_result_gen = Rcpp::wrap(ipc___RecordBatchFileReader__schema(reader));
-    return rcpp_result_gen;
-END_RCPP
-}
-// ipc___RecordBatchFileReader__num_record_batches
-int ipc___RecordBatchFileReader__num_record_batches(const std::shared_ptr<arrow::ipc::RecordBatchFileReader>& reader);
-RcppExport SEXP _arrow_ipc___RecordBatchFileReader__num_record_batches(SEXP readerSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::ipc::RecordBatchFileReader>& >::type reader(readerSEXP);
-    rcpp_result_gen = Rcpp::wrap(ipc___RecordBatchFileReader__num_record_batches(reader));
-    return rcpp_result_gen;
-END_RCPP
-}
-// ipc___RecordBatchFileReader__ReadRecordBatch
-std::shared_ptr<arrow::RecordBatch> ipc___RecordBatchFileReader__ReadRecordBatch(const std::shared_ptr<arrow::ipc::RecordBatchFileReader>& reader, int i);
-RcppExport SEXP _arrow_ipc___RecordBatchFileReader__ReadRecordBatch(SEXP readerSEXP, SEXP iSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::ipc::RecordBatchFileReader>& >::type reader(readerSEXP);
-    Rcpp::traits::input_parameter< int >::type i(iSEXP);
-    rcpp_result_gen = Rcpp::wrap(ipc___RecordBatchFileReader__ReadRecordBatch(reader, i));
-    return rcpp_result_gen;
-END_RCPP
-}
-// ipc___RecordBatchFileReader__Open
-std::shared_ptr<arrow::ipc::RecordBatchFileReader> ipc___RecordBatchFileReader__Open(const std::shared_ptr<arrow::io::RandomAccessFile>& file);
-RcppExport SEXP _arrow_ipc___RecordBatchFileReader__Open(SEXP fileSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::io::RandomAccessFile>& >::type file(fileSEXP);
-    rcpp_result_gen = Rcpp::wrap(ipc___RecordBatchFileReader__Open(file));
-    return rcpp_result_gen;
-END_RCPP
-}
-// Table__from_RecordBatchFileReader
-std::shared_ptr<arrow::Table> Table__from_RecordBatchFileReader(const std::shared_ptr<arrow::ipc::RecordBatchFileReader>& reader);
-RcppExport SEXP _arrow_Table__from_RecordBatchFileReader(SEXP readerSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::ipc::RecordBatchFileReader>& >::type reader(readerSEXP);
-    rcpp_result_gen = Rcpp::wrap(Table__from_RecordBatchFileReader(reader));
-    return rcpp_result_gen;
-END_RCPP
-}
-// Table__from_RecordBatchStreamReader
-std::shared_ptr<arrow::Table> Table__from_RecordBatchStreamReader(const std::shared_ptr<arrow::ipc::RecordBatchStreamReader>& reader);
-RcppExport SEXP _arrow_Table__from_RecordBatchStreamReader(SEXP readerSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::ipc::RecordBatchStreamReader>& >::type reader(readerSEXP);
-    rcpp_result_gen = Rcpp::wrap(Table__from_RecordBatchStreamReader(reader));
-    return rcpp_result_gen;
-END_RCPP
-}
-// ipc___RecordBatchFileReader__batches
-std::vector<std::shared_ptr<arrow::RecordBatch>> ipc___RecordBatchFileReader__batches(const std::shared_ptr<arrow::ipc::RecordBatchFileReader>& reader);
-RcppExport SEXP _arrow_ipc___RecordBatchFileReader__batches(SEXP readerSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::ipc::RecordBatchFileReader>& >::type reader(readerSEXP);
-    rcpp_result_gen = Rcpp::wrap(ipc___RecordBatchFileReader__batches(reader));
-    return rcpp_result_gen;
-END_RCPP
-}
-// ipc___RecordBatchWriter__WriteRecordBatch
-void ipc___RecordBatchWriter__WriteRecordBatch(const std::shared_ptr<arrow::ipc::RecordBatchWriter>& batch_writer, const std::shared_ptr<arrow::RecordBatch>& batch);
-RcppExport SEXP _arrow_ipc___RecordBatchWriter__WriteRecordBatch(SEXP batch_writerSEXP, SEXP batchSEXP) {
-BEGIN_RCPP
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::ipc::RecordBatchWriter>& >::type batch_writer(batch_writerSEXP);
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::RecordBatch>& >::type batch(batchSEXP);
-    ipc___RecordBatchWriter__WriteRecordBatch(batch_writer, batch);
-    return R_NilValue;
-END_RCPP
-}
-// ipc___RecordBatchWriter__WriteTable
-void ipc___RecordBatchWriter__WriteTable(const std::shared_ptr<arrow::ipc::RecordBatchWriter>& batch_writer, const std::shared_ptr<arrow::Table>& table);
-RcppExport SEXP _arrow_ipc___RecordBatchWriter__WriteTable(SEXP batch_writerSEXP, SEXP tableSEXP) {
-BEGIN_RCPP
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::ipc::RecordBatchWriter>& >::type batch_writer(batch_writerSEXP);
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::Table>& >::type table(tableSEXP);
-    ipc___RecordBatchWriter__WriteTable(batch_writer, table);
-    return R_NilValue;
-END_RCPP
-}
-// ipc___RecordBatchWriter__Close
-void ipc___RecordBatchWriter__Close(const std::shared_ptr<arrow::ipc::RecordBatchWriter>& batch_writer);
-RcppExport SEXP _arrow_ipc___RecordBatchWriter__Close(SEXP batch_writerSEXP) {
-BEGIN_RCPP
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::ipc::RecordBatchWriter>& >::type batch_writer(batch_writerSEXP);
-    ipc___RecordBatchWriter__Close(batch_writer);
-    return R_NilValue;
-END_RCPP
-}
-// ipc___RecordBatchFileWriter__Open
-std::shared_ptr<arrow::ipc::RecordBatchWriter> ipc___RecordBatchFileWriter__Open(const std::shared_ptr<arrow::io::OutputStream>& stream, const std::shared_ptr<arrow::Schema>& schema);
-RcppExport SEXP _arrow_ipc___RecordBatchFileWriter__Open(SEXP streamSEXP, SEXP schemaSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::io::OutputStream>& >::type stream(streamSEXP);
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::Schema>& >::type schema(schemaSEXP);
-    rcpp_result_gen = Rcpp::wrap(ipc___RecordBatchFileWriter__Open(stream, schema));
-    return rcpp_result_gen;
-END_RCPP
-}
-// ipc___RecordBatchStreamWriter__Open
-std::shared_ptr<arrow::ipc::RecordBatchWriter> ipc___RecordBatchStreamWriter__Open(const std::shared_ptr<arrow::io::OutputStream>& stream, const std::shared_ptr<arrow::Schema>& schema);
-RcppExport SEXP _arrow_ipc___RecordBatchStreamWriter__Open(SEXP streamSEXP, SEXP schemaSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::io::OutputStream>& >::type stream(streamSEXP);
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::Schema>& >::type schema(schemaSEXP);
-    rcpp_result_gen = Rcpp::wrap(ipc___RecordBatchStreamWriter__Open(stream, schema));
-    return rcpp_result_gen;
-END_RCPP
-}
-// Table__num_columns
-int Table__num_columns(const std::shared_ptr<arrow::Table>& x);
-RcppExport SEXP _arrow_Table__num_columns(SEXP xSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::Table>& >::type x(xSEXP);
-    rcpp_result_gen = Rcpp::wrap(Table__num_columns(x));
-    return rcpp_result_gen;
-END_RCPP
-}
-// Table__num_rows
-int Table__num_rows(const std::shared_ptr<arrow::Table>& x);
-RcppExport SEXP _arrow_Table__num_rows(SEXP xSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::Table>& >::type x(xSEXP);
-    rcpp_result_gen = Rcpp::wrap(Table__num_rows(x));
-    return rcpp_result_gen;
-END_RCPP
-}
-// Table__schema
-std::shared_ptr<arrow::Schema> Table__schema(const std::shared_ptr<arrow::Table>& x);
-RcppExport SEXP _arrow_Table__schema(SEXP xSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::Table>& >::type x(xSEXP);
-    rcpp_result_gen = Rcpp::wrap(Table__schema(x));
-    return rcpp_result_gen;
-END_RCPP
-}
-// Table__column
-std::shared_ptr<arrow::Column> Table__column(const std::shared_ptr<arrow::Table>& table, int i);
-RcppExport SEXP _arrow_Table__column(SEXP tableSEXP, SEXP iSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::Table>& >::type table(tableSEXP);
-    Rcpp::traits::input_parameter< int >::type i(iSEXP);
-    rcpp_result_gen = Rcpp::wrap(Table__column(table, i));
-    return rcpp_result_gen;
-END_RCPP
-}
-// Table__columns
-std::vector<std::shared_ptr<arrow::Column>> Table__columns(const std::shared_ptr<arrow::Table>& table);
-RcppExport SEXP _arrow_Table__columns(SEXP tableSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< const std::shared_ptr<arrow::Table>& >::type table(tableSEXP);
-    rcpp_result_gen = Rcpp::wrap(Table__columns(table));
-    return rcpp_result_gen;
-END_RCPP
-}
-// Table__from_dots
-std::shared_ptr<arrow::Table> Table__from_dots(SEXP lst, SEXP schema_sxp);
-RcppExport SEXP _arrow_Table__from_dots(SEXP lstSEXP, SEXP schema_sxpSEXP) {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< SEXP >::type lst(lstSEXP);
-    Rcpp::traits::input_parameter< SEXP >::type schema_sxp(schema_sxpSEXP);
-    rcpp_result_gen = Rcpp::wrap(Table__from_dots(lst, schema_sxp));
-    return rcpp_result_gen;
-END_RCPP
-}
-// GetCpuThreadPoolCapacity
-int GetCpuThreadPoolCapacity();
-RcppExport SEXP _arrow_GetCpuThreadPoolCapacity() {
-BEGIN_RCPP
-    Rcpp::RObject rcpp_result_gen;
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    rcpp_result_gen = Rcpp::wrap(GetCpuThreadPoolCapacity());
-    return rcpp_result_gen;
-END_RCPP
-}
-// SetCpuThreadPoolCapacity
-void SetCpuThreadPoolCapacity(int threads);
-RcppExport SEXP _arrow_SetCpuThreadPoolCapacity(SEXP threadsSEXP) {
-BEGIN_RCPP
-    Rcpp::RNGScope rcpp_rngScope_gen;
-    Rcpp::traits::input_parameter< int >::type threads(threadsSEXP);
-    SetCpuThreadPoolCapacity(threads);
-    return R_NilValue;
-END_RCPP
-}
-
-static const R_CallMethodDef CallEntries[] = {
-    {"_arrow_Array__Slice1", (DL_FUNC) &_arrow_Array__Slice1, 2},
-    {"_arrow_Array__Slice2", (DL_FUNC) &_arrow_Array__Slice2, 3},
-    {"_arrow_Array__IsNull", (DL_FUNC) &_arrow_Array__IsNull, 2},
-    {"_arrow_Array__IsValid", (DL_FUNC) &_arrow_Array__IsValid, 2},
-    {"_arrow_Array__length", (DL_FUNC) &_arrow_Array__length, 1},
-    {"_arrow_Array__offset", (DL_FUNC) &_arrow_Array__offset, 1},
-    {"_arrow_Array__null_count", (DL_FUNC) &_arrow_Array__null_count, 1},
-    {"_arrow_Array__type", (DL_FUNC) &_arrow_Array__type, 1},
-    {"_arrow_Array__ToString", (DL_FUNC) &_arrow_Array__ToString, 1},
-    {"_arrow_Array__type_id", (DL_FUNC) &_arrow_Array__type_id, 1},
-    {"_arrow_Array__Equals", (DL_FUNC) &_arrow_Array__Equals, 2},
-    {"_arrow_Array__ApproxEquals", (DL_FUNC) &_arrow_Array__ApproxEquals, 2},
-    {"_arrow_Array__data", (DL_FUNC) &_arrow_Array__data, 1},
-    {"_arrow_Array__RangeEquals", (DL_FUNC) &_arrow_Array__RangeEquals, 5},
-    {"_arrow_Array__Mask", (DL_FUNC) &_arrow_Array__Mask, 1},
-    {"_arrow_DictionaryArray__indices", (DL_FUNC) &_arrow_DictionaryArray__indices, 1},
-    {"_arrow_DictionaryArray__dictionary", (DL_FUNC) &_arrow_DictionaryArray__dictionary, 1},
-    {"_arrow_Array__as_vector", (DL_FUNC) &_arrow_Array__as_vector, 1},
-    {"_arrow_ChunkedArray__as_vector", (DL_FUNC) &_arrow_ChunkedArray__as_vector, 1},
-    {"_arrow_RecordBatch__to_dataframe", (DL_FUNC) &_arrow_RecordBatch__to_dataframe, 2},
-    {"_arrow_Table__to_dataframe", (DL_FUNC) &_arrow_Table__to_dataframe, 2},
-    {"_arrow_Array__infer_type", (DL_FUNC) &_arrow_Array__infer_type, 1},
-    {"_arrow_Array__from_vector", (DL_FUNC) &_arrow_Array__from_vector, 2},
-    {"_arrow_ChunkedArray__from_list", (DL_FUNC) &_arrow_ChunkedArray__from_list, 2},
-    {"_arrow_ArrayData__get_type", (DL_FUNC) &_arrow_ArrayData__get_type, 1},
-    {"_arrow_ArrayData__get_length", (DL_FUNC) &_arrow_ArrayData__get_length, 1},
-    {"_arrow_ArrayData__get_null_count", (DL_FUNC) &_arrow_ArrayData__get_null_count, 1},
-    {"_arrow_ArrayData__get_offset", (DL_FUNC) &_arrow_ArrayData__get_offset, 1},
-    {"_arrow_ArrayData__buffers", (DL_FUNC) &_arrow_ArrayData__buffers, 1},
-    {"_arrow_Buffer__is_mutable", (DL_FUNC) &_arrow_Buffer__is_mutable, 1},
-    {"_arrow_Buffer__ZeroPadding", (DL_FUNC) &_arrow_Buffer__ZeroPadding, 1},
-    {"_arrow_Buffer__capacity", (DL_FUNC) &_arrow_Buffer__capacity, 1},
-    {"_arrow_Buffer__size", (DL_FUNC) &_arrow_Buffer__size, 1},
-    {"_arrow_r___RBuffer__initialize", (DL_FUNC) &_arrow_r___RBuffer__initialize, 1},
-    {"_arrow_ChunkedArray__length", (DL_FUNC) &_arrow_ChunkedArray__length, 1},
-    {"_arrow_ChunkedArray__null_count", (DL_FUNC) &_arrow_ChunkedArray__null_count, 1},
-    {"_arrow_ChunkedArray__num_chunks", (DL_FUNC) &_arrow_ChunkedArray__num_chunks, 1},
-    {"_arrow_ChunkedArray__chunk", (DL_FUNC) &_arrow_ChunkedArray__chunk, 2},
-    {"_arrow_ChunkedArray__chunks", (DL_FUNC) &_arrow_ChunkedArray__chunks, 1},
-    {"_arrow_ChunkedArray__type", (DL_FUNC) &_arrow_ChunkedArray__type, 1},
-    {"_arrow_ChunkArray__Slice1", (DL_FUNC) &_arrow_ChunkArray__Slice1, 2},
-    {"_arrow_ChunkArray__Slice2", (DL_FUNC) &_arrow_ChunkArray__Slice2, 3},
-    {"_arrow_Column__length", (DL_FUNC) &_arrow_Column__length, 1},
-    {"_arrow_Column__null_count", (DL_FUNC) &_arrow_Column__null_count, 1},
-    {"_arrow_Column__type", (DL_FUNC) &_arrow_Column__type, 1},
-    {"_arrow_Column__data", (DL_FUNC) &_arrow_Column__data, 1},
-    {"_arrow_util___Codec__Create", (DL_FUNC) &_arrow_util___Codec__Create, 1},
-    {"_arrow_io___CompressedOutputStream__Make", (DL_FUNC) &_arrow_io___CompressedOutputStream__Make, 2},
-    {"_arrow_io___CompressedInputStream__Make", (DL_FUNC) &_arrow_io___CompressedInputStream__Make, 2},
-    {"_arrow_compute___CastOptions__initialize", (DL_FUNC) &_arrow_compute___CastOptions__initialize, 3},
-    {"_arrow_Array__cast", (DL_FUNC) &_arrow_Array__cast, 3},
-    {"_arrow_ChunkedArray__cast", (DL_FUNC) &_arrow_ChunkedArray__cast, 3},
-    {"_arrow_RecordBatch__cast", (DL_FUNC) &_arrow_RecordBatch__cast, 3},
-    {"_arrow_Table__cast", (DL_FUNC) &_arrow_Table__cast, 3},
-    {"_arrow_csv___ReadOptions__initialize", (DL_FUNC) &_arrow_csv___ReadOptions__initialize, 1},
-    {"_arrow_csv___ParseOptions__initialize", (DL_FUNC) &_arrow_csv___ParseOptions__initialize, 1},
-    {"_arrow_csv___ConvertOptions__initialize", (DL_FUNC) &_arrow_csv___ConvertOptions__initialize, 1},
-    {"_arrow_csv___TableReader__Make", (DL_FUNC) &_arrow_csv___TableReader__Make, 4},
-    {"_arrow_csv___TableReader__Read", (DL_FUNC) &_arrow_csv___TableReader__Read, 1},
-    {"_arrow_shared_ptr_is_null", (DL_FUNC) &_arrow_shared_ptr_is_null, 1},
-    {"_arrow_unique_ptr_is_null", (DL_FUNC) &_arrow_unique_ptr_is_null, 1},
-    {"_arrow_Int8__initialize", (DL_FUNC) &_arrow_Int8__initialize, 0},
-    {"_arrow_Int16__initialize", (DL_FUNC) &_arrow_Int16__initialize, 0},
-    {"_arrow_Int32__initialize", (DL_FUNC) &_arrow_Int32__initialize, 0},
-    {"_arrow_Int64__initialize", (DL_FUNC) &_arrow_Int64__initialize, 0},
-    {"_arrow_UInt8__initialize", (DL_FUNC) &_arrow_UInt8__initialize, 0},
-    {"_arrow_UInt16__initialize", (DL_FUNC) &_arrow_UInt16__initialize, 0},
-    {"_arrow_UInt32__initialize", (DL_FUNC) &_arrow_UInt32__initialize, 0},
-    {"_arrow_UInt64__initialize", (DL_FUNC) &_arrow_UInt64__initialize, 0},
-    {"_arrow_Float16__initialize", (DL_FUNC) &_arrow_Float16__initialize, 0},
-    {"_arrow_Float32__initialize", (DL_FUNC) &_arrow_Float32__initialize, 0},
-    {"_arrow_Float64__initialize", (DL_FUNC) &_arrow_Float64__initialize, 0},
-    {"_arrow_Boolean__initialize", (DL_FUNC) &_arrow_Boolean__initialize, 0},
-    {"_arrow_Utf8__initialize", (DL_FUNC) &_arrow_Utf8__initialize, 0},
-    {"_arrow_Date32__initialize", (DL_FUNC) &_arrow_Date32__initialize, 0},
-    {"_arrow_Date64__initialize", (DL_FUNC) &_arrow_Date64__initialize, 0},
-    {"_arrow_Null__initialize", (DL_FUNC) &_arrow_Null__initialize, 0},
-    {"_arrow_Decimal128Type__initialize", (DL_FUNC) &_arrow_Decimal128Type__initialize, 2},
-    {"_arrow_FixedSizeBinary__initialize", (DL_FUNC) &_arrow_FixedSizeBinary__initialize, 1},
-    {"_arrow_Timestamp__initialize1", (DL_FUNC) &_arrow_Timestamp__initialize1, 1},
-    {"_arrow_Timestamp__initialize2", (DL_FUNC) &_arrow_Timestamp__initialize2, 2},
-    {"_arrow_Time32__initialize", (DL_FUNC) &_arrow_Time32__initialize, 1},
-    {"_arrow_Time64__initialize", (DL_FUNC) &_arrow_Time64__initialize, 1},
-    {"_arrow_list__", (DL_FUNC) &_arrow_list__, 1},
-    {"_arrow_struct_", (DL_FUNC) &_arrow_struct_, 1},
-    {"_arrow_DataType__ToString", (DL_FUNC) &_arrow_DataType__ToString, 1},
-    {"_arrow_DataType__name", (DL_FUNC) &_arrow_DataType__name, 1},
-    {"_arrow_DataType__Equals", (DL_FUNC) &_arrow_DataType__Equals, 2},
-    {"_arrow_DataType__num_children", (DL_FUNC) &_arrow_DataType__num_children, 1},
-    {"_arrow_DataType__children_pointer", (DL_FUNC) &_arrow_DataType__children_pointer, 1},
-    {"_arrow_DataType__id", (DL_FUNC) &_arrow_DataType__id, 1},
-    {"_arrow_schema_", (DL_FUNC) &_arrow_schema_, 1},
-    {"_arrow_Schema__ToString", (DL_FUNC) &_arrow_Schema__ToString, 1},
-    {"_arrow_Schema__num_fields", (DL_FUNC) &_arrow_Schema__num_fields, 1},
-    {"_arrow_Schema__field", (DL_FUNC) &_arrow_Schema__field, 2},
-    {"_arrow_Schema__names", (DL_FUNC) &_arrow_Schema__names, 1},
-    {"_arrow_ListType__ToString", (DL_FUNC) &_arrow_ListType__ToString, 1},
-    {"_arrow_FixedWidthType__bit_width", (DL_FUNC) &_arrow_FixedWidthType__bit_width, 1},
-    {"_arrow_DateType__unit", (DL_FUNC) &_arrow_DateType__unit, 1},
-    {"_arrow_TimeType__unit", (DL_FUNC) &_arrow_TimeType__unit, 1},
-    {"_arrow_DecimalType__precision", (DL_FUNC) &_arrow_DecimalType__precision, 1},
-    {"_arrow_DecimalType__scale", (DL_FUNC) &_arrow_DecimalType__scale, 1},
-    {"_arrow_TimestampType__timezone", (DL_FUNC) &_arrow_TimestampType__timezone, 1},
-    {"_arrow_TimestampType__unit", (DL_FUNC) &_arrow_TimestampType__unit, 1},
-    {"_arrow_DictionaryType__initialize", (DL_FUNC) &_arrow_DictionaryType__initialize, 3},
-    {"_arrow_DictionaryType__index_type", (DL_FUNC) &_arrow_DictionaryType__index_type, 1},
-    {"_arrow_DictionaryType__value_type", (DL_FUNC) &_arrow_DictionaryType__value_type, 1},
-    {"_arrow_DictionaryType__name", (DL_FUNC) &_arrow_DictionaryType__name, 1},
-    {"_arrow_DictionaryType__ordered", (DL_FUNC) &_arrow_DictionaryType__ordered, 1},
-    {"_arrow_ipc___feather___TableWriter__SetDescription", (DL_FUNC) &_arrow_ipc___feather___TableWriter__SetDescription, 2},
-    {"_arrow_ipc___feather___TableWriter__SetNumRows", (DL_FUNC) &_arrow_ipc___feather___TableWriter__SetNumRows, 2},
-    {"_arrow_ipc___feather___TableWriter__Append", (DL_FUNC) &_arrow_ipc___feather___TableWriter__Append, 3},
-    {"_arrow_ipc___feather___TableWriter__Finalize", (DL_FUNC) &_arrow_ipc___feather___TableWriter__Finalize, 1},
-    {"_arrow_ipc___feather___TableWriter__Open", (DL_FUNC) &_arrow_ipc___feather___TableWriter__Open, 1},
-    {"_arrow_ipc___TableWriter__RecordBatch__WriteFeather", (DL_FUNC) &_arrow_ipc___TableWriter__RecordBatch__WriteFeather, 2},
-    {"_arrow_ipc___feather___TableReader__GetDescription", (DL_FUNC) &_arrow_ipc___feather___TableReader__GetDescription, 1},
-    {"_arrow_ipc___feather___TableReader__HasDescription", (DL_FUNC) &_arrow_ipc___feather___TableReader__HasDescription, 1},
-    {"_arrow_ipc___feather___TableReader__version", (DL_FUNC) &_arrow_ipc___feather___TableReader__version, 1},
-    {"_arrow_ipc___feather___TableReader__num_rows", (DL_FUNC) &_arrow_ipc___feather___TableReader__num_rows, 1},
-    {"_arrow_ipc___feather___TableReader__num_columns", (DL_FUNC) &_arrow_ipc___feather___TableReader__num_columns, 1},
-    {"_arrow_ipc___feather___TableReader__GetColumnName", (DL_FUNC) &_arrow_ipc___feather___TableReader__GetColumnName, 2},
-    {"_arrow_ipc___feather___TableReader__GetColumn", (DL_FUNC) &_arrow_ipc___feather___TableReader__GetColumn, 2},
-    {"_arrow_ipc___feather___TableReader__Read", (DL_FUNC) &_arrow_ipc___feather___TableReader__Read, 2},
-    {"_arrow_ipc___feather___TableReader__Open", (DL_FUNC) &_arrow_ipc___feather___TableReader__Open, 1},
-    {"_arrow_Field__initialize", (DL_FUNC) &_arrow_Field__initialize, 3},
-    {"_arrow_Field__ToString", (DL_FUNC) &_arrow_Field__ToString, 1},
-    {"_arrow_Field__name", (DL_FUNC) &_arrow_Field__name, 1},
-    {"_arrow_Field__Equals", (DL_FUNC) &_arrow_Field__Equals, 2},
-    {"_arrow_Field__nullable", (DL_FUNC) &_arrow_Field__nullable, 1},
-    {"_arrow_Field__type", (DL_FUNC) &_arrow_Field__type, 1},
-    {"_arrow_io___Readable__Read", (DL_FUNC) &_arrow_io___Readable__Read, 2},
-    {"_arrow_io___InputStream__Close", (DL_FUNC) &_arrow_io___InputStream__Close, 1},
-    {"_arrow_io___OutputStream__Close", (DL_FUNC) &_arrow_io___OutputStream__Close, 1},
-    {"_arrow_io___RandomAccessFile__GetSize", (DL_FUNC) &_arrow_io___RandomAccessFile__GetSize, 1},
-    {"_arrow_io___RandomAccessFile__supports_zero_copy", (DL_FUNC) &_arrow_io___RandomAccessFile__supports_zero_copy, 1},
-    {"_arrow_io___RandomAccessFile__Seek", (DL_FUNC) &_arrow_io___RandomAccessFile__Seek, 2},
-    {"_arrow_io___RandomAccessFile__Tell", (DL_FUNC) &_arrow_io___RandomAccessFile__Tell, 1},
-    {"_arrow_io___MemoryMappedFile__Create", (DL_FUNC) &_arrow_io___MemoryMappedFile__Create, 2},
-    {"_arrow_io___MemoryMappedFile__Open", (DL_FUNC) &_arrow_io___MemoryMappedFile__Open, 2},
-    {"_arrow_io___MemoryMappedFile__Resize", (DL_FUNC) &_arrow_io___MemoryMappedFile__Resize, 2},
-    {"_arrow_io___ReadableFile__Open", (DL_FUNC) &_arrow_io___ReadableFile__Open, 1},
-    {"_arrow_io___BufferReader__initialize", (DL_FUNC) &_arrow_io___BufferReader__initialize, 1},
-    {"_arrow_io___Writable__write", (DL_FUNC) &_arrow_io___Writable__write, 2},
-    {"_arrow_io___OutputStream__Tell", (DL_FUNC) &_arrow_io___OutputStream__Tell, 1},
-    {"_arrow_io___FileOutputStream__Open", (DL_FUNC) &_arrow_io___FileOutputStream__Open, 1},
-    {"_arrow_io___BufferOutputStream__Create", (DL_FUNC) &_arrow_io___BufferOutputStream__Create, 1},
-    {"_arrow_io___BufferOutputStream__capacity", (DL_FUNC) &_arrow_io___BufferOutputStream__capacity, 1},
-    {"_arrow_io___BufferOutputStream__Finish", (DL_FUNC) &_arrow_io___BufferOutputStream__Finish, 1},
-    {"_arrow_io___BufferOutputStream__Tell", (DL_FUNC) &_arrow_io___BufferOutputStream__Tell, 1},
-    {"_arrow_io___BufferOutputStream__Write", (DL_FUNC) &_arrow_io___BufferOutputStream__Write, 2},
-    {"_arrow_io___MockOutputStream__initialize", (DL_FUNC) &_arrow_io___MockOutputStream__initialize, 0},
-    {"_arrow_io___MockOutputStream__GetExtentBytesWritten", (DL_FUNC) &_arrow_io___MockOutputStream__GetExtentBytesWritten, 1},
-    {"_arrow_io___FixedSizeBufferWriter__initialize", (DL_FUNC) &_arrow_io___FixedSizeBufferWriter__initialize, 1},
-    {"_arrow_MemoryPool__default", (DL_FUNC) &_arrow_MemoryPool__default, 0},
-    {"_arrow_MemoryPool__bytes_allocated", (DL_FUNC) &_arrow_MemoryPool__bytes_allocated, 1},
-    {"_arrow_MemoryPool__max_memory", (DL_FUNC) &_arrow_MemoryPool__max_memory, 1},
-    {"_arrow_ipc___Message__body_length", (DL_FUNC) &_arrow_ipc___Message__body_length, 1},
-    {"_arrow_ipc___Message__metadata", (DL_FUNC) &_arrow_ipc___Message__metadata, 1},
-    {"_arrow_ipc___Message__body", (DL_FUNC) &_arrow_ipc___Message__body, 1},
-    {"_arrow_ipc___Message__Verify", (DL_FUNC) &_arrow_ipc___Message__Verify, 1},
-    {"_arrow_ipc___Message__type", (DL_FUNC) &_arrow_ipc___Message__type, 1},
-    {"_arrow_ipc___Message__Equals", (DL_FUNC) &_arrow_ipc___Message__Equals, 2},
-    {"_arrow_ipc___ReadRecordBatch__Message__Schema", (DL_FUNC) &_arrow_ipc___ReadRecordBatch__Message__Schema, 2},
-    {"_arrow_ipc___ReadSchema_InputStream", (DL_FUNC) &_arrow_ipc___ReadSchema_InputStream, 1},
-    {"_arrow_ipc___MessageReader__Open", (DL_FUNC) &_arrow_ipc___MessageReader__Open, 1},
-    {"_arrow_ipc___MessageReader__ReadNextMessage", (DL_FUNC) &_arrow_ipc___MessageReader__ReadNextMessage, 1},
-    {"_arrow_ipc___ReadMessage", (DL_FUNC) &_arrow_ipc___ReadMessage, 1},
-    {"_arrow_read_parquet_file", (DL_FUNC) &_arrow_read_parquet_file, 1},
-    {"_arrow_RecordBatch__num_columns", (DL_FUNC) &_arrow_RecordBatch__num_columns, 1},
-    {"_arrow_RecordBatch__num_rows", (DL_FUNC) &_arrow_RecordBatch__num_rows, 1},
-    {"_arrow_RecordBatch__schema", (DL_FUNC) &_arrow_RecordBatch__schema, 1},
-    {"_arrow_RecordBatch__columns", (DL_FUNC) &_arrow_RecordBatch__columns, 1},
-    {"_arrow_RecordBatch__column", (DL_FUNC) &_arrow_RecordBatch__column, 2},
-    {"_arrow_RecordBatch__Equals", (DL_FUNC) &_arrow_RecordBatch__Equals, 2},
-    {"_arrow_RecordBatch__RemoveColumn", (DL_FUNC) &_arrow_RecordBatch__RemoveColumn, 2},
-    {"_arrow_RecordBatch__column_name", (DL_FUNC) &_arrow_RecordBatch__column_name, 2},
-    {"_arrow_RecordBatch__names", (DL_FUNC) &_arrow_RecordBatch__names, 1},
-    {"_arrow_RecordBatch__Slice1", (DL_FUNC) &_arrow_RecordBatch__Slice1, 2},
-    {"_arrow_RecordBatch__Slice2", (DL_FUNC) &_arrow_RecordBatch__Slice2, 3},
-    {"_arrow_ipc___SerializeRecordBatch__Raw", (DL_FUNC) &_arrow_ipc___SerializeRecordBatch__Raw, 1},
-    {"_arrow_ipc___ReadRecordBatch__InputStream__Schema", (DL_FUNC) &_arrow_ipc___ReadRecordBatch__InputStream__Schema, 2},
-    {"_arrow_RecordBatch__from_arrays", (DL_FUNC) &_arrow_RecordBatch__from_arrays, 2},
-    {"_arrow_RecordBatchReader__schema", (DL_FUNC) &_arrow_RecordBatchReader__schema, 1},
-    {"_arrow_RecordBatchReader__ReadNext", (DL_FUNC) &_arrow_RecordBatchReader__ReadNext, 1},
-    {"_arrow_ipc___RecordBatchStreamReader__Open", (DL_FUNC) &_arrow_ipc___RecordBatchStreamReader__Open, 1},
-    {"_arrow_ipc___RecordBatchStreamReader__batches", (DL_FUNC) &_arrow_ipc___RecordBatchStreamReader__batches, 1},
-    {"_arrow_ipc___RecordBatchFileReader__schema", (DL_FUNC) &_arrow_ipc___RecordBatchFileReader__schema, 1},
-    {"_arrow_ipc___RecordBatchFileReader__num_record_batches", (DL_FUNC) &_arrow_ipc___RecordBatchFileReader__num_record_batches, 1},
-    {"_arrow_ipc___RecordBatchFileReader__ReadRecordBatch", (DL_FUNC) &_arrow_ipc___RecordBatchFileReader__ReadRecordBatch, 2},
-    {"_arrow_ipc___RecordBatchFileReader__Open", (DL_FUNC) &_arrow_ipc___RecordBatchFileReader__Open, 1},
-    {"_arrow_Table__from_RecordBatchFileReader", (DL_FUNC) &_arrow_Table__from_RecordBatchFileReader, 1},
-    {"_arrow_Table__from_RecordBatchStreamReader", (DL_FUNC) &_arrow_Table__from_RecordBatchStreamReader, 1},
-    {"_arrow_ipc___RecordBatchFileReader__batches", (DL_FUNC) &_arrow_ipc___RecordBatchFileReader__batches, 1},
-    {"_arrow_ipc___RecordBatchWriter__WriteRecordBatch", (DL_FUNC) &_arrow_ipc___RecordBatchWriter__WriteRecordBatch, 2},
-    {"_arrow_ipc___RecordBatchWriter__WriteTable", (DL_FUNC) &_arrow_ipc___RecordBatchWriter__WriteTable, 2},
-    {"_arrow_ipc___RecordBatchWriter__Close", (DL_FUNC) &_arrow_ipc___RecordBatchWriter__Close, 1},
-    {"_arrow_ipc___RecordBatchFileWriter__Open", (DL_FUNC) &_arrow_ipc___RecordBatchFileWriter__Open, 2},
-    {"_arrow_ipc___RecordBatchStreamWriter__Open", (DL_FUNC) &_arrow_ipc___RecordBatchStreamWriter__Open, 2},
-    {"_arrow_Table__num_columns", (DL_FUNC) &_arrow_Table__num_columns, 1},
-    {"_arrow_Table__num_rows", (DL_FUNC) &_arrow_Table__num_rows, 1},
-    {"_arrow_Table__schema", (DL_FUNC) &_arrow_Table__schema, 1},
-    {"_arrow_Table__column", (DL_FUNC) &_arrow_Table__column, 2},
-    {"_arrow_Table__columns", (DL_FUNC) &_arrow_Table__columns, 1},
-    {"_arrow_Table__from_dots", (DL_FUNC) &_arrow_Table__from_dots, 2},
-    {"_arrow_GetCpuThreadPoolCapacity", (DL_FUNC) &_arrow_GetCpuThreadPoolCapacity, 0},
-    {"_arrow_SetCpuThreadPoolCapacity", (DL_FUNC) &_arrow_SetCpuThreadPoolCapacity, 1},
-    {NULL, NULL, 0}
-};
-
-RcppExport void R_init_arrow(DllInfo *dll) {
-    R_registerRoutines(dll, NULL, CallEntries, NULL, NULL);
-    R_useDynamicSymbols(dll, FALSE);
-}
diff --git a/r/src/array.cpp b/r/src/array.cpp
index e4ad68d..60fd7da 100644
--- a/r/src/array.cpp
+++ b/r/src/array.cpp
@@ -20,76 +20,78 @@
 using Rcpp::LogicalVector;
 using Rcpp::no_init;
 
-// [[Rcpp::export]]
+#if defined(ARROW_R_WITH_ARROW)
+
+// [[arrow::export]]
 std::shared_ptr<arrow::Array> Array__Slice1(const std::shared_ptr<arrow::Array>& array,
                                             int offset) {
   return array->Slice(offset);
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 std::shared_ptr<arrow::Array> Array__Slice2(const std::shared_ptr<arrow::Array>& array,
                                             int offset, int length) {
   return array->Slice(offset, length);
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 bool Array__IsNull(const std::shared_ptr<arrow::Array>& x, int i) { return x->IsNull(i); }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 bool Array__IsValid(const std::shared_ptr<arrow::Array>& x, int i) {
   return x->IsValid(i);
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 int Array__length(const std::shared_ptr<arrow::Array>& x) { return x->length(); }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 int Array__offset(const std::shared_ptr<arrow::Array>& x) { return x->offset(); }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 int Array__null_count(const std::shared_ptr<arrow::Array>& x) { return x->null_count(); }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 std::shared_ptr<arrow::DataType> Array__type(const std::shared_ptr<arrow::Array>& x) {
   return x->type();
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 std::string Array__ToString(const std::shared_ptr<arrow::Array>& x) {
   return x->ToString();
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 arrow::Type::type Array__type_id(const std::shared_ptr<arrow::Array>& x) {
   return x->type_id();
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 bool Array__Equals(const std::shared_ptr<arrow::Array>& lhs,
                    const std::shared_ptr<arrow::Array>& rhs) {
   return lhs->Equals(rhs);
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 bool Array__ApproxEquals(const std::shared_ptr<arrow::Array>& lhs,
                          const std::shared_ptr<arrow::Array>& rhs) {
   return lhs->ApproxEquals(rhs);
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 std::shared_ptr<arrow::ArrayData> Array__data(
     const std::shared_ptr<arrow::Array>& array) {
   return array->data();
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 bool Array__RangeEquals(const std::shared_ptr<arrow::Array>& self,
                         const std::shared_ptr<arrow::Array>& other, int start_idx,
                         int end_idx, int other_start_idx) {
   return self->RangeEquals(*other, start_idx, end_idx, other_start_idx);
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 LogicalVector Array__Mask(const std::shared_ptr<arrow::Array>& array) {
   if (array->null_count() == 0) {
     return LogicalVector(array->length(), true);
@@ -105,14 +107,16 @@ LogicalVector Array__Mask(const std::shared_ptr<arrow::Array>& array) {
   return res;
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 std::shared_ptr<arrow::Array> DictionaryArray__indices(
     const std::shared_ptr<arrow::DictionaryArray>& array) {
   return array->indices();
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 std::shared_ptr<arrow::Array> DictionaryArray__dictionary(
     const std::shared_ptr<arrow::DictionaryArray>& array) {
   return array->dictionary();
 }
+
+#endif
diff --git a/r/src/array__to_vector.cpp b/r/src/array__to_vector.cpp
index 219b19d..65670e8 100644
--- a/r/src/array__to_vector.cpp
+++ b/r/src/array__to_vector.cpp
@@ -15,9 +15,11 @@
 // specific language governing permissions and limitations
 // under the License.
 
+#include "./arrow_types.h"
+#if defined(ARROW_R_WITH_ARROW)
+
 #include <arrow/util/parallel.h>
 #include <arrow/util/task-group.h>
-#include "./arrow_types.h"
 
 namespace arrow {
 namespace r {
@@ -664,18 +666,18 @@ Rcpp::List to_dataframe_parallel(
 }  // namespace r
 }  // namespace arrow
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 SEXP Array__as_vector(const std::shared_ptr<arrow::Array>& array) {
   return arrow::r::ArrayVector__as_vector(array->length(), {array});
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 SEXP ChunkedArray__as_vector(const std::shared_ptr<arrow::ChunkedArray>& chunked_array) {
   return arrow::r::ArrayVector__as_vector(chunked_array->length(),
                                           chunked_array->chunks());
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 Rcpp::List RecordBatch__to_dataframe(const std::shared_ptr<arrow::RecordBatch>& batch,
                                      bool use_threads) {
   int64_t nc = batch->num_columns();
@@ -697,7 +699,7 @@ Rcpp::List RecordBatch__to_dataframe(const std::shared_ptr<arrow::RecordBatch>&
   }
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 Rcpp::List Table__to_dataframe(const std::shared_ptr<arrow::Table>& table,
                                bool use_threads) {
   int64_t nc = table->num_columns();
@@ -716,3 +718,5 @@ Rcpp::List Table__to_dataframe(const std::shared_ptr<arrow::Table>& table,
     return arrow::r::to_dataframe_serial(nr, nc, names, converters);
   }
 }
+
+#endif
diff --git a/r/src/array_from_vector.cpp b/r/src/array_from_vector.cpp
index e2b82ac..98e59ff 100644
--- a/r/src/array_from_vector.cpp
+++ b/r/src/array_from_vector.cpp
@@ -16,6 +16,7 @@
 // under the License.
 
 #include "./arrow_types.h"
+#if defined(ARROW_R_WITH_ARROW)
 
 namespace arrow {
 namespace r {
@@ -955,12 +956,12 @@ std::shared_ptr<arrow::Array> Array__from_vector(
 }  // namespace r
 }  // namespace arrow
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 std::shared_ptr<arrow::DataType> Array__infer_type(SEXP x) {
   return arrow::r::InferType(x);
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 std::shared_ptr<arrow::Array> Array__from_vector(SEXP x, SEXP s_type) {
   // the type might be NULL, in which case we need to infer it from the data
   // we keep track of whether it was infered or supplied
@@ -975,7 +976,7 @@ std::shared_ptr<arrow::Array> Array__from_vector(SEXP x, SEXP s_type) {
   return arrow::r::Array__from_vector(x, type, type_infered);
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 std::shared_ptr<arrow::ChunkedArray> ChunkedArray__from_list(Rcpp::List chunks,
                                                              SEXP s_type) {
   std::vector<std::shared_ptr<arrow::Array>> vec;
@@ -1016,3 +1017,5 @@ std::shared_ptr<arrow::ChunkedArray> ChunkedArray__from_list(Rcpp::List chunks,
 
   return std::make_shared<arrow::ChunkedArray>(std::move(vec));
 }
+
+#endif
diff --git a/r/src/arraydata.cpp b/r/src/arraydata.cpp
index 52367c5..f82d305 100644
--- a/r/src/arraydata.cpp
+++ b/r/src/arraydata.cpp
@@ -20,28 +20,32 @@
 using Rcpp::List;
 using Rcpp::wrap;
 
-// [[Rcpp::export]]
+#if defined(ARROW_R_WITH_ARROW)
+
+// [[arrow::export]]
 std::shared_ptr<arrow::DataType> ArrayData__get_type(
     const std::shared_ptr<arrow::ArrayData>& x) {
   return x->type;
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 int ArrayData__get_length(const std::shared_ptr<arrow::ArrayData>& x) {
   return x->length;
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 int ArrayData__get_null_count(const std::shared_ptr<arrow::ArrayData>& x) {
   return x->null_count;
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 int ArrayData__get_offset(const std::shared_ptr<arrow::ArrayData>& x) {
   return x->offset;
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 List ArrayData__buffers(const std::shared_ptr<arrow::ArrayData>& x) {
   return wrap(x->buffers);
 }
+
+#endif
diff --git a/r/src/arrowExports.cpp b/r/src/arrowExports.cpp
new file mode 100644
index 0000000..0ddac30
--- /dev/null
+++ b/r/src/arrowExports.cpp
@@ -0,0 +1,3428 @@
+// Generated by using data-raw/codegen.R -> do not edit by hand
+#include "./arrow_types.h"
+#include <Rcpp.h>
+
+using namespace Rcpp;
+
+// array.cpp:25
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::Array> Array__Slice1(const std::shared_ptr<arrow::Array>& array, int offset);
+RcppExport SEXP _arrow_Array__Slice1(SEXP array_sexp, SEXP offset_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::Array>&>::type array(array_sexp);
+	Rcpp::traits::input_parameter<int>::type offset(offset_sexp);
+	return Rcpp::wrap(Array__Slice1(array, offset));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_Array__Slice1(SEXP array_sexp, SEXP offset_sexp){
+	Rf_error("Cannot call Array__Slice1(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// array.cpp:31
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::Array> Array__Slice2(const std::shared_ptr<arrow::Array>& array, int offset, int length);
+RcppExport SEXP _arrow_Array__Slice2(SEXP array_sexp, SEXP offset_sexp, SEXP length_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::Array>&>::type array(array_sexp);
+	Rcpp::traits::input_parameter<int>::type offset(offset_sexp);
+	Rcpp::traits::input_parameter<int>::type length(length_sexp);
+	return Rcpp::wrap(Array__Slice2(array, offset, length));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_Array__Slice2(SEXP array_sexp, SEXP offset_sexp, SEXP length_sexp){
+	Rf_error("Cannot call Array__Slice2(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// array.cpp:37
+#if defined(ARROW_R_WITH_ARROW)
+bool Array__IsNull(const std::shared_ptr<arrow::Array>& x, int i);
+RcppExport SEXP _arrow_Array__IsNull(SEXP x_sexp, SEXP i_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::Array>&>::type x(x_sexp);
+	Rcpp::traits::input_parameter<int>::type i(i_sexp);
+	return Rcpp::wrap(Array__IsNull(x, i));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_Array__IsNull(SEXP x_sexp, SEXP i_sexp){
+	Rf_error("Cannot call Array__IsNull(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// array.cpp:40
+#if defined(ARROW_R_WITH_ARROW)
+bool Array__IsValid(const std::shared_ptr<arrow::Array>& x, int i);
+RcppExport SEXP _arrow_Array__IsValid(SEXP x_sexp, SEXP i_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::Array>&>::type x(x_sexp);
+	Rcpp::traits::input_parameter<int>::type i(i_sexp);
+	return Rcpp::wrap(Array__IsValid(x, i));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_Array__IsValid(SEXP x_sexp, SEXP i_sexp){
+	Rf_error("Cannot call Array__IsValid(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// array.cpp:45
+#if defined(ARROW_R_WITH_ARROW)
+int Array__length(const std::shared_ptr<arrow::Array>& x);
+RcppExport SEXP _arrow_Array__length(SEXP x_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::Array>&>::type x(x_sexp);
+	return Rcpp::wrap(Array__length(x));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_Array__length(SEXP x_sexp){
+	Rf_error("Cannot call Array__length(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// array.cpp:48
+#if defined(ARROW_R_WITH_ARROW)
+int Array__offset(const std::shared_ptr<arrow::Array>& x);
+RcppExport SEXP _arrow_Array__offset(SEXP x_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::Array>&>::type x(x_sexp);
+	return Rcpp::wrap(Array__offset(x));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_Array__offset(SEXP x_sexp){
+	Rf_error("Cannot call Array__offset(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// array.cpp:51
+#if defined(ARROW_R_WITH_ARROW)
+int Array__null_count(const std::shared_ptr<arrow::Array>& x);
+RcppExport SEXP _arrow_Array__null_count(SEXP x_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::Array>&>::type x(x_sexp);
+	return Rcpp::wrap(Array__null_count(x));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_Array__null_count(SEXP x_sexp){
+	Rf_error("Cannot call Array__null_count(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// array.cpp:54
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::DataType> Array__type(const std::shared_ptr<arrow::Array>& x);
+RcppExport SEXP _arrow_Array__type(SEXP x_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::Array>&>::type x(x_sexp);
+	return Rcpp::wrap(Array__type(x));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_Array__type(SEXP x_sexp){
+	Rf_error("Cannot call Array__type(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// array.cpp:59
+#if defined(ARROW_R_WITH_ARROW)
+std::string Array__ToString(const std::shared_ptr<arrow::Array>& x);
+RcppExport SEXP _arrow_Array__ToString(SEXP x_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::Array>&>::type x(x_sexp);
+	return Rcpp::wrap(Array__ToString(x));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_Array__ToString(SEXP x_sexp){
+	Rf_error("Cannot call Array__ToString(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// array.cpp:64
+#if defined(ARROW_R_WITH_ARROW)
+arrow::Type::type Array__type_id(const std::shared_ptr<arrow::Array>& x);
+RcppExport SEXP _arrow_Array__type_id(SEXP x_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::Array>&>::type x(x_sexp);
+	return Rcpp::wrap(Array__type_id(x));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_Array__type_id(SEXP x_sexp){
+	Rf_error("Cannot call Array__type_id(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// array.cpp:69
+#if defined(ARROW_R_WITH_ARROW)
+bool Array__Equals(const std::shared_ptr<arrow::Array>& lhs, const std::shared_ptr<arrow::Array>& rhs);
+RcppExport SEXP _arrow_Array__Equals(SEXP lhs_sexp, SEXP rhs_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::Array>&>::type lhs(lhs_sexp);
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::Array>&>::type rhs(rhs_sexp);
+	return Rcpp::wrap(Array__Equals(lhs, rhs));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_Array__Equals(SEXP lhs_sexp, SEXP rhs_sexp){
+	Rf_error("Cannot call Array__Equals(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// array.cpp:75
+#if defined(ARROW_R_WITH_ARROW)
+bool Array__ApproxEquals(const std::shared_ptr<arrow::Array>& lhs, const std::shared_ptr<arrow::Array>& rhs);
+RcppExport SEXP _arrow_Array__ApproxEquals(SEXP lhs_sexp, SEXP rhs_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::Array>&>::type lhs(lhs_sexp);
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::Array>&>::type rhs(rhs_sexp);
+	return Rcpp::wrap(Array__ApproxEquals(lhs, rhs));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_Array__ApproxEquals(SEXP lhs_sexp, SEXP rhs_sexp){
+	Rf_error("Cannot call Array__ApproxEquals(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// array.cpp:81
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::ArrayData> Array__data(const std::shared_ptr<arrow::Array>& array);
+RcppExport SEXP _arrow_Array__data(SEXP array_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::Array>&>::type array(array_sexp);
+	return Rcpp::wrap(Array__data(array));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_Array__data(SEXP array_sexp){
+	Rf_error("Cannot call Array__data(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// array.cpp:87
+#if defined(ARROW_R_WITH_ARROW)
+bool Array__RangeEquals(const std::shared_ptr<arrow::Array>& self, const std::shared_ptr<arrow::Array>& other, int start_idx, int end_idx, int other_start_idx);
+RcppExport SEXP _arrow_Array__RangeEquals(SEXP self_sexp, SEXP other_sexp, SEXP start_idx_sexp, SEXP end_idx_sexp, SEXP other_start_idx_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::Array>&>::type self(self_sexp);
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::Array>&>::type other(other_sexp);
+	Rcpp::traits::input_parameter<int>::type start_idx(start_idx_sexp);
+	Rcpp::traits::input_parameter<int>::type end_idx(end_idx_sexp);
+	Rcpp::traits::input_parameter<int>::type other_start_idx(other_start_idx_sexp);
+	return Rcpp::wrap(Array__RangeEquals(self, other, start_idx, end_idx, other_start_idx));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_Array__RangeEquals(SEXP self_sexp, SEXP other_sexp, SEXP start_idx_sexp, SEXP end_idx_sexp, SEXP other_start_idx_sexp){
+	Rf_error("Cannot call Array__RangeEquals(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// array.cpp:94
+#if defined(ARROW_R_WITH_ARROW)
+LogicalVector Array__Mask(const std::shared_ptr<arrow::Array>& array);
+RcppExport SEXP _arrow_Array__Mask(SEXP array_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::Array>&>::type array(array_sexp);
+	return Rcpp::wrap(Array__Mask(array));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_Array__Mask(SEXP array_sexp){
+	Rf_error("Cannot call Array__Mask(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// array.cpp:110
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::Array> DictionaryArray__indices(const std::shared_ptr<arrow::DictionaryArray>& array);
+RcppExport SEXP _arrow_DictionaryArray__indices(SEXP array_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::DictionaryArray>&>::type array(array_sexp);
+	return Rcpp::wrap(DictionaryArray__indices(array));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_DictionaryArray__indices(SEXP array_sexp){
+	Rf_error("Cannot call DictionaryArray__indices(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// array.cpp:116
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::Array> DictionaryArray__dictionary(const std::shared_ptr<arrow::DictionaryArray>& array);
+RcppExport SEXP _arrow_DictionaryArray__dictionary(SEXP array_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::DictionaryArray>&>::type array(array_sexp);
+	return Rcpp::wrap(DictionaryArray__dictionary(array));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_DictionaryArray__dictionary(SEXP array_sexp){
+	Rf_error("Cannot call DictionaryArray__dictionary(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// array__to_vector.cpp:669
+#if defined(ARROW_R_WITH_ARROW)
+SEXP Array__as_vector(const std::shared_ptr<arrow::Array>& array);
+RcppExport SEXP _arrow_Array__as_vector(SEXP array_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::Array>&>::type array(array_sexp);
+	return Rcpp::wrap(Array__as_vector(array));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_Array__as_vector(SEXP array_sexp){
+	Rf_error("Cannot call Array__as_vector(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// array__to_vector.cpp:674
+#if defined(ARROW_R_WITH_ARROW)
+SEXP ChunkedArray__as_vector(const std::shared_ptr<arrow::ChunkedArray>& chunked_array);
+RcppExport SEXP _arrow_ChunkedArray__as_vector(SEXP chunked_array_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::ChunkedArray>&>::type chunked_array(chunked_array_sexp);
+	return Rcpp::wrap(ChunkedArray__as_vector(chunked_array));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_ChunkedArray__as_vector(SEXP chunked_array_sexp){
+	Rf_error("Cannot call ChunkedArray__as_vector(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// array__to_vector.cpp:680
+#if defined(ARROW_R_WITH_ARROW)
+Rcpp::List RecordBatch__to_dataframe(const std::shared_ptr<arrow::RecordBatch>& batch, bool use_threads);
+RcppExport SEXP _arrow_RecordBatch__to_dataframe(SEXP batch_sexp, SEXP use_threads_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::RecordBatch>&>::type batch(batch_sexp);
+	Rcpp::traits::input_parameter<bool>::type use_threads(use_threads_sexp);
+	return Rcpp::wrap(RecordBatch__to_dataframe(batch, use_threads));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_RecordBatch__to_dataframe(SEXP batch_sexp, SEXP use_threads_sexp){
+	Rf_error("Cannot call RecordBatch__to_dataframe(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// array__to_vector.cpp:702
+#if defined(ARROW_R_WITH_ARROW)
+Rcpp::List Table__to_dataframe(const std::shared_ptr<arrow::Table>& table, bool use_threads);
+RcppExport SEXP _arrow_Table__to_dataframe(SEXP table_sexp, SEXP use_threads_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::Table>&>::type table(table_sexp);
+	Rcpp::traits::input_parameter<bool>::type use_threads(use_threads_sexp);
+	return Rcpp::wrap(Table__to_dataframe(table, use_threads));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_Table__to_dataframe(SEXP table_sexp, SEXP use_threads_sexp){
+	Rf_error("Cannot call Table__to_dataframe(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// array_from_vector.cpp:959
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::DataType> Array__infer_type(SEXP x);
+RcppExport SEXP _arrow_Array__infer_type(SEXP x_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<SEXP>::type x(x_sexp);
+	return Rcpp::wrap(Array__infer_type(x));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_Array__infer_type(SEXP x_sexp){
+	Rf_error("Cannot call Array__infer_type(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// array_from_vector.cpp:964
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::Array> Array__from_vector(SEXP x, SEXP s_type);
+RcppExport SEXP _arrow_Array__from_vector(SEXP x_sexp, SEXP s_type_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<SEXP>::type x(x_sexp);
+	Rcpp::traits::input_parameter<SEXP>::type s_type(s_type_sexp);
+	return Rcpp::wrap(Array__from_vector(x, s_type));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_Array__from_vector(SEXP x_sexp, SEXP s_type_sexp){
+	Rf_error("Cannot call Array__from_vector(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// array_from_vector.cpp:979
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::ChunkedArray> ChunkedArray__from_list(Rcpp::List chunks, SEXP s_type);
+RcppExport SEXP _arrow_ChunkedArray__from_list(SEXP chunks_sexp, SEXP s_type_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<Rcpp::List>::type chunks(chunks_sexp);
+	Rcpp::traits::input_parameter<SEXP>::type s_type(s_type_sexp);
+	return Rcpp::wrap(ChunkedArray__from_list(chunks, s_type));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_ChunkedArray__from_list(SEXP chunks_sexp, SEXP s_type_sexp){
+	Rf_error("Cannot call ChunkedArray__from_list(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// arraydata.cpp:25
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::DataType> ArrayData__get_type(const std::shared_ptr<arrow::ArrayData>& x);
+RcppExport SEXP _arrow_ArrayData__get_type(SEXP x_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::ArrayData>&>::type x(x_sexp);
+	return Rcpp::wrap(ArrayData__get_type(x));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_ArrayData__get_type(SEXP x_sexp){
+	Rf_error("Cannot call ArrayData__get_type(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// arraydata.cpp:31
+#if defined(ARROW_R_WITH_ARROW)
+int ArrayData__get_length(const std::shared_ptr<arrow::ArrayData>& x);
+RcppExport SEXP _arrow_ArrayData__get_length(SEXP x_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::ArrayData>&>::type x(x_sexp);
+	return Rcpp::wrap(ArrayData__get_length(x));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_ArrayData__get_length(SEXP x_sexp){
+	Rf_error("Cannot call ArrayData__get_length(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// arraydata.cpp:36
+#if defined(ARROW_R_WITH_ARROW)
+int ArrayData__get_null_count(const std::shared_ptr<arrow::ArrayData>& x);
+RcppExport SEXP _arrow_ArrayData__get_null_count(SEXP x_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::ArrayData>&>::type x(x_sexp);
+	return Rcpp::wrap(ArrayData__get_null_count(x));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_ArrayData__get_null_count(SEXP x_sexp){
+	Rf_error("Cannot call ArrayData__get_null_count(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// arraydata.cpp:41
+#if defined(ARROW_R_WITH_ARROW)
+int ArrayData__get_offset(const std::shared_ptr<arrow::ArrayData>& x);
+RcppExport SEXP _arrow_ArrayData__get_offset(SEXP x_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::ArrayData>&>::type x(x_sexp);
+	return Rcpp::wrap(ArrayData__get_offset(x));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_ArrayData__get_offset(SEXP x_sexp){
+	Rf_error("Cannot call ArrayData__get_offset(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// arraydata.cpp:46
+#if defined(ARROW_R_WITH_ARROW)
+List ArrayData__buffers(const std::shared_ptr<arrow::ArrayData>& x);
+RcppExport SEXP _arrow_ArrayData__buffers(SEXP x_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::ArrayData>&>::type x(x_sexp);
+	return Rcpp::wrap(ArrayData__buffers(x));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_ArrayData__buffers(SEXP x_sexp){
+	Rf_error("Cannot call ArrayData__buffers(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// buffer.cpp:22
+#if defined(ARROW_R_WITH_ARROW)
+bool Buffer__is_mutable(const std::shared_ptr<arrow::Buffer>& buffer);
+RcppExport SEXP _arrow_Buffer__is_mutable(SEXP buffer_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::Buffer>&>::type buffer(buffer_sexp);
+	return Rcpp::wrap(Buffer__is_mutable(buffer));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_Buffer__is_mutable(SEXP buffer_sexp){
+	Rf_error("Cannot call Buffer__is_mutable(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// buffer.cpp:27
+#if defined(ARROW_R_WITH_ARROW)
+void Buffer__ZeroPadding(const std::shared_ptr<arrow::Buffer>& buffer);
+RcppExport SEXP _arrow_Buffer__ZeroPadding(SEXP buffer_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::Buffer>&>::type buffer(buffer_sexp);
+	Buffer__ZeroPadding(buffer);
+	return R_NilValue;
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_Buffer__ZeroPadding(SEXP buffer_sexp){
+	Rf_error("Cannot call Buffer__ZeroPadding(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// buffer.cpp:32
+#if defined(ARROW_R_WITH_ARROW)
+int64_t Buffer__capacity(const std::shared_ptr<arrow::Buffer>& buffer);
+RcppExport SEXP _arrow_Buffer__capacity(SEXP buffer_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::Buffer>&>::type buffer(buffer_sexp);
+	return Rcpp::wrap(Buffer__capacity(buffer));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_Buffer__capacity(SEXP buffer_sexp){
+	Rf_error("Cannot call Buffer__capacity(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// buffer.cpp:37
+#if defined(ARROW_R_WITH_ARROW)
+int64_t Buffer__size(const std::shared_ptr<arrow::Buffer>& buffer);
+RcppExport SEXP _arrow_Buffer__size(SEXP buffer_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::Buffer>&>::type buffer(buffer_sexp);
+	return Rcpp::wrap(Buffer__size(buffer));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_Buffer__size(SEXP buffer_sexp){
+	Rf_error("Cannot call Buffer__size(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// buffer.cpp:42
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::Buffer> r___RBuffer__initialize(SEXP x);
+RcppExport SEXP _arrow_r___RBuffer__initialize(SEXP x_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<SEXP>::type x(x_sexp);
+	return Rcpp::wrap(r___RBuffer__initialize(x));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_r___RBuffer__initialize(SEXP x_sexp){
+	Rf_error("Cannot call r___RBuffer__initialize(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// chunkedarray.cpp:25
+#if defined(ARROW_R_WITH_ARROW)
+int ChunkedArray__length(const std::shared_ptr<arrow::ChunkedArray>& chunked_array);
+RcppExport SEXP _arrow_ChunkedArray__length(SEXP chunked_array_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::ChunkedArray>&>::type chunked_array(chunked_array_sexp);
+	return Rcpp::wrap(ChunkedArray__length(chunked_array));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_ChunkedArray__length(SEXP chunked_array_sexp){
+	Rf_error("Cannot call ChunkedArray__length(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// chunkedarray.cpp:30
+#if defined(ARROW_R_WITH_ARROW)
+int ChunkedArray__null_count(const std::shared_ptr<arrow::ChunkedArray>& chunked_array);
+RcppExport SEXP _arrow_ChunkedArray__null_count(SEXP chunked_array_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::ChunkedArray>&>::type chunked_array(chunked_array_sexp);
+	return Rcpp::wrap(ChunkedArray__null_count(chunked_array));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_ChunkedArray__null_count(SEXP chunked_array_sexp){
+	Rf_error("Cannot call ChunkedArray__null_count(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// chunkedarray.cpp:35
+#if defined(ARROW_R_WITH_ARROW)
+int ChunkedArray__num_chunks(const std::shared_ptr<arrow::ChunkedArray>& chunked_array);
+RcppExport SEXP _arrow_ChunkedArray__num_chunks(SEXP chunked_array_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::ChunkedArray>&>::type chunked_array(chunked_array_sexp);
+	return Rcpp::wrap(ChunkedArray__num_chunks(chunked_array));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_ChunkedArray__num_chunks(SEXP chunked_array_sexp){
+	Rf_error("Cannot call ChunkedArray__num_chunks(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// chunkedarray.cpp:40
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::Array> ChunkedArray__chunk(const std::shared_ptr<arrow::ChunkedArray>& chunked_array, int i);
+RcppExport SEXP _arrow_ChunkedArray__chunk(SEXP chunked_array_sexp, SEXP i_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::ChunkedArray>&>::type chunked_array(chunked_array_sexp);
+	Rcpp::traits::input_parameter<int>::type i(i_sexp);
+	return Rcpp::wrap(ChunkedArray__chunk(chunked_array, i));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_ChunkedArray__chunk(SEXP chunked_array_sexp, SEXP i_sexp){
+	Rf_error("Cannot call ChunkedArray__chunk(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// chunkedarray.cpp:46
+#if defined(ARROW_R_WITH_ARROW)
+List ChunkedArray__chunks(const std::shared_ptr<arrow::ChunkedArray>& chunked_array);
+RcppExport SEXP _arrow_ChunkedArray__chunks(SEXP chunked_array_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::ChunkedArray>&>::type chunked_array(chunked_array_sexp);
+	return Rcpp::wrap(ChunkedArray__chunks(chunked_array));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_ChunkedArray__chunks(SEXP chunked_array_sexp){
+	Rf_error("Cannot call ChunkedArray__chunks(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// chunkedarray.cpp:51
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::DataType> ChunkedArray__type(const std::shared_ptr<arrow::ChunkedArray>& chunked_array);
+RcppExport SEXP _arrow_ChunkedArray__type(SEXP chunked_array_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::ChunkedArray>&>::type chunked_array(chunked_array_sexp);
+	return Rcpp::wrap(ChunkedArray__type(chunked_array));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_ChunkedArray__type(SEXP chunked_array_sexp){
+	Rf_error("Cannot call ChunkedArray__type(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// chunkedarray.cpp:57
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::ChunkedArray> ChunkArray__Slice1(const std::shared_ptr<arrow::ChunkedArray>& chunked_array, int offset);
+RcppExport SEXP _arrow_ChunkArray__Slice1(SEXP chunked_array_sexp, SEXP offset_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::ChunkedArray>&>::type chunked_array(chunked_array_sexp);
+	Rcpp::traits::input_parameter<int>::type offset(offset_sexp);
+	return Rcpp::wrap(ChunkArray__Slice1(chunked_array, offset));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_ChunkArray__Slice1(SEXP chunked_array_sexp, SEXP offset_sexp){
+	Rf_error("Cannot call ChunkArray__Slice1(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// chunkedarray.cpp:63
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::ChunkedArray> ChunkArray__Slice2(const std::shared_ptr<arrow::ChunkedArray>& chunked_array, int offset, int length);
+RcppExport SEXP _arrow_ChunkArray__Slice2(SEXP chunked_array_sexp, SEXP offset_sexp, SEXP length_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::ChunkedArray>&>::type chunked_array(chunked_array_sexp);
+	Rcpp::traits::input_parameter<int>::type offset(offset_sexp);
+	Rcpp::traits::input_parameter<int>::type length(length_sexp);
+	return Rcpp::wrap(ChunkArray__Slice2(chunked_array, offset, length));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_ChunkArray__Slice2(SEXP chunked_array_sexp, SEXP offset_sexp, SEXP length_sexp){
+	Rf_error("Cannot call ChunkArray__Slice2(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// column.cpp:22
+#if defined(ARROW_R_WITH_ARROW)
+int Column__length(const std::shared_ptr<arrow::Column>& column);
+RcppExport SEXP _arrow_Column__length(SEXP column_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::Column>&>::type column(column_sexp);
+	return Rcpp::wrap(Column__length(column));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_Column__length(SEXP column_sexp){
+	Rf_error("Cannot call Column__length(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// column.cpp:27
+#if defined(ARROW_R_WITH_ARROW)
+int Column__null_count(const std::shared_ptr<arrow::Column>& column);
+RcppExport SEXP _arrow_Column__null_count(SEXP column_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::Column>&>::type column(column_sexp);
+	return Rcpp::wrap(Column__null_count(column));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_Column__null_count(SEXP column_sexp){
+	Rf_error("Cannot call Column__null_count(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// column.cpp:32
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::DataType> Column__type(const std::shared_ptr<arrow::Column>& column);
+RcppExport SEXP _arrow_Column__type(SEXP column_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::Column>&>::type column(column_sexp);
+	return Rcpp::wrap(Column__type(column));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_Column__type(SEXP column_sexp){
+	Rf_error("Cannot call Column__type(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// column.cpp:38
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::ChunkedArray> Column__data(const std::shared_ptr<arrow::Column>& column);
+RcppExport SEXP _arrow_Column__data(SEXP column_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::Column>&>::type column(column_sexp);
+	return Rcpp::wrap(Column__data(column));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_Column__data(SEXP column_sexp){
+	Rf_error("Cannot call Column__data(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// compression.cpp:22
+#if defined(ARROW_R_WITH_ARROW)
+std::unique_ptr<arrow::util::Codec> util___Codec__Create(arrow::Compression::type codec);
+RcppExport SEXP _arrow_util___Codec__Create(SEXP codec_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<arrow::Compression::type>::type codec(codec_sexp);
+	return Rcpp::wrap(util___Codec__Create(codec));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_util___Codec__Create(SEXP codec_sexp){
+	Rf_error("Cannot call util___Codec__Create(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// compression.cpp:29
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::io::CompressedOutputStream> io___CompressedOutputStream__Make(const std::unique_ptr<arrow::util::Codec>& codec, const std::shared_ptr<arrow::io::OutputStream>& raw);
+RcppExport SEXP _arrow_io___CompressedOutputStream__Make(SEXP codec_sexp, SEXP raw_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::unique_ptr<arrow::util::Codec>&>::type codec(codec_sexp);
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::io::OutputStream>&>::type raw(raw_sexp);
+	return Rcpp::wrap(io___CompressedOutputStream__Make(codec, raw));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_io___CompressedOutputStream__Make(SEXP codec_sexp, SEXP raw_sexp){
+	Rf_error("Cannot call io___CompressedOutputStream__Make(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// compression.cpp:38
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::io::CompressedInputStream> io___CompressedInputStream__Make(const std::unique_ptr<arrow::util::Codec>& codec, const std::shared_ptr<arrow::io::InputStream>& raw);
+RcppExport SEXP _arrow_io___CompressedInputStream__Make(SEXP codec_sexp, SEXP raw_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::unique_ptr<arrow::util::Codec>&>::type codec(codec_sexp);
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::io::InputStream>&>::type raw(raw_sexp);
+	return Rcpp::wrap(io___CompressedInputStream__Make(codec, raw));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_io___CompressedInputStream__Make(SEXP codec_sexp, SEXP raw_sexp){
+	Rf_error("Cannot call io___CompressedInputStream__Make(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// compute.cpp:22
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::compute::CastOptions> compute___CastOptions__initialize(bool allow_int_overflow, bool allow_time_truncate, bool allow_float_truncate);
+RcppExport SEXP _arrow_compute___CastOptions__initialize(SEXP allow_int_overflow_sexp, SEXP allow_time_truncate_sexp, SEXP allow_float_truncate_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<bool>::type allow_int_overflow(allow_int_overflow_sexp);
+	Rcpp::traits::input_parameter<bool>::type allow_time_truncate(allow_time_truncate_sexp);
+	Rcpp::traits::input_parameter<bool>::type allow_float_truncate(allow_float_truncate_sexp);
+	return Rcpp::wrap(compute___CastOptions__initialize(allow_int_overflow, allow_time_truncate, allow_float_truncate));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_compute___CastOptions__initialize(SEXP allow_int_overflow_sexp, SEXP allow_time_truncate_sexp, SEXP allow_float_truncate_sexp){
+	Rf_error("Cannot call compute___CastOptions__initialize(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// compute.cpp:32
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::Array> Array__cast(const std::shared_ptr<arrow::Array>& array, const std::shared_ptr<arrow::DataType>& target_type, const std::shared_ptr<arrow::compute::CastOptions>& options);
+RcppExport SEXP _arrow_Array__cast(SEXP array_sexp, SEXP target_type_sexp, SEXP options_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::Array>&>::type array(array_sexp);
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::DataType>&>::type target_type(target_type_sexp);
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::compute::CastOptions>&>::type options(options_sexp);
+	return Rcpp::wrap(Array__cast(array, target_type, options));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_Array__cast(SEXP array_sexp, SEXP target_type_sexp, SEXP options_sexp){
+	Rf_error("Cannot call Array__cast(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// compute.cpp:43
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::ChunkedArray> ChunkedArray__cast(const std::shared_ptr<arrow::ChunkedArray>& chunked_array, const std::shared_ptr<arrow::DataType>& target_type, const std::shared_ptr<arrow::compute::CastOptions>& options);
+RcppExport SEXP _arrow_ChunkedArray__cast(SEXP chunked_array_sexp, SEXP target_type_sexp, SEXP options_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::ChunkedArray>&>::type chunked_array(chunked_array_sexp);
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::DataType>&>::type target_type(target_type_sexp);
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::compute::CastOptions>&>::type options(options_sexp);
+	return Rcpp::wrap(ChunkedArray__cast(chunked_array, target_type, options));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_ChunkedArray__cast(SEXP chunked_array_sexp, SEXP target_type_sexp, SEXP options_sexp){
+	Rf_error("Cannot call ChunkedArray__cast(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// compute.cpp:55
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::RecordBatch> RecordBatch__cast(const std::shared_ptr<arrow::RecordBatch>& batch, const std::shared_ptr<arrow::Schema>& schema, const std::shared_ptr<arrow::compute::CastOptions>& options);
+RcppExport SEXP _arrow_RecordBatch__cast(SEXP batch_sexp, SEXP schema_sexp, SEXP options_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::RecordBatch>&>::type batch(batch_sexp);
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::Schema>&>::type schema(schema_sexp);
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::compute::CastOptions>&>::type options(options_sexp);
+	return Rcpp::wrap(RecordBatch__cast(batch, schema, options));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_RecordBatch__cast(SEXP batch_sexp, SEXP schema_sexp, SEXP options_sexp){
+	Rf_error("Cannot call RecordBatch__cast(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// compute.cpp:70
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::Table> Table__cast(const std::shared_ptr<arrow::Table>& table, const std::shared_ptr<arrow::Schema>& schema, const std::shared_ptr<arrow::compute::CastOptions>& options);
+RcppExport SEXP _arrow_Table__cast(SEXP table_sexp, SEXP schema_sexp, SEXP options_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::Table>&>::type table(table_sexp);
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::Schema>&>::type schema(schema_sexp);
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::compute::CastOptions>&>::type options(options_sexp);
+	return Rcpp::wrap(Table__cast(table, schema, options));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_Table__cast(SEXP table_sexp, SEXP schema_sexp, SEXP options_sexp){
+	Rf_error("Cannot call Table__cast(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// csv.cpp:25
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::csv::ReadOptions> csv___ReadOptions__initialize(List_ options);
+RcppExport SEXP _arrow_csv___ReadOptions__initialize(SEXP options_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<List_>::type options(options_sexp);
+	return Rcpp::wrap(csv___ReadOptions__initialize(options));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_csv___ReadOptions__initialize(SEXP options_sexp){
+	Rf_error("Cannot call csv___ReadOptions__initialize(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// csv.cpp:36
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::csv::ParseOptions> csv___ParseOptions__initialize(List_ options);
+RcppExport SEXP _arrow_csv___ParseOptions__initialize(SEXP options_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<List_>::type options(options_sexp);
+	return Rcpp::wrap(csv___ParseOptions__initialize(options));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_csv___ParseOptions__initialize(SEXP options_sexp){
+	Rf_error("Cannot call csv___ParseOptions__initialize(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// csv.cpp:51
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::csv::ConvertOptions> csv___ConvertOptions__initialize(List_ options);
+RcppExport SEXP _arrow_csv___ConvertOptions__initialize(SEXP options_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<List_>::type options(options_sexp);
+	return Rcpp::wrap(csv___ConvertOptions__initialize(options));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_csv___ConvertOptions__initialize(SEXP options_sexp){
+	Rf_error("Cannot call csv___ConvertOptions__initialize(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// csv.cpp:60
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::csv::TableReader> csv___TableReader__Make(const std::shared_ptr<arrow::io::InputStream>& input, const std::shared_ptr<arrow::csv::ReadOptions>& read_options, const std::shared_ptr<arrow::csv::ParseOptions>& parse_options, const std::shared_ptr<arrow::csv::ConvertOptions>& convert_options);
+RcppExport SEXP _arrow_csv___TableReader__Make(SEXP input_sexp, SEXP read_options_sexp, SEXP parse_options_sexp, SEXP convert_options_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::io::InputStream>&>::type input(input_sexp);
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::csv::ReadOptions>&>::type read_options(read_options_sexp);
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::csv::ParseOptions>&>::type parse_options(parse_options_sexp);
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::csv::ConvertOptions>&>::type convert_options(convert_options_sexp);
+	return Rcpp::wrap(csv___TableReader__Make(input, read_options, parse_options, convert_options));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_csv___TableReader__Make(SEXP input_sexp, SEXP read_options_sexp, SEXP parse_options_sexp, SEXP convert_options_sexp){
+	Rf_error("Cannot call csv___TableReader__Make(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// csv.cpp:73
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::Table> csv___TableReader__Read(const std::shared_ptr<arrow::csv::TableReader>& table_reader);
+RcppExport SEXP _arrow_csv___TableReader__Read(SEXP table_reader_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::csv::TableReader>&>::type table_reader(table_reader_sexp);
+	return Rcpp::wrap(csv___TableReader__Read(table_reader));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_csv___TableReader__Read(SEXP table_reader_sexp){
+	Rf_error("Cannot call csv___TableReader__Read(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// datatype.cpp:27
+#if defined(ARROW_R_WITH_ARROW)
+bool shared_ptr_is_null(SEXP xp);
+RcppExport SEXP _arrow_shared_ptr_is_null(SEXP xp_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<SEXP>::type xp(xp_sexp);
+	return Rcpp::wrap(shared_ptr_is_null(xp));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_shared_ptr_is_null(SEXP xp_sexp){
+	Rf_error("Cannot call shared_ptr_is_null(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// datatype.cpp:32
+#if defined(ARROW_R_WITH_ARROW)
+bool unique_ptr_is_null(SEXP xp);
+RcppExport SEXP _arrow_unique_ptr_is_null(SEXP xp_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<SEXP>::type xp(xp_sexp);
+	return Rcpp::wrap(unique_ptr_is_null(xp));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_unique_ptr_is_null(SEXP xp_sexp){
+	Rf_error("Cannot call unique_ptr_is_null(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// datatype.cpp:37
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::DataType> Int8__initialize();
+RcppExport SEXP _arrow_Int8__initialize(){
+BEGIN_RCPP
+	return Rcpp::wrap(Int8__initialize());
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_Int8__initialize(){
+	Rf_error("Cannot call Int8__initialize(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// datatype.cpp:40
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::DataType> Int16__initialize();
+RcppExport SEXP _arrow_Int16__initialize(){
+BEGIN_RCPP
+	return Rcpp::wrap(Int16__initialize());
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_Int16__initialize(){
+	Rf_error("Cannot call Int16__initialize(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// datatype.cpp:43
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::DataType> Int32__initialize();
+RcppExport SEXP _arrow_Int32__initialize(){
+BEGIN_RCPP
+	return Rcpp::wrap(Int32__initialize());
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_Int32__initialize(){
+	Rf_error("Cannot call Int32__initialize(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// datatype.cpp:46
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::DataType> Int64__initialize();
+RcppExport SEXP _arrow_Int64__initialize(){
+BEGIN_RCPP
+	return Rcpp::wrap(Int64__initialize());
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_Int64__initialize(){
+	Rf_error("Cannot call Int64__initialize(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// datatype.cpp:49
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::DataType> UInt8__initialize();
+RcppExport SEXP _arrow_UInt8__initialize(){
+BEGIN_RCPP
+	return Rcpp::wrap(UInt8__initialize());
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_UInt8__initialize(){
+	Rf_error("Cannot call UInt8__initialize(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// datatype.cpp:52
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::DataType> UInt16__initialize();
+RcppExport SEXP _arrow_UInt16__initialize(){
+BEGIN_RCPP
+	return Rcpp::wrap(UInt16__initialize());
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_UInt16__initialize(){
+	Rf_error("Cannot call UInt16__initialize(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// datatype.cpp:55
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::DataType> UInt32__initialize();
+RcppExport SEXP _arrow_UInt32__initialize(){
+BEGIN_RCPP
+	return Rcpp::wrap(UInt32__initialize());
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_UInt32__initialize(){
+	Rf_error("Cannot call UInt32__initialize(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// datatype.cpp:58
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::DataType> UInt64__initialize();
+RcppExport SEXP _arrow_UInt64__initialize(){
+BEGIN_RCPP
+	return Rcpp::wrap(UInt64__initialize());
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_UInt64__initialize(){
+	Rf_error("Cannot call UInt64__initialize(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// datatype.cpp:61
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::DataType> Float16__initialize();
+RcppExport SEXP _arrow_Float16__initialize(){
+BEGIN_RCPP
+	return Rcpp::wrap(Float16__initialize());
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_Float16__initialize(){
+	Rf_error("Cannot call Float16__initialize(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// datatype.cpp:64
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::DataType> Float32__initialize();
+RcppExport SEXP _arrow_Float32__initialize(){
+BEGIN_RCPP
+	return Rcpp::wrap(Float32__initialize());
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_Float32__initialize(){
+	Rf_error("Cannot call Float32__initialize(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// datatype.cpp:67
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::DataType> Float64__initialize();
+RcppExport SEXP _arrow_Float64__initialize(){
+BEGIN_RCPP
+	return Rcpp::wrap(Float64__initialize());
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_Float64__initialize(){
+	Rf_error("Cannot call Float64__initialize(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// datatype.cpp:70
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::DataType> Boolean__initialize();
+RcppExport SEXP _arrow_Boolean__initialize(){
+BEGIN_RCPP
+	return Rcpp::wrap(Boolean__initialize());
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_Boolean__initialize(){
+	Rf_error("Cannot call Boolean__initialize(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// datatype.cpp:73
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::DataType> Utf8__initialize();
+RcppExport SEXP _arrow_Utf8__initialize(){
+BEGIN_RCPP
+	return Rcpp::wrap(Utf8__initialize());
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_Utf8__initialize(){
+	Rf_error("Cannot call Utf8__initialize(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// datatype.cpp:76
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::DataType> Date32__initialize();
+RcppExport SEXP _arrow_Date32__initialize(){
+BEGIN_RCPP
+	return Rcpp::wrap(Date32__initialize());
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_Date32__initialize(){
+	Rf_error("Cannot call Date32__initialize(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// datatype.cpp:79
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::DataType> Date64__initialize();
+RcppExport SEXP _arrow_Date64__initialize(){
+BEGIN_RCPP
+	return Rcpp::wrap(Date64__initialize());
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_Date64__initialize(){
+	Rf_error("Cannot call Date64__initialize(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// datatype.cpp:82
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::DataType> Null__initialize();
+RcppExport SEXP _arrow_Null__initialize(){
+BEGIN_RCPP
+	return Rcpp::wrap(Null__initialize());
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_Null__initialize(){
+	Rf_error("Cannot call Null__initialize(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// datatype.cpp:85
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::DataType> Decimal128Type__initialize(int32_t precision, int32_t scale);
+RcppExport SEXP _arrow_Decimal128Type__initialize(SEXP precision_sexp, SEXP scale_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<int32_t>::type precision(precision_sexp);
+	Rcpp::traits::input_parameter<int32_t>::type scale(scale_sexp);
+	return Rcpp::wrap(Decimal128Type__initialize(precision, scale));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_Decimal128Type__initialize(SEXP precision_sexp, SEXP scale_sexp){
+	Rf_error("Cannot call Decimal128Type__initialize(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// datatype.cpp:91
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::DataType> FixedSizeBinary__initialize(int32_t byte_width);
+RcppExport SEXP _arrow_FixedSizeBinary__initialize(SEXP byte_width_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<int32_t>::type byte_width(byte_width_sexp);
+	return Rcpp::wrap(FixedSizeBinary__initialize(byte_width));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_FixedSizeBinary__initialize(SEXP byte_width_sexp){
+	Rf_error("Cannot call FixedSizeBinary__initialize(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// datatype.cpp:96
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::DataType> Timestamp__initialize1(arrow::TimeUnit::type unit);
+RcppExport SEXP _arrow_Timestamp__initialize1(SEXP unit_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<arrow::TimeUnit::type>::type unit(unit_sexp);
+	return Rcpp::wrap(Timestamp__initialize1(unit));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_Timestamp__initialize1(SEXP unit_sexp){
+	Rf_error("Cannot call Timestamp__initialize1(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// datatype.cpp:101
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::DataType> Timestamp__initialize2(arrow::TimeUnit::type unit, const std::string& timezone);
+RcppExport SEXP _arrow_Timestamp__initialize2(SEXP unit_sexp, SEXP timezone_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<arrow::TimeUnit::type>::type unit(unit_sexp);
+	Rcpp::traits::input_parameter<const std::string&>::type timezone(timezone_sexp);
+	return Rcpp::wrap(Timestamp__initialize2(unit, timezone));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_Timestamp__initialize2(SEXP unit_sexp, SEXP timezone_sexp){
+	Rf_error("Cannot call Timestamp__initialize2(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// datatype.cpp:107
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::DataType> Time32__initialize(arrow::TimeUnit::type unit);
+RcppExport SEXP _arrow_Time32__initialize(SEXP unit_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<arrow::TimeUnit::type>::type unit(unit_sexp);
+	return Rcpp::wrap(Time32__initialize(unit));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_Time32__initialize(SEXP unit_sexp){
+	Rf_error("Cannot call Time32__initialize(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// datatype.cpp:112
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::DataType> Time64__initialize(arrow::TimeUnit::type unit);
+RcppExport SEXP _arrow_Time64__initialize(SEXP unit_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<arrow::TimeUnit::type>::type unit(unit_sexp);
+	return Rcpp::wrap(Time64__initialize(unit));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_Time64__initialize(SEXP unit_sexp){
+	Rf_error("Cannot call Time64__initialize(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// datatype.cpp:117
+#if defined(ARROW_R_WITH_ARROW)
+SEXP list__(SEXP x);
+RcppExport SEXP _arrow_list__(SEXP x_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<SEXP>::type x(x_sexp);
+	return Rcpp::wrap(list__(x));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_list__(SEXP x_sexp){
+	Rf_error("Cannot call list__(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// datatype.cpp:133
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::DataType> struct_(List fields);
+RcppExport SEXP _arrow_struct_(SEXP fields_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<List>::type fields(fields_sexp);
+	return Rcpp::wrap(struct_(fields));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_struct_(SEXP fields_sexp){
+	Rf_error("Cannot call struct_(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// datatype.cpp:138
+#if defined(ARROW_R_WITH_ARROW)
+std::string DataType__ToString(const std::shared_ptr<arrow::DataType>& type);
+RcppExport SEXP _arrow_DataType__ToString(SEXP type_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::DataType>&>::type type(type_sexp);
+	return Rcpp::wrap(DataType__ToString(type));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_DataType__ToString(SEXP type_sexp){
+	Rf_error("Cannot call DataType__ToString(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// datatype.cpp:143
+#if defined(ARROW_R_WITH_ARROW)
+std::string DataType__name(const std::shared_ptr<arrow::DataType>& type);
+RcppExport SEXP _arrow_DataType__name(SEXP type_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::DataType>&>::type type(type_sexp);
+	return Rcpp::wrap(DataType__name(type));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_DataType__name(SEXP type_sexp){
+	Rf_error("Cannot call DataType__name(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// datatype.cpp:148
+#if defined(ARROW_R_WITH_ARROW)
+bool DataType__Equals(const std::shared_ptr<arrow::DataType>& lhs, const std::shared_ptr<arrow::DataType>& rhs);
+RcppExport SEXP _arrow_DataType__Equals(SEXP lhs_sexp, SEXP rhs_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::DataType>&>::type lhs(lhs_sexp);
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::DataType>&>::type rhs(rhs_sexp);
+	return Rcpp::wrap(DataType__Equals(lhs, rhs));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_DataType__Equals(SEXP lhs_sexp, SEXP rhs_sexp){
+	Rf_error("Cannot call DataType__Equals(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// datatype.cpp:154
+#if defined(ARROW_R_WITH_ARROW)
+int DataType__num_children(const std::shared_ptr<arrow::DataType>& type);
+RcppExport SEXP _arrow_DataType__num_children(SEXP type_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::DataType>&>::type type(type_sexp);
+	return Rcpp::wrap(DataType__num_children(type));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_DataType__num_children(SEXP type_sexp){
+	Rf_error("Cannot call DataType__num_children(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// datatype.cpp:159
+#if defined(ARROW_R_WITH_ARROW)
+List DataType__children_pointer(const std::shared_ptr<arrow::DataType>& type);
+RcppExport SEXP _arrow_DataType__children_pointer(SEXP type_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::DataType>&>::type type(type_sexp);
+	return Rcpp::wrap(DataType__children_pointer(type));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_DataType__children_pointer(SEXP type_sexp){
+	Rf_error("Cannot call DataType__children_pointer(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// datatype.cpp:164
+#if defined(ARROW_R_WITH_ARROW)
+arrow::Type::type DataType__id(const std::shared_ptr<arrow::DataType>& type);
+RcppExport SEXP _arrow_DataType__id(SEXP type_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::DataType>&>::type type(type_sexp);
+	return Rcpp::wrap(DataType__id(type));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_DataType__id(SEXP type_sexp){
+	Rf_error("Cannot call DataType__id(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// datatype.cpp:169
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::Schema> schema_(List fields);
+RcppExport SEXP _arrow_schema_(SEXP fields_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<List>::type fields(fields_sexp);
+	return Rcpp::wrap(schema_(fields));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_schema_(SEXP fields_sexp){
+	Rf_error("Cannot call schema_(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// datatype.cpp:174
+#if defined(ARROW_R_WITH_ARROW)
+std::string Schema__ToString(const std::shared_ptr<arrow::Schema>& s);
+RcppExport SEXP _arrow_Schema__ToString(SEXP s_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::Schema>&>::type s(s_sexp);
+	return Rcpp::wrap(Schema__ToString(s));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_Schema__ToString(SEXP s_sexp){
+	Rf_error("Cannot call Schema__ToString(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// datatype.cpp:179
+#if defined(ARROW_R_WITH_ARROW)
+int Schema__num_fields(const std::shared_ptr<arrow::Schema>& s);
+RcppExport SEXP _arrow_Schema__num_fields(SEXP s_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::Schema>&>::type s(s_sexp);
+	return Rcpp::wrap(Schema__num_fields(s));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_Schema__num_fields(SEXP s_sexp){
+	Rf_error("Cannot call Schema__num_fields(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// datatype.cpp:184
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::Field> Schema__field(const std::shared_ptr<arrow::Schema>& s, int i);
+RcppExport SEXP _arrow_Schema__field(SEXP s_sexp, SEXP i_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::Schema>&>::type s(s_sexp);
+	Rcpp::traits::input_parameter<int>::type i(i_sexp);
+	return Rcpp::wrap(Schema__field(s, i));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_Schema__field(SEXP s_sexp, SEXP i_sexp){
+	Rf_error("Cannot call Schema__field(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// datatype.cpp:194
+#if defined(ARROW_R_WITH_ARROW)
+CharacterVector Schema__names(const std::shared_ptr<arrow::Schema>& schema);
+RcppExport SEXP _arrow_Schema__names(SEXP schema_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::Schema>&>::type schema(schema_sexp);
+	return Rcpp::wrap(Schema__names(schema));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_Schema__names(SEXP schema_sexp){
+	Rf_error("Cannot call Schema__names(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// datatype.cpp:202
+#if defined(ARROW_R_WITH_ARROW)
+std::string ListType__ToString(const std::shared_ptr<arrow::ListType>& type);
+RcppExport SEXP _arrow_ListType__ToString(SEXP type_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::ListType>&>::type type(type_sexp);
+	return Rcpp::wrap(ListType__ToString(type));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_ListType__ToString(SEXP type_sexp){
+	Rf_error("Cannot call ListType__ToString(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// datatype.cpp:207
+#if defined(ARROW_R_WITH_ARROW)
+int FixedWidthType__bit_width(const std::shared_ptr<arrow::FixedWidthType>& type);
+RcppExport SEXP _arrow_FixedWidthType__bit_width(SEXP type_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::FixedWidthType>&>::type type(type_sexp);
+	return Rcpp::wrap(FixedWidthType__bit_width(type));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_FixedWidthType__bit_width(SEXP type_sexp){
+	Rf_error("Cannot call FixedWidthType__bit_width(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// datatype.cpp:212
+#if defined(ARROW_R_WITH_ARROW)
+arrow::DateUnit DateType__unit(const std::shared_ptr<arrow::DateType>& type);
+RcppExport SEXP _arrow_DateType__unit(SEXP type_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::DateType>&>::type type(type_sexp);
+	return Rcpp::wrap(DateType__unit(type));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_DateType__unit(SEXP type_sexp){
+	Rf_error("Cannot call DateType__unit(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// datatype.cpp:217
+#if defined(ARROW_R_WITH_ARROW)
+arrow::TimeUnit::type TimeType__unit(const std::shared_ptr<arrow::TimeType>& type);
+RcppExport SEXP _arrow_TimeType__unit(SEXP type_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::TimeType>&>::type type(type_sexp);
+	return Rcpp::wrap(TimeType__unit(type));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_TimeType__unit(SEXP type_sexp){
+	Rf_error("Cannot call TimeType__unit(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// datatype.cpp:222
+#if defined(ARROW_R_WITH_ARROW)
+int32_t DecimalType__precision(const std::shared_ptr<arrow::DecimalType>& type);
+RcppExport SEXP _arrow_DecimalType__precision(SEXP type_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::DecimalType>&>::type type(type_sexp);
+	return Rcpp::wrap(DecimalType__precision(type));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_DecimalType__precision(SEXP type_sexp){
+	Rf_error("Cannot call DecimalType__precision(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// datatype.cpp:227
+#if defined(ARROW_R_WITH_ARROW)
+int32_t DecimalType__scale(const std::shared_ptr<arrow::DecimalType>& type);
+RcppExport SEXP _arrow_DecimalType__scale(SEXP type_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::DecimalType>&>::type type(type_sexp);
+	return Rcpp::wrap(DecimalType__scale(type));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_DecimalType__scale(SEXP type_sexp){
+	Rf_error("Cannot call DecimalType__scale(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// datatype.cpp:232
+#if defined(ARROW_R_WITH_ARROW)
+std::string TimestampType__timezone(const std::shared_ptr<arrow::TimestampType>& type);
+RcppExport SEXP _arrow_TimestampType__timezone(SEXP type_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::TimestampType>&>::type type(type_sexp);
+	return Rcpp::wrap(TimestampType__timezone(type));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_TimestampType__timezone(SEXP type_sexp){
+	Rf_error("Cannot call TimestampType__timezone(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// datatype.cpp:237
+#if defined(ARROW_R_WITH_ARROW)
+arrow::TimeUnit::type TimestampType__unit(const std::shared_ptr<arrow::TimestampType>& type);
+RcppExport SEXP _arrow_TimestampType__unit(SEXP type_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::TimestampType>&>::type type(type_sexp);
+	return Rcpp::wrap(TimestampType__unit(type));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_TimestampType__unit(SEXP type_sexp){
+	Rf_error("Cannot call TimestampType__unit(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// datatype.cpp:243
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::DataType> DictionaryType__initialize(const std::shared_ptr<arrow::DataType>& index_type, const std::shared_ptr<arrow::DataType>& value_type, bool ordered);
+RcppExport SEXP _arrow_DictionaryType__initialize(SEXP index_type_sexp, SEXP value_type_sexp, SEXP ordered_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::DataType>&>::type index_type(index_type_sexp);
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::DataType>&>::type value_type(value_type_sexp);
+	Rcpp::traits::input_parameter<bool>::type ordered(ordered_sexp);
+	return Rcpp::wrap(DictionaryType__initialize(index_type, value_type, ordered));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_DictionaryType__initialize(SEXP index_type_sexp, SEXP value_type_sexp, SEXP ordered_sexp){
+	Rf_error("Cannot call DictionaryType__initialize(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// datatype.cpp:250
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::DataType> DictionaryType__index_type(const std::shared_ptr<arrow::DictionaryType>& type);
+RcppExport SEXP _arrow_DictionaryType__index_type(SEXP type_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::DictionaryType>&>::type type(type_sexp);
+	return Rcpp::wrap(DictionaryType__index_type(type));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_DictionaryType__index_type(SEXP type_sexp){
+	Rf_error("Cannot call DictionaryType__index_type(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// datatype.cpp:256
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::DataType> DictionaryType__value_type(const std::shared_ptr<arrow::DictionaryType>& type);
+RcppExport SEXP _arrow_DictionaryType__value_type(SEXP type_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::DictionaryType>&>::type type(type_sexp);
+	return Rcpp::wrap(DictionaryType__value_type(type));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_DictionaryType__value_type(SEXP type_sexp){
+	Rf_error("Cannot call DictionaryType__value_type(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// datatype.cpp:262
+#if defined(ARROW_R_WITH_ARROW)
+std::string DictionaryType__name(const std::shared_ptr<arrow::DictionaryType>& type);
+RcppExport SEXP _arrow_DictionaryType__name(SEXP type_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::DictionaryType>&>::type type(type_sexp);
+	return Rcpp::wrap(DictionaryType__name(type));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_DictionaryType__name(SEXP type_sexp){
+	Rf_error("Cannot call DictionaryType__name(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// datatype.cpp:267
+#if defined(ARROW_R_WITH_ARROW)
+bool DictionaryType__ordered(const std::shared_ptr<arrow::DictionaryType>& type);
+RcppExport SEXP _arrow_DictionaryType__ordered(SEXP type_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::DictionaryType>&>::type type(type_sexp);
+	return Rcpp::wrap(DictionaryType__ordered(type));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_DictionaryType__ordered(SEXP type_sexp){
+	Rf_error("Cannot call DictionaryType__ordered(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// feather.cpp:24
+#if defined(ARROW_R_WITH_ARROW)
+void ipc___feather___TableWriter__SetDescription(const std::unique_ptr<arrow::ipc::feather::TableWriter>& writer, const std::string& description);
+RcppExport SEXP _arrow_ipc___feather___TableWriter__SetDescription(SEXP writer_sexp, SEXP description_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::unique_ptr<arrow::ipc::feather::TableWriter>&>::type writer(writer_sexp);
+	Rcpp::traits::input_parameter<const std::string&>::type description(description_sexp);
+	ipc___feather___TableWriter__SetDescription(writer, description);
+	return R_NilValue;
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_ipc___feather___TableWriter__SetDescription(SEXP writer_sexp, SEXP description_sexp){
+	Rf_error("Cannot call ipc___feather___TableWriter__SetDescription(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// feather.cpp:31
+#if defined(ARROW_R_WITH_ARROW)
+void ipc___feather___TableWriter__SetNumRows(const std::unique_ptr<arrow::ipc::feather::TableWriter>& writer, int64_t num_rows);
+RcppExport SEXP _arrow_ipc___feather___TableWriter__SetNumRows(SEXP writer_sexp, SEXP num_rows_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::unique_ptr<arrow::ipc::feather::TableWriter>&>::type writer(writer_sexp);
+	Rcpp::traits::input_parameter<int64_t>::type num_rows(num_rows_sexp);
+	ipc___feather___TableWriter__SetNumRows(writer, num_rows);
+	return R_NilValue;
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_ipc___feather___TableWriter__SetNumRows(SEXP writer_sexp, SEXP num_rows_sexp){
+	Rf_error("Cannot call ipc___feather___TableWriter__SetNumRows(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// feather.cpp:37
+#if defined(ARROW_R_WITH_ARROW)
+void ipc___feather___TableWriter__Append(const std::unique_ptr<arrow::ipc::feather::TableWriter>& writer, const std::string& name, const std::shared_ptr<arrow::Array>& values);
+RcppExport SEXP _arrow_ipc___feather___TableWriter__Append(SEXP writer_sexp, SEXP name_sexp, SEXP values_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::unique_ptr<arrow::ipc::feather::TableWriter>&>::type writer(writer_sexp);
+	Rcpp::traits::input_parameter<const std::string&>::type name(name_sexp);
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::Array>&>::type values(values_sexp);
+	ipc___feather___TableWriter__Append(writer, name, values);
+	return R_NilValue;
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_ipc___feather___TableWriter__Append(SEXP writer_sexp, SEXP name_sexp, SEXP values_sexp){
+	Rf_error("Cannot call ipc___feather___TableWriter__Append(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// feather.cpp:44
+#if defined(ARROW_R_WITH_ARROW)
+void ipc___feather___TableWriter__Finalize(const std::unique_ptr<arrow::ipc::feather::TableWriter>& writer);
+RcppExport SEXP _arrow_ipc___feather___TableWriter__Finalize(SEXP writer_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::unique_ptr<arrow::ipc::feather::TableWriter>&>::type writer(writer_sexp);
+	ipc___feather___TableWriter__Finalize(writer);
+	return R_NilValue;
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_ipc___feather___TableWriter__Finalize(SEXP writer_sexp){
+	Rf_error("Cannot call ipc___feather___TableWriter__Finalize(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// feather.cpp:50
+#if defined(ARROW_R_WITH_ARROW)
+std::unique_ptr<arrow::ipc::feather::TableWriter> ipc___feather___TableWriter__Open(const std::shared_ptr<arrow::io::OutputStream>& stream);
+RcppExport SEXP _arrow_ipc___feather___TableWriter__Open(SEXP stream_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::io::OutputStream>&>::type stream(stream_sexp);
+	return Rcpp::wrap(ipc___feather___TableWriter__Open(stream));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_ipc___feather___TableWriter__Open(SEXP stream_sexp){
+	Rf_error("Cannot call ipc___feather___TableWriter__Open(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// feather.cpp:58
+#if defined(ARROW_R_WITH_ARROW)
+void ipc___TableWriter__RecordBatch__WriteFeather(const std::unique_ptr<arrow::ipc::feather::TableWriter>& writer, const std::shared_ptr<arrow::RecordBatch>& batch);
+RcppExport SEXP _arrow_ipc___TableWriter__RecordBatch__WriteFeather(SEXP writer_sexp, SEXP batch_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::unique_ptr<arrow::ipc::feather::TableWriter>&>::type writer(writer_sexp);
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::RecordBatch>&>::type batch(batch_sexp);
+	ipc___TableWriter__RecordBatch__WriteFeather(writer, batch);
+	return R_NilValue;
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_ipc___TableWriter__RecordBatch__WriteFeather(SEXP writer_sexp, SEXP batch_sexp){
+	Rf_error("Cannot call ipc___TableWriter__RecordBatch__WriteFeather(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// feather.cpp:72
+#if defined(ARROW_R_WITH_ARROW)
+std::string ipc___feather___TableReader__GetDescription(const std::unique_ptr<arrow::ipc::feather::TableReader>& reader);
+RcppExport SEXP _arrow_ipc___feather___TableReader__GetDescription(SEXP reader_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::unique_ptr<arrow::ipc::feather::TableReader>&>::type reader(reader_sexp);
+	return Rcpp::wrap(ipc___feather___TableReader__GetDescription(reader));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_ipc___feather___TableReader__GetDescription(SEXP reader_sexp){
+	Rf_error("Cannot call ipc___feather___TableReader__GetDescription(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// feather.cpp:78
+#if defined(ARROW_R_WITH_ARROW)
+bool ipc___feather___TableReader__HasDescription(const std::unique_ptr<arrow::ipc::feather::TableReader>& reader);
+RcppExport SEXP _arrow_ipc___feather___TableReader__HasDescription(SEXP reader_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::unique_ptr<arrow::ipc::feather::TableReader>&>::type reader(reader_sexp);
+	return Rcpp::wrap(ipc___feather___TableReader__HasDescription(reader));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_ipc___feather___TableReader__HasDescription(SEXP reader_sexp){
+	Rf_error("Cannot call ipc___feather___TableReader__HasDescription(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// feather.cpp:84
+#if defined(ARROW_R_WITH_ARROW)
+int ipc___feather___TableReader__version(const std::unique_ptr<arrow::ipc::feather::TableReader>& reader);
+RcppExport SEXP _arrow_ipc___feather___TableReader__version(SEXP reader_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::unique_ptr<arrow::ipc::feather::TableReader>&>::type reader(reader_sexp);
+	return Rcpp::wrap(ipc___feather___TableReader__version(reader));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_ipc___feather___TableReader__version(SEXP reader_sexp){
+	Rf_error("Cannot call ipc___feather___TableReader__version(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// feather.cpp:90
+#if defined(ARROW_R_WITH_ARROW)
+int64_t ipc___feather___TableReader__num_rows(const std::unique_ptr<arrow::ipc::feather::TableReader>& reader);
+RcppExport SEXP _arrow_ipc___feather___TableReader__num_rows(SEXP reader_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::unique_ptr<arrow::ipc::feather::TableReader>&>::type reader(reader_sexp);
+	return Rcpp::wrap(ipc___feather___TableReader__num_rows(reader));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_ipc___feather___TableReader__num_rows(SEXP reader_sexp){
+	Rf_error("Cannot call ipc___feather___TableReader__num_rows(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// feather.cpp:96
+#if defined(ARROW_R_WITH_ARROW)
+int64_t ipc___feather___TableReader__num_columns(const std::unique_ptr<arrow::ipc::feather::TableReader>& reader);
+RcppExport SEXP _arrow_ipc___feather___TableReader__num_columns(SEXP reader_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::unique_ptr<arrow::ipc::feather::TableReader>&>::type reader(reader_sexp);
+	return Rcpp::wrap(ipc___feather___TableReader__num_columns(reader));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_ipc___feather___TableReader__num_columns(SEXP reader_sexp){
+	Rf_error("Cannot call ipc___feather___TableReader__num_columns(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// feather.cpp:102
+#if defined(ARROW_R_WITH_ARROW)
+std::string ipc___feather___TableReader__GetColumnName(const std::unique_ptr<arrow::ipc::feather::TableReader>& reader, int i);
+RcppExport SEXP _arrow_ipc___feather___TableReader__GetColumnName(SEXP reader_sexp, SEXP i_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::unique_ptr<arrow::ipc::feather::TableReader>&>::type reader(reader_sexp);
+	Rcpp::traits::input_parameter<int>::type i(i_sexp);
+	return Rcpp::wrap(ipc___feather___TableReader__GetColumnName(reader, i));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_ipc___feather___TableReader__GetColumnName(SEXP reader_sexp, SEXP i_sexp){
+	Rf_error("Cannot call ipc___feather___TableReader__GetColumnName(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// feather.cpp:108
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::Column> ipc___feather___TableReader__GetColumn(const std::unique_ptr<arrow::ipc::feather::TableReader>& reader, int i);
+RcppExport SEXP _arrow_ipc___feather___TableReader__GetColumn(SEXP reader_sexp, SEXP i_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::unique_ptr<arrow::ipc::feather::TableReader>&>::type reader(reader_sexp);
+	Rcpp::traits::input_parameter<int>::type i(i_sexp);
+	return Rcpp::wrap(ipc___feather___TableReader__GetColumn(reader, i));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_ipc___feather___TableReader__GetColumn(SEXP reader_sexp, SEXP i_sexp){
+	Rf_error("Cannot call ipc___feather___TableReader__GetColumn(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// feather.cpp:116
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::Table> ipc___feather___TableReader__Read(const std::unique_ptr<arrow::ipc::feather::TableReader>& reader, SEXP columns);
+RcppExport SEXP _arrow_ipc___feather___TableReader__Read(SEXP reader_sexp, SEXP columns_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::unique_ptr<arrow::ipc::feather::TableReader>&>::type reader(reader_sexp);
+	Rcpp::traits::input_parameter<SEXP>::type columns(columns_sexp);
+	return Rcpp::wrap(ipc___feather___TableReader__Read(reader, columns));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_ipc___feather___TableReader__Read(SEXP reader_sexp, SEXP columns_sexp){
+	Rf_error("Cannot call ipc___feather___TableReader__Read(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// feather.cpp:152
+#if defined(ARROW_R_WITH_ARROW)
+std::unique_ptr<arrow::ipc::feather::TableReader> ipc___feather___TableReader__Open(const std::shared_ptr<arrow::io::RandomAccessFile>& stream);
+RcppExport SEXP _arrow_ipc___feather___TableReader__Open(SEXP stream_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::io::RandomAccessFile>&>::type stream(stream_sexp);
+	return Rcpp::wrap(ipc___feather___TableReader__Open(stream));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_ipc___feather___TableReader__Open(SEXP stream_sexp){
+	Rf_error("Cannot call ipc___feather___TableReader__Open(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// field.cpp:22
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::Field> Field__initialize(const std::string& name, const std::shared_ptr<arrow::DataType>& field, bool nullable);
+RcppExport SEXP _arrow_Field__initialize(SEXP name_sexp, SEXP field_sexp, SEXP nullable_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::string&>::type name(name_sexp);
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::DataType>&>::type field(field_sexp);
+	Rcpp::traits::input_parameter<bool>::type nullable(nullable_sexp);
+	return Rcpp::wrap(Field__initialize(name, field, nullable));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_Field__initialize(SEXP name_sexp, SEXP field_sexp, SEXP nullable_sexp){
+	Rf_error("Cannot call Field__initialize(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// field.cpp:29
+#if defined(ARROW_R_WITH_ARROW)
+std::string Field__ToString(const std::shared_ptr<arrow::Field>& field);
+RcppExport SEXP _arrow_Field__ToString(SEXP field_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::Field>&>::type field(field_sexp);
+	return Rcpp::wrap(Field__ToString(field));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_Field__ToString(SEXP field_sexp){
+	Rf_error("Cannot call Field__ToString(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// field.cpp:34
+#if defined(ARROW_R_WITH_ARROW)
+std::string Field__name(const std::shared_ptr<arrow::Field>& field);
+RcppExport SEXP _arrow_Field__name(SEXP field_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::Field>&>::type field(field_sexp);
+	return Rcpp::wrap(Field__name(field));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_Field__name(SEXP field_sexp){
+	Rf_error("Cannot call Field__name(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// field.cpp:39
+#if defined(ARROW_R_WITH_ARROW)
+bool Field__Equals(const std::shared_ptr<arrow::Field>& field, const std::shared_ptr<arrow::Field>& other);
+RcppExport SEXP _arrow_Field__Equals(SEXP field_sexp, SEXP other_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::Field>&>::type field(field_sexp);
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::Field>&>::type other(other_sexp);
+	return Rcpp::wrap(Field__Equals(field, other));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_Field__Equals(SEXP field_sexp, SEXP other_sexp){
+	Rf_error("Cannot call Field__Equals(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// field.cpp:45
+#if defined(ARROW_R_WITH_ARROW)
+bool Field__nullable(const std::shared_ptr<arrow::Field>& field);
+RcppExport SEXP _arrow_Field__nullable(SEXP field_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::Field>&>::type field(field_sexp);
+	return Rcpp::wrap(Field__nullable(field));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_Field__nullable(SEXP field_sexp){
+	Rf_error("Cannot call Field__nullable(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// field.cpp:50
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::DataType> Field__type(const std::shared_ptr<arrow::Field>& field);
+RcppExport SEXP _arrow_Field__type(SEXP field_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::Field>&>::type field(field_sexp);
+	return Rcpp::wrap(Field__type(field));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_Field__type(SEXP field_sexp){
+	Rf_error("Cannot call Field__type(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// io.cpp:26
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::Buffer> io___Readable__Read(const std::shared_ptr<arrow::io::Readable>& x, int64_t nbytes);
+RcppExport SEXP _arrow_io___Readable__Read(SEXP x_sexp, SEXP nbytes_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::io::Readable>&>::type x(x_sexp);
+	Rcpp::traits::input_parameter<int64_t>::type nbytes(nbytes_sexp);
+	return Rcpp::wrap(io___Readable__Read(x, nbytes));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_io___Readable__Read(SEXP x_sexp, SEXP nbytes_sexp){
+	Rf_error("Cannot call io___Readable__Read(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// io.cpp:36
+#if defined(ARROW_R_WITH_ARROW)
+void io___InputStream__Close(const std::shared_ptr<arrow::io::InputStream>& x);
+RcppExport SEXP _arrow_io___InputStream__Close(SEXP x_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::io::InputStream>&>::type x(x_sexp);
+	io___InputStream__Close(x);
+	return R_NilValue;
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_io___InputStream__Close(SEXP x_sexp){
+	Rf_error("Cannot call io___InputStream__Close(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// io.cpp:43
+#if defined(ARROW_R_WITH_ARROW)
+void io___OutputStream__Close(const std::shared_ptr<arrow::io::OutputStream>& x);
+RcppExport SEXP _arrow_io___OutputStream__Close(SEXP x_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::io::OutputStream>&>::type x(x_sexp);
+	io___OutputStream__Close(x);
+	return R_NilValue;
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_io___OutputStream__Close(SEXP x_sexp){
+	Rf_error("Cannot call io___OutputStream__Close(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// io.cpp:50
+#if defined(ARROW_R_WITH_ARROW)
+int64_t io___RandomAccessFile__GetSize(const std::shared_ptr<arrow::io::RandomAccessFile>& x);
+RcppExport SEXP _arrow_io___RandomAccessFile__GetSize(SEXP x_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::io::RandomAccessFile>&>::type x(x_sexp);
+	return Rcpp::wrap(io___RandomAccessFile__GetSize(x));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_io___RandomAccessFile__GetSize(SEXP x_sexp){
+	Rf_error("Cannot call io___RandomAccessFile__GetSize(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// io.cpp:58
+#if defined(ARROW_R_WITH_ARROW)
+bool io___RandomAccessFile__supports_zero_copy(const std::shared_ptr<arrow::io::RandomAccessFile>& x);
+RcppExport SEXP _arrow_io___RandomAccessFile__supports_zero_copy(SEXP x_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::io::RandomAccessFile>&>::type x(x_sexp);
+	return Rcpp::wrap(io___RandomAccessFile__supports_zero_copy(x));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_io___RandomAccessFile__supports_zero_copy(SEXP x_sexp){
+	Rf_error("Cannot call io___RandomAccessFile__supports_zero_copy(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// io.cpp:64
+#if defined(ARROW_R_WITH_ARROW)
+void io___RandomAccessFile__Seek(const std::shared_ptr<arrow::io::RandomAccessFile>& x, int64_t position);
+RcppExport SEXP _arrow_io___RandomAccessFile__Seek(SEXP x_sexp, SEXP position_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::io::RandomAccessFile>&>::type x(x_sexp);
+	Rcpp::traits::input_parameter<int64_t>::type position(position_sexp);
+	io___RandomAccessFile__Seek(x, position);
+	return R_NilValue;
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_io___RandomAccessFile__Seek(SEXP x_sexp, SEXP position_sexp){
+	Rf_error("Cannot call io___RandomAccessFile__Seek(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// io.cpp:70
+#if defined(ARROW_R_WITH_ARROW)
+int64_t io___RandomAccessFile__Tell(const std::shared_ptr<arrow::io::RandomAccessFile>& x);
+RcppExport SEXP _arrow_io___RandomAccessFile__Tell(SEXP x_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::io::RandomAccessFile>&>::type x(x_sexp);
+	return Rcpp::wrap(io___RandomAccessFile__Tell(x));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_io___RandomAccessFile__Tell(SEXP x_sexp){
+	Rf_error("Cannot call io___RandomAccessFile__Tell(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// io.cpp:80
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::io::MemoryMappedFile> io___MemoryMappedFile__Create(const std::string& path, int64_t size);
+RcppExport SEXP _arrow_io___MemoryMappedFile__Create(SEXP path_sexp, SEXP size_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::string&>::type path(path_sexp);
+	Rcpp::traits::input_parameter<int64_t>::type size(size_sexp);
+	return Rcpp::wrap(io___MemoryMappedFile__Create(path, size));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_io___MemoryMappedFile__Create(SEXP path_sexp, SEXP size_sexp){
+	Rf_error("Cannot call io___MemoryMappedFile__Create(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// io.cpp:88
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::io::MemoryMappedFile> io___MemoryMappedFile__Open(const std::string& path, arrow::io::FileMode::type mode);
+RcppExport SEXP _arrow_io___MemoryMappedFile__Open(SEXP path_sexp, SEXP mode_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::string&>::type path(path_sexp);
+	Rcpp::traits::input_parameter<arrow::io::FileMode::type>::type mode(mode_sexp);
+	return Rcpp::wrap(io___MemoryMappedFile__Open(path, mode));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_io___MemoryMappedFile__Open(SEXP path_sexp, SEXP mode_sexp){
+	Rf_error("Cannot call io___MemoryMappedFile__Open(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// io.cpp:96
+#if defined(ARROW_R_WITH_ARROW)
+void io___MemoryMappedFile__Resize(const std::shared_ptr<arrow::io::MemoryMappedFile>& x, int64_t size);
+RcppExport SEXP _arrow_io___MemoryMappedFile__Resize(SEXP x_sexp, SEXP size_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::io::MemoryMappedFile>&>::type x(x_sexp);
+	Rcpp::traits::input_parameter<int64_t>::type size(size_sexp);
+	io___MemoryMappedFile__Resize(x, size);
+	return R_NilValue;
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_io___MemoryMappedFile__Resize(SEXP x_sexp, SEXP size_sexp){
+	Rf_error("Cannot call io___MemoryMappedFile__Resize(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// io.cpp:104
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::io::ReadableFile> io___ReadableFile__Open(const std::string& path);
+RcppExport SEXP _arrow_io___ReadableFile__Open(SEXP path_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::string&>::type path(path_sexp);
+	return Rcpp::wrap(io___ReadableFile__Open(path));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_io___ReadableFile__Open(SEXP path_sexp){
+	Rf_error("Cannot call io___ReadableFile__Open(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// io.cpp:114
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::io::BufferReader> io___BufferReader__initialize(const std::shared_ptr<arrow::Buffer>& buffer);
+RcppExport SEXP _arrow_io___BufferReader__initialize(SEXP buffer_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::Buffer>&>::type buffer(buffer_sexp);
+	return Rcpp::wrap(io___BufferReader__initialize(buffer));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_io___BufferReader__initialize(SEXP buffer_sexp){
+	Rf_error("Cannot call io___BufferReader__initialize(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// io.cpp:122
+#if defined(ARROW_R_WITH_ARROW)
+void io___Writable__write(const std::shared_ptr<arrow::io::Writable>& stream, const std::shared_ptr<arrow::Buffer>& buf);
+RcppExport SEXP _arrow_io___Writable__write(SEXP stream_sexp, SEXP buf_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::io::Writable>&>::type stream(stream_sexp);
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::Buffer>&>::type buf(buf_sexp);
+	io___Writable__write(stream, buf);
+	return R_NilValue;
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_io___Writable__write(SEXP stream_sexp, SEXP buf_sexp){
+	Rf_error("Cannot call io___Writable__write(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// io.cpp:130
+#if defined(ARROW_R_WITH_ARROW)
+int64_t io___OutputStream__Tell(const std::shared_ptr<arrow::io::OutputStream>& stream);
+RcppExport SEXP _arrow_io___OutputStream__Tell(SEXP stream_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::io::OutputStream>&>::type stream(stream_sexp);
+	return Rcpp::wrap(io___OutputStream__Tell(stream));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_io___OutputStream__Tell(SEXP stream_sexp){
+	Rf_error("Cannot call io___OutputStream__Tell(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// io.cpp:139
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::io::FileOutputStream> io___FileOutputStream__Open(const std::string& path);
+RcppExport SEXP _arrow_io___FileOutputStream__Open(SEXP path_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::string&>::type path(path_sexp);
+	return Rcpp::wrap(io___FileOutputStream__Open(path));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_io___FileOutputStream__Open(SEXP path_sexp){
+	Rf_error("Cannot call io___FileOutputStream__Open(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// io.cpp:149
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::io::BufferOutputStream> io___BufferOutputStream__Create(int64_t initial_capacity);
+RcppExport SEXP _arrow_io___BufferOutputStream__Create(SEXP initial_capacity_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<int64_t>::type initial_capacity(initial_capacity_sexp);
+	return Rcpp::wrap(io___BufferOutputStream__Create(initial_capacity));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_io___BufferOutputStream__Create(SEXP initial_capacity_sexp){
+	Rf_error("Cannot call io___BufferOutputStream__Create(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// io.cpp:158
+#if defined(ARROW_R_WITH_ARROW)
+int64_t io___BufferOutputStream__capacity(const std::shared_ptr<arrow::io::BufferOutputStream>& stream);
+RcppExport SEXP _arrow_io___BufferOutputStream__capacity(SEXP stream_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::io::BufferOutputStream>&>::type stream(stream_sexp);
+	return Rcpp::wrap(io___BufferOutputStream__capacity(stream));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_io___BufferOutputStream__capacity(SEXP stream_sexp){
+	Rf_error("Cannot call io___BufferOutputStream__capacity(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// io.cpp:164
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::Buffer> io___BufferOutputStream__Finish(const std::shared_ptr<arrow::io::BufferOutputStream>& stream);
+RcppExport SEXP _arrow_io___BufferOutputStream__Finish(SEXP stream_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::io::BufferOutputStream>&>::type stream(stream_sexp);
+	return Rcpp::wrap(io___BufferOutputStream__Finish(stream));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_io___BufferOutputStream__Finish(SEXP stream_sexp){
+	Rf_error("Cannot call io___BufferOutputStream__Finish(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// io.cpp:172
+#if defined(ARROW_R_WITH_ARROW)
+int64_t io___BufferOutputStream__Tell(const std::shared_ptr<arrow::io::BufferOutputStream>& stream);
+RcppExport SEXP _arrow_io___BufferOutputStream__Tell(SEXP stream_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::io::BufferOutputStream>&>::type stream(stream_sexp);
+	return Rcpp::wrap(io___BufferOutputStream__Tell(stream));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_io___BufferOutputStream__Tell(SEXP stream_sexp){
+	Rf_error("Cannot call io___BufferOutputStream__Tell(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// io.cpp:180
+#if defined(ARROW_R_WITH_ARROW)
+void io___BufferOutputStream__Write(const std::shared_ptr<arrow::io::BufferOutputStream>& stream, RawVector_ bytes);
+RcppExport SEXP _arrow_io___BufferOutputStream__Write(SEXP stream_sexp, SEXP bytes_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::io::BufferOutputStream>&>::type stream(stream_sexp);
+	Rcpp::traits::input_parameter<RawVector_>::type bytes(bytes_sexp);
+	io___BufferOutputStream__Write(stream, bytes);
+	return R_NilValue;
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_io___BufferOutputStream__Write(SEXP stream_sexp, SEXP bytes_sexp){
+	Rf_error("Cannot call io___BufferOutputStream__Write(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// io.cpp:188
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::io::MockOutputStream> io___MockOutputStream__initialize();
+RcppExport SEXP _arrow_io___MockOutputStream__initialize(){
+BEGIN_RCPP
+	return Rcpp::wrap(io___MockOutputStream__initialize());
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_io___MockOutputStream__initialize(){
+	Rf_error("Cannot call io___MockOutputStream__initialize(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// io.cpp:193
+#if defined(ARROW_R_WITH_ARROW)
+int64_t io___MockOutputStream__GetExtentBytesWritten(const std::shared_ptr<arrow::io::MockOutputStream>& stream);
+RcppExport SEXP _arrow_io___MockOutputStream__GetExtentBytesWritten(SEXP stream_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::io::MockOutputStream>&>::type stream(stream_sexp);
+	return Rcpp::wrap(io___MockOutputStream__GetExtentBytesWritten(stream));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_io___MockOutputStream__GetExtentBytesWritten(SEXP stream_sexp){
+	Rf_error("Cannot call io___MockOutputStream__GetExtentBytesWritten(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// io.cpp:201
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::io::FixedSizeBufferWriter> io___FixedSizeBufferWriter__initialize(const std::shared_ptr<arrow::Buffer>& buffer);
+RcppExport SEXP _arrow_io___FixedSizeBufferWriter__initialize(SEXP buffer_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::Buffer>&>::type buffer(buffer_sexp);
+	return Rcpp::wrap(io___FixedSizeBufferWriter__initialize(buffer));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_io___FixedSizeBufferWriter__initialize(SEXP buffer_sexp){
+	Rf_error("Cannot call io___FixedSizeBufferWriter__initialize(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// memorypool.cpp:21
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::MemoryPool> MemoryPool__default();
+RcppExport SEXP _arrow_MemoryPool__default(){
+BEGIN_RCPP
+	return Rcpp::wrap(MemoryPool__default());
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_MemoryPool__default(){
+	Rf_error("Cannot call MemoryPool__default(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// memorypool.cpp:27
+#if defined(ARROW_R_WITH_ARROW)
+int MemoryPool__bytes_allocated(const std::shared_ptr<arrow::MemoryPool>& pool);
+RcppExport SEXP _arrow_MemoryPool__bytes_allocated(SEXP pool_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::MemoryPool>&>::type pool(pool_sexp);
+	return Rcpp::wrap(MemoryPool__bytes_allocated(pool));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_MemoryPool__bytes_allocated(SEXP pool_sexp){
+	Rf_error("Cannot call MemoryPool__bytes_allocated(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// memorypool.cpp:32
+#if defined(ARROW_R_WITH_ARROW)
+int MemoryPool__max_memory(const std::shared_ptr<arrow::MemoryPool>& pool);
+RcppExport SEXP _arrow_MemoryPool__max_memory(SEXP pool_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::MemoryPool>&>::type pool(pool_sexp);
+	return Rcpp::wrap(MemoryPool__max_memory(pool));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_MemoryPool__max_memory(SEXP pool_sexp){
+	Rf_error("Cannot call MemoryPool__max_memory(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// message.cpp:22
+#if defined(ARROW_R_WITH_ARROW)
+int64_t ipc___Message__body_length(const std::unique_ptr<arrow::ipc::Message>& message);
+RcppExport SEXP _arrow_ipc___Message__body_length(SEXP message_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::unique_ptr<arrow::ipc::Message>&>::type message(message_sexp);
+	return Rcpp::wrap(ipc___Message__body_length(message));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_ipc___Message__body_length(SEXP message_sexp){
+	Rf_error("Cannot call ipc___Message__body_length(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// message.cpp:27
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::Buffer> ipc___Message__metadata(const std::unique_ptr<arrow::ipc::Message>& message);
+RcppExport SEXP _arrow_ipc___Message__metadata(SEXP message_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::unique_ptr<arrow::ipc::Message>&>::type message(message_sexp);
+	return Rcpp::wrap(ipc___Message__metadata(message));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_ipc___Message__metadata(SEXP message_sexp){
+	Rf_error("Cannot call ipc___Message__metadata(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// message.cpp:33
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::Buffer> ipc___Message__body(const std::unique_ptr<arrow::ipc::Message>& message);
+RcppExport SEXP _arrow_ipc___Message__body(SEXP message_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::unique_ptr<arrow::ipc::Message>&>::type message(message_sexp);
+	return Rcpp::wrap(ipc___Message__body(message));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_ipc___Message__body(SEXP message_sexp){
+	Rf_error("Cannot call ipc___Message__body(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// message.cpp:39
+#if defined(ARROW_R_WITH_ARROW)
+int64_t ipc___Message__Verify(const std::unique_ptr<arrow::ipc::Message>& message);
+RcppExport SEXP _arrow_ipc___Message__Verify(SEXP message_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::unique_ptr<arrow::ipc::Message>&>::type message(message_sexp);
+	return Rcpp::wrap(ipc___Message__Verify(message));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_ipc___Message__Verify(SEXP message_sexp){
+	Rf_error("Cannot call ipc___Message__Verify(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// message.cpp:44
+#if defined(ARROW_R_WITH_ARROW)
+arrow::ipc::Message::Type ipc___Message__type(const std::unique_ptr<arrow::ipc::Message>& message);
+RcppExport SEXP _arrow_ipc___Message__type(SEXP message_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::unique_ptr<arrow::ipc::Message>&>::type message(message_sexp);
+	return Rcpp::wrap(ipc___Message__type(message));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_ipc___Message__type(SEXP message_sexp){
+	Rf_error("Cannot call ipc___Message__type(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// message.cpp:50
+#if defined(ARROW_R_WITH_ARROW)
+bool ipc___Message__Equals(const std::unique_ptr<arrow::ipc::Message>& x, const std::unique_ptr<arrow::ipc::Message>& y);
+RcppExport SEXP _arrow_ipc___Message__Equals(SEXP x_sexp, SEXP y_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::unique_ptr<arrow::ipc::Message>&>::type x(x_sexp);
+	Rcpp::traits::input_parameter<const std::unique_ptr<arrow::ipc::Message>&>::type y(y_sexp);
+	return Rcpp::wrap(ipc___Message__Equals(x, y));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_ipc___Message__Equals(SEXP x_sexp, SEXP y_sexp){
+	Rf_error("Cannot call ipc___Message__Equals(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// message.cpp:56
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::RecordBatch> ipc___ReadRecordBatch__Message__Schema(const std::unique_ptr<arrow::ipc::Message>& message, const std::shared_ptr<arrow::Schema>& schema);
+RcppExport SEXP _arrow_ipc___ReadRecordBatch__Message__Schema(SEXP message_sexp, SEXP schema_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::unique_ptr<arrow::ipc::Message>&>::type message(message_sexp);
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::Schema>&>::type schema(schema_sexp);
+	return Rcpp::wrap(ipc___ReadRecordBatch__Message__Schema(message, schema));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_ipc___ReadRecordBatch__Message__Schema(SEXP message_sexp, SEXP schema_sexp){
+	Rf_error("Cannot call ipc___ReadRecordBatch__Message__Schema(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// message.cpp:68
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::Schema> ipc___ReadSchema_InputStream(const std::shared_ptr<arrow::io::InputStream>& stream);
+RcppExport SEXP _arrow_ipc___ReadSchema_InputStream(SEXP stream_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::io::InputStream>&>::type stream(stream_sexp);
+	return Rcpp::wrap(ipc___ReadSchema_InputStream(stream));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_ipc___ReadSchema_InputStream(SEXP stream_sexp){
+	Rf_error("Cannot call ipc___ReadSchema_InputStream(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// message.cpp:80
+#if defined(ARROW_R_WITH_ARROW)
+std::unique_ptr<arrow::ipc::MessageReader> ipc___MessageReader__Open(const std::shared_ptr<arrow::io::InputStream>& stream);
+RcppExport SEXP _arrow_ipc___MessageReader__Open(SEXP stream_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::io::InputStream>&>::type stream(stream_sexp);
+	return Rcpp::wrap(ipc___MessageReader__Open(stream));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_ipc___MessageReader__Open(SEXP stream_sexp){
+	Rf_error("Cannot call ipc___MessageReader__Open(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// message.cpp:86
+#if defined(ARROW_R_WITH_ARROW)
+std::unique_ptr<arrow::ipc::Message> ipc___MessageReader__ReadNextMessage(const std::unique_ptr<arrow::ipc::MessageReader>& reader);
+RcppExport SEXP _arrow_ipc___MessageReader__ReadNextMessage(SEXP reader_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::unique_ptr<arrow::ipc::MessageReader>&>::type reader(reader_sexp);
+	return Rcpp::wrap(ipc___MessageReader__ReadNextMessage(reader));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_ipc___MessageReader__ReadNextMessage(SEXP reader_sexp){
+	Rf_error("Cannot call ipc___MessageReader__ReadNextMessage(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// message.cpp:94
+#if defined(ARROW_R_WITH_ARROW)
+std::unique_ptr<arrow::ipc::Message> ipc___ReadMessage(const std::shared_ptr<arrow::io::InputStream>& stream);
+RcppExport SEXP _arrow_ipc___ReadMessage(SEXP stream_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::io::InputStream>&>::type stream(stream_sexp);
+	return Rcpp::wrap(ipc___ReadMessage(stream));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_ipc___ReadMessage(SEXP stream_sexp){
+	Rf_error("Cannot call ipc___ReadMessage(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// parquet.cpp:28
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::Table> read_parquet_file(std::string filename);
+RcppExport SEXP _arrow_read_parquet_file(SEXP filename_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<std::string>::type filename(filename_sexp);
+	return Rcpp::wrap(read_parquet_file(filename));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_read_parquet_file(SEXP filename_sexp){
+	Rf_error("Cannot call read_parquet_file(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// recordbatch.cpp:26
+#if defined(ARROW_R_WITH_ARROW)
+int RecordBatch__num_columns(const std::shared_ptr<arrow::RecordBatch>& x);
+RcppExport SEXP _arrow_RecordBatch__num_columns(SEXP x_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::RecordBatch>&>::type x(x_sexp);
+	return Rcpp::wrap(RecordBatch__num_columns(x));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_RecordBatch__num_columns(SEXP x_sexp){
+	Rf_error("Cannot call RecordBatch__num_columns(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// recordbatch.cpp:31
+#if defined(ARROW_R_WITH_ARROW)
+int RecordBatch__num_rows(const std::shared_ptr<arrow::RecordBatch>& x);
+RcppExport SEXP _arrow_RecordBatch__num_rows(SEXP x_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::RecordBatch>&>::type x(x_sexp);
+	return Rcpp::wrap(RecordBatch__num_rows(x));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_RecordBatch__num_rows(SEXP x_sexp){
+	Rf_error("Cannot call RecordBatch__num_rows(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// recordbatch.cpp:36
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::Schema> RecordBatch__schema(const std::shared_ptr<arrow::RecordBatch>& x);
+RcppExport SEXP _arrow_RecordBatch__schema(SEXP x_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::RecordBatch>&>::type x(x_sexp);
+	return Rcpp::wrap(RecordBatch__schema(x));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_RecordBatch__schema(SEXP x_sexp){
+	Rf_error("Cannot call RecordBatch__schema(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// recordbatch.cpp:42
+#if defined(ARROW_R_WITH_ARROW)
+arrow::ArrayVector RecordBatch__columns(const std::shared_ptr<arrow::RecordBatch>& batch);
+RcppExport SEXP _arrow_RecordBatch__columns(SEXP batch_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::RecordBatch>&>::type batch(batch_sexp);
+	return Rcpp::wrap(RecordBatch__columns(batch));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_RecordBatch__columns(SEXP batch_sexp){
+	Rf_error("Cannot call RecordBatch__columns(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// recordbatch.cpp:53
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::Array> RecordBatch__column(const std::shared_ptr<arrow::RecordBatch>& batch, int i);
+RcppExport SEXP _arrow_RecordBatch__column(SEXP batch_sexp, SEXP i_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::RecordBatch>&>::type batch(batch_sexp);
+	Rcpp::traits::input_parameter<int>::type i(i_sexp);
+	return Rcpp::wrap(RecordBatch__column(batch, i));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_RecordBatch__column(SEXP batch_sexp, SEXP i_sexp){
+	Rf_error("Cannot call RecordBatch__column(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// recordbatch.cpp:59
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::RecordBatch> RecordBatch__from_dataframe(Rcpp::DataFrame tbl);
+RcppExport SEXP _arrow_RecordBatch__from_dataframe(SEXP tbl_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<Rcpp::DataFrame>::type tbl(tbl_sexp);
+	return Rcpp::wrap(RecordBatch__from_dataframe(tbl));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_RecordBatch__from_dataframe(SEXP tbl_sexp){
+	Rf_error("Cannot call RecordBatch__from_dataframe(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// recordbatch.cpp:77
+#if defined(ARROW_R_WITH_ARROW)
+bool RecordBatch__Equals(const std::shared_ptr<arrow::RecordBatch>& self, const std::shared_ptr<arrow::RecordBatch>& other);
+RcppExport SEXP _arrow_RecordBatch__Equals(SEXP self_sexp, SEXP other_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::RecordBatch>&>::type self(self_sexp);
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::RecordBatch>&>::type other(other_sexp);
+	return Rcpp::wrap(RecordBatch__Equals(self, other));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_RecordBatch__Equals(SEXP self_sexp, SEXP other_sexp){
+	Rf_error("Cannot call RecordBatch__Equals(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// recordbatch.cpp:83
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::RecordBatch> RecordBatch__RemoveColumn(const std::shared_ptr<arrow::RecordBatch>& batch, int i);
+RcppExport SEXP _arrow_RecordBatch__RemoveColumn(SEXP batch_sexp, SEXP i_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::RecordBatch>&>::type batch(batch_sexp);
+	Rcpp::traits::input_parameter<int>::type i(i_sexp);
+	return Rcpp::wrap(RecordBatch__RemoveColumn(batch, i));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_RecordBatch__RemoveColumn(SEXP batch_sexp, SEXP i_sexp){
+	Rf_error("Cannot call RecordBatch__RemoveColumn(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// recordbatch.cpp:91
+#if defined(ARROW_R_WITH_ARROW)
+std::string RecordBatch__column_name(const std::shared_ptr<arrow::RecordBatch>& batch, int i);
+RcppExport SEXP _arrow_RecordBatch__column_name(SEXP batch_sexp, SEXP i_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::RecordBatch>&>::type batch(batch_sexp);
+	Rcpp::traits::input_parameter<int>::type i(i_sexp);
+	return Rcpp::wrap(RecordBatch__column_name(batch, i));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_RecordBatch__column_name(SEXP batch_sexp, SEXP i_sexp){
+	Rf_error("Cannot call RecordBatch__column_name(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// recordbatch.cpp:97
+#if defined(ARROW_R_WITH_ARROW)
+Rcpp::CharacterVector RecordBatch__names(const std::shared_ptr<arrow::RecordBatch>& batch);
+RcppExport SEXP _arrow_RecordBatch__names(SEXP batch_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::RecordBatch>&>::type batch(batch_sexp);
+	return Rcpp::wrap(RecordBatch__names(batch));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_RecordBatch__names(SEXP batch_sexp){
+	Rf_error("Cannot call RecordBatch__names(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// recordbatch.cpp:108
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::RecordBatch> RecordBatch__Slice1(const std::shared_ptr<arrow::RecordBatch>& self, int offset);
+RcppExport SEXP _arrow_RecordBatch__Slice1(SEXP self_sexp, SEXP offset_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::RecordBatch>&>::type self(self_sexp);
+	Rcpp::traits::input_parameter<int>::type offset(offset_sexp);
+	return Rcpp::wrap(RecordBatch__Slice1(self, offset));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_RecordBatch__Slice1(SEXP self_sexp, SEXP offset_sexp){
+	Rf_error("Cannot call RecordBatch__Slice1(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// recordbatch.cpp:114
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::RecordBatch> RecordBatch__Slice2(const std::shared_ptr<arrow::RecordBatch>& self, int offset, int length);
+RcppExport SEXP _arrow_RecordBatch__Slice2(SEXP self_sexp, SEXP offset_sexp, SEXP length_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::RecordBatch>&>::type self(self_sexp);
+	Rcpp::traits::input_parameter<int>::type offset(offset_sexp);
+	Rcpp::traits::input_parameter<int>::type length(length_sexp);
+	return Rcpp::wrap(RecordBatch__Slice2(self, offset, length));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_RecordBatch__Slice2(SEXP self_sexp, SEXP offset_sexp, SEXP length_sexp){
+	Rf_error("Cannot call RecordBatch__Slice2(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// recordbatch.cpp:120
+#if defined(ARROW_R_WITH_ARROW)
+Rcpp::RawVector ipc___SerializeRecordBatch__Raw(const std::shared_ptr<arrow::RecordBatch>& batch);
+RcppExport SEXP _arrow_ipc___SerializeRecordBatch__Raw(SEXP batch_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::RecordBatch>&>::type batch(batch_sexp);
+	return Rcpp::wrap(ipc___SerializeRecordBatch__Raw(batch));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_ipc___SerializeRecordBatch__Raw(SEXP batch_sexp){
+	Rf_error("Cannot call ipc___SerializeRecordBatch__Raw(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// recordbatch.cpp:140
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::RecordBatch> ipc___ReadRecordBatch__InputStream__Schema(const std::shared_ptr<arrow::io::InputStream>& stream, const std::shared_ptr<arrow::Schema>& schema);
+RcppExport SEXP _arrow_ipc___ReadRecordBatch__InputStream__Schema(SEXP stream_sexp, SEXP schema_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::io::InputStream>&>::type stream(stream_sexp);
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::Schema>&>::type schema(schema_sexp);
+	return Rcpp::wrap(ipc___ReadRecordBatch__InputStream__Schema(stream, schema));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_ipc___ReadRecordBatch__InputStream__Schema(SEXP stream_sexp, SEXP schema_sexp){
+	Rf_error("Cannot call ipc___ReadRecordBatch__InputStream__Schema(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// recordbatch.cpp:189
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::RecordBatch> RecordBatch__from_arrays(SEXP schema_sxp, SEXP lst);
+RcppExport SEXP _arrow_RecordBatch__from_arrays(SEXP schema_sxp_sexp, SEXP lst_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<SEXP>::type schema_sxp(schema_sxp_sexp);
+	Rcpp::traits::input_parameter<SEXP>::type lst(lst_sexp);
+	return Rcpp::wrap(RecordBatch__from_arrays(schema_sxp, lst));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_RecordBatch__from_arrays(SEXP schema_sxp_sexp, SEXP lst_sexp){
+	Rf_error("Cannot call RecordBatch__from_arrays(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// recordbatchreader.cpp:22
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::Schema> RecordBatchReader__schema(const std::shared_ptr<arrow::RecordBatchReader>& reader);
+RcppExport SEXP _arrow_RecordBatchReader__schema(SEXP reader_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::RecordBatchReader>&>::type reader(reader_sexp);
+	return Rcpp::wrap(RecordBatchReader__schema(reader));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_RecordBatchReader__schema(SEXP reader_sexp){
+	Rf_error("Cannot call RecordBatchReader__schema(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// recordbatchreader.cpp:28
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::RecordBatch> RecordBatchReader__ReadNext(const std::shared_ptr<arrow::RecordBatchReader>& reader);
+RcppExport SEXP _arrow_RecordBatchReader__ReadNext(SEXP reader_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::RecordBatchReader>&>::type reader(reader_sexp);
+	return Rcpp::wrap(RecordBatchReader__ReadNext(reader));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_RecordBatchReader__ReadNext(SEXP reader_sexp){
+	Rf_error("Cannot call RecordBatchReader__ReadNext(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// recordbatchreader.cpp:38
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::RecordBatchReader> ipc___RecordBatchStreamReader__Open(const std::shared_ptr<arrow::io::InputStream>& stream);
+RcppExport SEXP _arrow_ipc___RecordBatchStreamReader__Open(SEXP stream_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::io::InputStream>&>::type stream(stream_sexp);
+	return Rcpp::wrap(ipc___RecordBatchStreamReader__Open(stream));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_ipc___RecordBatchStreamReader__Open(SEXP stream_sexp){
+	Rf_error("Cannot call ipc___RecordBatchStreamReader__Open(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// recordbatchreader.cpp:46
+#if defined(ARROW_R_WITH_ARROW)
+std::vector<std::shared_ptr<arrow::RecordBatch>> ipc___RecordBatchStreamReader__batches(const std::shared_ptr<arrow::ipc::RecordBatchStreamReader>& reader);
+RcppExport SEXP _arrow_ipc___RecordBatchStreamReader__batches(SEXP reader_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::ipc::RecordBatchStreamReader>&>::type reader(reader_sexp);
+	return Rcpp::wrap(ipc___RecordBatchStreamReader__batches(reader));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_ipc___RecordBatchStreamReader__batches(SEXP reader_sexp){
+	Rf_error("Cannot call ipc___RecordBatchStreamReader__batches(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// recordbatchreader.cpp:64
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::Schema> ipc___RecordBatchFileReader__schema(const std::shared_ptr<arrow::ipc::RecordBatchFileReader>& reader);
+RcppExport SEXP _arrow_ipc___RecordBatchFileReader__schema(SEXP reader_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::ipc::RecordBatchFileReader>&>::type reader(reader_sexp);
+	return Rcpp::wrap(ipc___RecordBatchFileReader__schema(reader));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_ipc___RecordBatchFileReader__schema(SEXP reader_sexp){
+	Rf_error("Cannot call ipc___RecordBatchFileReader__schema(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// recordbatchreader.cpp:70
+#if defined(ARROW_R_WITH_ARROW)
+int ipc___RecordBatchFileReader__num_record_batches(const std::shared_ptr<arrow::ipc::RecordBatchFileReader>& reader);
+RcppExport SEXP _arrow_ipc___RecordBatchFileReader__num_record_batches(SEXP reader_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::ipc::RecordBatchFileReader>&>::type reader(reader_sexp);
+	return Rcpp::wrap(ipc___RecordBatchFileReader__num_record_batches(reader));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_ipc___RecordBatchFileReader__num_record_batches(SEXP reader_sexp){
+	Rf_error("Cannot call ipc___RecordBatchFileReader__num_record_batches(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// recordbatchreader.cpp:76
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::RecordBatch> ipc___RecordBatchFileReader__ReadRecordBatch(const std::shared_ptr<arrow::ipc::RecordBatchFileReader>& reader, int i);
+RcppExport SEXP _arrow_ipc___RecordBatchFileReader__ReadRecordBatch(SEXP reader_sexp, SEXP i_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::ipc::RecordBatchFileReader>&>::type reader(reader_sexp);
+	Rcpp::traits::input_parameter<int>::type i(i_sexp);
+	return Rcpp::wrap(ipc___RecordBatchFileReader__ReadRecordBatch(reader, i));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_ipc___RecordBatchFileReader__ReadRecordBatch(SEXP reader_sexp, SEXP i_sexp){
+	Rf_error("Cannot call ipc___RecordBatchFileReader__ReadRecordBatch(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// recordbatchreader.cpp:86
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::ipc::RecordBatchFileReader> ipc___RecordBatchFileReader__Open(const std::shared_ptr<arrow::io::RandomAccessFile>& file);
+RcppExport SEXP _arrow_ipc___RecordBatchFileReader__Open(SEXP file_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::io::RandomAccessFile>&>::type file(file_sexp);
+	return Rcpp::wrap(ipc___RecordBatchFileReader__Open(file));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_ipc___RecordBatchFileReader__Open(SEXP file_sexp){
+	Rf_error("Cannot call ipc___RecordBatchFileReader__Open(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// recordbatchreader.cpp:94
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::Table> Table__from_RecordBatchFileReader(const std::shared_ptr<arrow::ipc::RecordBatchFileReader>& reader);
+RcppExport SEXP _arrow_Table__from_RecordBatchFileReader(SEXP reader_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::ipc::RecordBatchFileReader>&>::type reader(reader_sexp);
+	return Rcpp::wrap(Table__from_RecordBatchFileReader(reader));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_Table__from_RecordBatchFileReader(SEXP reader_sexp){
+	Rf_error("Cannot call Table__from_RecordBatchFileReader(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// recordbatchreader.cpp:109
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::Table> Table__from_RecordBatchStreamReader(const std::shared_ptr<arrow::ipc::RecordBatchStreamReader>& reader);
+RcppExport SEXP _arrow_Table__from_RecordBatchStreamReader(SEXP reader_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::ipc::RecordBatchStreamReader>&>::type reader(reader_sexp);
+	return Rcpp::wrap(Table__from_RecordBatchStreamReader(reader));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_Table__from_RecordBatchStreamReader(SEXP reader_sexp){
+	Rf_error("Cannot call Table__from_RecordBatchStreamReader(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// recordbatchreader.cpp:126
+#if defined(ARROW_R_WITH_ARROW)
+std::vector<std::shared_ptr<arrow::RecordBatch>> ipc___RecordBatchFileReader__batches(const std::shared_ptr<arrow::ipc::RecordBatchFileReader>& reader);
+RcppExport SEXP _arrow_ipc___RecordBatchFileReader__batches(SEXP reader_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::ipc::RecordBatchFileReader>&>::type reader(reader_sexp);
+	return Rcpp::wrap(ipc___RecordBatchFileReader__batches(reader));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_ipc___RecordBatchFileReader__batches(SEXP reader_sexp){
+	Rf_error("Cannot call ipc___RecordBatchFileReader__batches(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// recordbatchwriter.cpp:22
+#if defined(ARROW_R_WITH_ARROW)
+void ipc___RecordBatchWriter__WriteRecordBatch(const std::shared_ptr<arrow::ipc::RecordBatchWriter>& batch_writer, const std::shared_ptr<arrow::RecordBatch>& batch);
+RcppExport SEXP _arrow_ipc___RecordBatchWriter__WriteRecordBatch(SEXP batch_writer_sexp, SEXP batch_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::ipc::RecordBatchWriter>&>::type batch_writer(batch_writer_sexp);
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::RecordBatch>&>::type batch(batch_sexp);
+	ipc___RecordBatchWriter__WriteRecordBatch(batch_writer, batch);
+	return R_NilValue;
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_ipc___RecordBatchWriter__WriteRecordBatch(SEXP batch_writer_sexp, SEXP batch_sexp){
+	Rf_error("Cannot call ipc___RecordBatchWriter__WriteRecordBatch(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// recordbatchwriter.cpp:29
+#if defined(ARROW_R_WITH_ARROW)
+void ipc___RecordBatchWriter__WriteTable(const std::shared_ptr<arrow::ipc::RecordBatchWriter>& batch_writer, const std::shared_ptr<arrow::Table>& table);
+RcppExport SEXP _arrow_ipc___RecordBatchWriter__WriteTable(SEXP batch_writer_sexp, SEXP table_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::ipc::RecordBatchWriter>&>::type batch_writer(batch_writer_sexp);
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::Table>&>::type table(table_sexp);
+	ipc___RecordBatchWriter__WriteTable(batch_writer, table);
+	return R_NilValue;
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_ipc___RecordBatchWriter__WriteTable(SEXP batch_writer_sexp, SEXP table_sexp){
+	Rf_error("Cannot call ipc___RecordBatchWriter__WriteTable(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// recordbatchwriter.cpp:36
+#if defined(ARROW_R_WITH_ARROW)
+void ipc___RecordBatchWriter__Close(const std::shared_ptr<arrow::ipc::RecordBatchWriter>& batch_writer);
+RcppExport SEXP _arrow_ipc___RecordBatchWriter__Close(SEXP batch_writer_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::ipc::RecordBatchWriter>&>::type batch_writer(batch_writer_sexp);
+	ipc___RecordBatchWriter__Close(batch_writer);
+	return R_NilValue;
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_ipc___RecordBatchWriter__Close(SEXP batch_writer_sexp){
+	Rf_error("Cannot call ipc___RecordBatchWriter__Close(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// recordbatchwriter.cpp:42
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::ipc::RecordBatchWriter> ipc___RecordBatchFileWriter__Open(const std::shared_ptr<arrow::io::OutputStream>& stream, const std::shared_ptr<arrow::Schema>& schema);
+RcppExport SEXP _arrow_ipc___RecordBatchFileWriter__Open(SEXP stream_sexp, SEXP schema_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::io::OutputStream>&>::type stream(stream_sexp);
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::Schema>&>::type schema(schema_sexp);
+	return Rcpp::wrap(ipc___RecordBatchFileWriter__Open(stream, schema));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_ipc___RecordBatchFileWriter__Open(SEXP stream_sexp, SEXP schema_sexp){
+	Rf_error("Cannot call ipc___RecordBatchFileWriter__Open(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// recordbatchwriter.cpp:52
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::ipc::RecordBatchWriter> ipc___RecordBatchStreamWriter__Open(const std::shared_ptr<arrow::io::OutputStream>& stream, const std::shared_ptr<arrow::Schema>& schema);
+RcppExport SEXP _arrow_ipc___RecordBatchStreamWriter__Open(SEXP stream_sexp, SEXP schema_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::io::OutputStream>&>::type stream(stream_sexp);
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::Schema>&>::type schema(schema_sexp);
+	return Rcpp::wrap(ipc___RecordBatchStreamWriter__Open(stream, schema));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_ipc___RecordBatchStreamWriter__Open(SEXP stream_sexp, SEXP schema_sexp){
+	Rf_error("Cannot call ipc___RecordBatchStreamWriter__Open(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// table.cpp:27
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::Table> Table__from_dataframe(DataFrame tbl);
+RcppExport SEXP _arrow_Table__from_dataframe(SEXP tbl_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<DataFrame>::type tbl(tbl_sexp);
+	return Rcpp::wrap(Table__from_dataframe(tbl));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_Table__from_dataframe(SEXP tbl_sexp){
+	Rf_error("Cannot call Table__from_dataframe(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// table.cpp:36
+#if defined(ARROW_R_WITH_ARROW)
+int Table__num_columns(const std::shared_ptr<arrow::Table>& x);
+RcppExport SEXP _arrow_Table__num_columns(SEXP x_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::Table>&>::type x(x_sexp);
+	return Rcpp::wrap(Table__num_columns(x));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_Table__num_columns(SEXP x_sexp){
+	Rf_error("Cannot call Table__num_columns(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// table.cpp:41
+#if defined(ARROW_R_WITH_ARROW)
+int Table__num_rows(const std::shared_ptr<arrow::Table>& x);
+RcppExport SEXP _arrow_Table__num_rows(SEXP x_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::Table>&>::type x(x_sexp);
+	return Rcpp::wrap(Table__num_rows(x));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_Table__num_rows(SEXP x_sexp){
+	Rf_error("Cannot call Table__num_rows(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// table.cpp:44
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::Schema> Table__schema(const std::shared_ptr<arrow::Table>& x);
+RcppExport SEXP _arrow_Table__schema(SEXP x_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::Table>&>::type x(x_sexp);
+	return Rcpp::wrap(Table__schema(x));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_Table__schema(SEXP x_sexp){
+	Rf_error("Cannot call Table__schema(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// table.cpp:49
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::Column> Table__column(const std::shared_ptr<arrow::Table>& table, int i);
+RcppExport SEXP _arrow_Table__column(SEXP table_sexp, SEXP i_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::Table>&>::type table(table_sexp);
+	Rcpp::traits::input_parameter<int>::type i(i_sexp);
+	return Rcpp::wrap(Table__column(table, i));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_Table__column(SEXP table_sexp, SEXP i_sexp){
+	Rf_error("Cannot call Table__column(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// table.cpp:55
+#if defined(ARROW_R_WITH_ARROW)
+std::vector<std::shared_ptr<arrow::Column>> Table__columns(const std::shared_ptr<arrow::Table>& table);
+RcppExport SEXP _arrow_Table__columns(SEXP table_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::Table>&>::type table(table_sexp);
+	return Rcpp::wrap(Table__columns(table));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_Table__columns(SEXP table_sexp){
+	Rf_error("Cannot call Table__columns(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// table.cpp:74
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::Table> Table__from_dots(SEXP lst, SEXP schema_sxp);
+RcppExport SEXP _arrow_Table__from_dots(SEXP lst_sexp, SEXP schema_sxp_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<SEXP>::type lst(lst_sexp);
+	Rcpp::traits::input_parameter<SEXP>::type schema_sxp(schema_sxp_sexp);
+	return Rcpp::wrap(Table__from_dots(lst, schema_sxp));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_Table__from_dots(SEXP lst_sexp, SEXP schema_sxp_sexp){
+	Rf_error("Cannot call Table__from_dots(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// threadpool.cpp:41
+#if defined(ARROW_R_WITH_ARROW)
+int GetCpuThreadPoolCapacity();
+RcppExport SEXP _arrow_GetCpuThreadPoolCapacity(){
+BEGIN_RCPP
+	return Rcpp::wrap(GetCpuThreadPoolCapacity());
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_GetCpuThreadPoolCapacity(){
+	Rf_error("Cannot call GetCpuThreadPoolCapacity(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// threadpool.cpp:46
+#if defined(ARROW_R_WITH_ARROW)
+void SetCpuThreadPoolCapacity(int threads);
+RcppExport SEXP _arrow_SetCpuThreadPoolCapacity(SEXP threads_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<int>::type threads(threads_sexp);
+	SetCpuThreadPoolCapacity(threads);
+	return R_NilValue;
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_SetCpuThreadPoolCapacity(SEXP threads_sexp){
+	Rf_error("Cannot call SetCpuThreadPoolCapacity(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+
+extern "C" SEXP _arrow_available() {
+return Rf_ScalarLogical(
+#if defined(ARROW_R_WITH_ARROW)
+  TRUE
+#else
+  FALSE
+#endif
+);
+}
+
+static const R_CallMethodDef CallEntries[] = {
+		{ "_arrow_available", (DL_FUNC)& _arrow_available, 0 },
+		{ "_arrow_Array__Slice1", (DL_FUNC) &_arrow_Array__Slice1, 2}, 
+		{ "_arrow_Array__Slice2", (DL_FUNC) &_arrow_Array__Slice2, 3}, 
+		{ "_arrow_Array__IsNull", (DL_FUNC) &_arrow_Array__IsNull, 2}, 
+		{ "_arrow_Array__IsValid", (DL_FUNC) &_arrow_Array__IsValid, 2}, 
+		{ "_arrow_Array__length", (DL_FUNC) &_arrow_Array__length, 1}, 
+		{ "_arrow_Array__offset", (DL_FUNC) &_arrow_Array__offset, 1}, 
+		{ "_arrow_Array__null_count", (DL_FUNC) &_arrow_Array__null_count, 1}, 
+		{ "_arrow_Array__type", (DL_FUNC) &_arrow_Array__type, 1}, 
+		{ "_arrow_Array__ToString", (DL_FUNC) &_arrow_Array__ToString, 1}, 
+		{ "_arrow_Array__type_id", (DL_FUNC) &_arrow_Array__type_id, 1}, 
+		{ "_arrow_Array__Equals", (DL_FUNC) &_arrow_Array__Equals, 2}, 
+		{ "_arrow_Array__ApproxEquals", (DL_FUNC) &_arrow_Array__ApproxEquals, 2}, 
+		{ "_arrow_Array__data", (DL_FUNC) &_arrow_Array__data, 1}, 
+		{ "_arrow_Array__RangeEquals", (DL_FUNC) &_arrow_Array__RangeEquals, 5}, 
+		{ "_arrow_Array__Mask", (DL_FUNC) &_arrow_Array__Mask, 1}, 
+		{ "_arrow_DictionaryArray__indices", (DL_FUNC) &_arrow_DictionaryArray__indices, 1}, 
+		{ "_arrow_DictionaryArray__dictionary", (DL_FUNC) &_arrow_DictionaryArray__dictionary, 1}, 
+		{ "_arrow_Array__as_vector", (DL_FUNC) &_arrow_Array__as_vector, 1}, 
+		{ "_arrow_ChunkedArray__as_vector", (DL_FUNC) &_arrow_ChunkedArray__as_vector, 1}, 
+		{ "_arrow_RecordBatch__to_dataframe", (DL_FUNC) &_arrow_RecordBatch__to_dataframe, 2}, 
+		{ "_arrow_Table__to_dataframe", (DL_FUNC) &_arrow_Table__to_dataframe, 2}, 
+		{ "_arrow_Array__infer_type", (DL_FUNC) &_arrow_Array__infer_type, 1}, 
+		{ "_arrow_Array__from_vector", (DL_FUNC) &_arrow_Array__from_vector, 2}, 
+		{ "_arrow_ChunkedArray__from_list", (DL_FUNC) &_arrow_ChunkedArray__from_list, 2}, 
+		{ "_arrow_ArrayData__get_type", (DL_FUNC) &_arrow_ArrayData__get_type, 1}, 
+		{ "_arrow_ArrayData__get_length", (DL_FUNC) &_arrow_ArrayData__get_length, 1}, 
+		{ "_arrow_ArrayData__get_null_count", (DL_FUNC) &_arrow_ArrayData__get_null_count, 1}, 
+		{ "_arrow_ArrayData__get_offset", (DL_FUNC) &_arrow_ArrayData__get_offset, 1}, 
+		{ "_arrow_ArrayData__buffers", (DL_FUNC) &_arrow_ArrayData__buffers, 1}, 
+		{ "_arrow_Buffer__is_mutable", (DL_FUNC) &_arrow_Buffer__is_mutable, 1}, 
+		{ "_arrow_Buffer__ZeroPadding", (DL_FUNC) &_arrow_Buffer__ZeroPadding, 1}, 
+		{ "_arrow_Buffer__capacity", (DL_FUNC) &_arrow_Buffer__capacity, 1}, 
+		{ "_arrow_Buffer__size", (DL_FUNC) &_arrow_Buffer__size, 1}, 
+		{ "_arrow_r___RBuffer__initialize", (DL_FUNC) &_arrow_r___RBuffer__initialize, 1}, 
+		{ "_arrow_ChunkedArray__length", (DL_FUNC) &_arrow_ChunkedArray__length, 1}, 
+		{ "_arrow_ChunkedArray__null_count", (DL_FUNC) &_arrow_ChunkedArray__null_count, 1}, 
+		{ "_arrow_ChunkedArray__num_chunks", (DL_FUNC) &_arrow_ChunkedArray__num_chunks, 1}, 
+		{ "_arrow_ChunkedArray__chunk", (DL_FUNC) &_arrow_ChunkedArray__chunk, 2}, 
+		{ "_arrow_ChunkedArray__chunks", (DL_FUNC) &_arrow_ChunkedArray__chunks, 1}, 
+		{ "_arrow_ChunkedArray__type", (DL_FUNC) &_arrow_ChunkedArray__type, 1}, 
+		{ "_arrow_ChunkArray__Slice1", (DL_FUNC) &_arrow_ChunkArray__Slice1, 2}, 
+		{ "_arrow_ChunkArray__Slice2", (DL_FUNC) &_arrow_ChunkArray__Slice2, 3}, 
+		{ "_arrow_Column__length", (DL_FUNC) &_arrow_Column__length, 1}, 
+		{ "_arrow_Column__null_count", (DL_FUNC) &_arrow_Column__null_count, 1}, 
+		{ "_arrow_Column__type", (DL_FUNC) &_arrow_Column__type, 1}, 
+		{ "_arrow_Column__data", (DL_FUNC) &_arrow_Column__data, 1}, 
+		{ "_arrow_util___Codec__Create", (DL_FUNC) &_arrow_util___Codec__Create, 1}, 
+		{ "_arrow_io___CompressedOutputStream__Make", (DL_FUNC) &_arrow_io___CompressedOutputStream__Make, 2}, 
+		{ "_arrow_io___CompressedInputStream__Make", (DL_FUNC) &_arrow_io___CompressedInputStream__Make, 2}, 
+		{ "_arrow_compute___CastOptions__initialize", (DL_FUNC) &_arrow_compute___CastOptions__initialize, 3}, 
+		{ "_arrow_Array__cast", (DL_FUNC) &_arrow_Array__cast, 3}, 
+		{ "_arrow_ChunkedArray__cast", (DL_FUNC) &_arrow_ChunkedArray__cast, 3}, 
+		{ "_arrow_RecordBatch__cast", (DL_FUNC) &_arrow_RecordBatch__cast, 3}, 
+		{ "_arrow_Table__cast", (DL_FUNC) &_arrow_Table__cast, 3}, 
+		{ "_arrow_csv___ReadOptions__initialize", (DL_FUNC) &_arrow_csv___ReadOptions__initialize, 1}, 
+		{ "_arrow_csv___ParseOptions__initialize", (DL_FUNC) &_arrow_csv___ParseOptions__initialize, 1}, 
+		{ "_arrow_csv___ConvertOptions__initialize", (DL_FUNC) &_arrow_csv___ConvertOptions__initialize, 1}, 
+		{ "_arrow_csv___TableReader__Make", (DL_FUNC) &_arrow_csv___TableReader__Make, 4}, 
+		{ "_arrow_csv___TableReader__Read", (DL_FUNC) &_arrow_csv___TableReader__Read, 1}, 
+		{ "_arrow_shared_ptr_is_null", (DL_FUNC) &_arrow_shared_ptr_is_null, 1}, 
+		{ "_arrow_unique_ptr_is_null", (DL_FUNC) &_arrow_unique_ptr_is_null, 1}, 
+		{ "_arrow_Int8__initialize", (DL_FUNC) &_arrow_Int8__initialize, 0}, 
+		{ "_arrow_Int16__initialize", (DL_FUNC) &_arrow_Int16__initialize, 0}, 
+		{ "_arrow_Int32__initialize", (DL_FUNC) &_arrow_Int32__initialize, 0}, 
+		{ "_arrow_Int64__initialize", (DL_FUNC) &_arrow_Int64__initialize, 0}, 
+		{ "_arrow_UInt8__initialize", (DL_FUNC) &_arrow_UInt8__initialize, 0}, 
+		{ "_arrow_UInt16__initialize", (DL_FUNC) &_arrow_UInt16__initialize, 0}, 
+		{ "_arrow_UInt32__initialize", (DL_FUNC) &_arrow_UInt32__initialize, 0}, 
+		{ "_arrow_UInt64__initialize", (DL_FUNC) &_arrow_UInt64__initialize, 0}, 
+		{ "_arrow_Float16__initialize", (DL_FUNC) &_arrow_Float16__initialize, 0}, 
+		{ "_arrow_Float32__initialize", (DL_FUNC) &_arrow_Float32__initialize, 0}, 
+		{ "_arrow_Float64__initialize", (DL_FUNC) &_arrow_Float64__initialize, 0}, 
+		{ "_arrow_Boolean__initialize", (DL_FUNC) &_arrow_Boolean__initialize, 0}, 
+		{ "_arrow_Utf8__initialize", (DL_FUNC) &_arrow_Utf8__initialize, 0}, 
+		{ "_arrow_Date32__initialize", (DL_FUNC) &_arrow_Date32__initialize, 0}, 
+		{ "_arrow_Date64__initialize", (DL_FUNC) &_arrow_Date64__initialize, 0}, 
+		{ "_arrow_Null__initialize", (DL_FUNC) &_arrow_Null__initialize, 0}, 
+		{ "_arrow_Decimal128Type__initialize", (DL_FUNC) &_arrow_Decimal128Type__initialize, 2}, 
+		{ "_arrow_FixedSizeBinary__initialize", (DL_FUNC) &_arrow_FixedSizeBinary__initialize, 1}, 
+		{ "_arrow_Timestamp__initialize1", (DL_FUNC) &_arrow_Timestamp__initialize1, 1}, 
+		{ "_arrow_Timestamp__initialize2", (DL_FUNC) &_arrow_Timestamp__initialize2, 2}, 
+		{ "_arrow_Time32__initialize", (DL_FUNC) &_arrow_Time32__initialize, 1}, 
+		{ "_arrow_Time64__initialize", (DL_FUNC) &_arrow_Time64__initialize, 1}, 
+		{ "_arrow_list__", (DL_FUNC) &_arrow_list__, 1}, 
+		{ "_arrow_struct_", (DL_FUNC) &_arrow_struct_, 1}, 
+		{ "_arrow_DataType__ToString", (DL_FUNC) &_arrow_DataType__ToString, 1}, 
+		{ "_arrow_DataType__name", (DL_FUNC) &_arrow_DataType__name, 1}, 
+		{ "_arrow_DataType__Equals", (DL_FUNC) &_arrow_DataType__Equals, 2}, 
+		{ "_arrow_DataType__num_children", (DL_FUNC) &_arrow_DataType__num_children, 1}, 
+		{ "_arrow_DataType__children_pointer", (DL_FUNC) &_arrow_DataType__children_pointer, 1}, 
+		{ "_arrow_DataType__id", (DL_FUNC) &_arrow_DataType__id, 1}, 
+		{ "_arrow_schema_", (DL_FUNC) &_arrow_schema_, 1}, 
+		{ "_arrow_Schema__ToString", (DL_FUNC) &_arrow_Schema__ToString, 1}, 
+		{ "_arrow_Schema__num_fields", (DL_FUNC) &_arrow_Schema__num_fields, 1}, 
+		{ "_arrow_Schema__field", (DL_FUNC) &_arrow_Schema__field, 2}, 
+		{ "_arrow_Schema__names", (DL_FUNC) &_arrow_Schema__names, 1}, 
+		{ "_arrow_ListType__ToString", (DL_FUNC) &_arrow_ListType__ToString, 1}, 
+		{ "_arrow_FixedWidthType__bit_width", (DL_FUNC) &_arrow_FixedWidthType__bit_width, 1}, 
+		{ "_arrow_DateType__unit", (DL_FUNC) &_arrow_DateType__unit, 1}, 
+		{ "_arrow_TimeType__unit", (DL_FUNC) &_arrow_TimeType__unit, 1}, 
+		{ "_arrow_DecimalType__precision", (DL_FUNC) &_arrow_DecimalType__precision, 1}, 
+		{ "_arrow_DecimalType__scale", (DL_FUNC) &_arrow_DecimalType__scale, 1}, 
+		{ "_arrow_TimestampType__timezone", (DL_FUNC) &_arrow_TimestampType__timezone, 1}, 
+		{ "_arrow_TimestampType__unit", (DL_FUNC) &_arrow_TimestampType__unit, 1}, 
+		{ "_arrow_DictionaryType__initialize", (DL_FUNC) &_arrow_DictionaryType__initialize, 3}, 
+		{ "_arrow_DictionaryType__index_type", (DL_FUNC) &_arrow_DictionaryType__index_type, 1}, 
+		{ "_arrow_DictionaryType__value_type", (DL_FUNC) &_arrow_DictionaryType__value_type, 1}, 
+		{ "_arrow_DictionaryType__name", (DL_FUNC) &_arrow_DictionaryType__name, 1}, 
+		{ "_arrow_DictionaryType__ordered", (DL_FUNC) &_arrow_DictionaryType__ordered, 1}, 
+		{ "_arrow_ipc___feather___TableWriter__SetDescription", (DL_FUNC) &_arrow_ipc___feather___TableWriter__SetDescription, 2}, 
+		{ "_arrow_ipc___feather___TableWriter__SetNumRows", (DL_FUNC) &_arrow_ipc___feather___TableWriter__SetNumRows, 2}, 
+		{ "_arrow_ipc___feather___TableWriter__Append", (DL_FUNC) &_arrow_ipc___feather___TableWriter__Append, 3}, 
+		{ "_arrow_ipc___feather___TableWriter__Finalize", (DL_FUNC) &_arrow_ipc___feather___TableWriter__Finalize, 1}, 
+		{ "_arrow_ipc___feather___TableWriter__Open", (DL_FUNC) &_arrow_ipc___feather___TableWriter__Open, 1}, 
+		{ "_arrow_ipc___TableWriter__RecordBatch__WriteFeather", (DL_FUNC) &_arrow_ipc___TableWriter__RecordBatch__WriteFeather, 2}, 
+		{ "_arrow_ipc___feather___TableReader__GetDescription", (DL_FUNC) &_arrow_ipc___feather___TableReader__GetDescription, 1}, 
+		{ "_arrow_ipc___feather___TableReader__HasDescription", (DL_FUNC) &_arrow_ipc___feather___TableReader__HasDescription, 1}, 
+		{ "_arrow_ipc___feather___TableReader__version", (DL_FUNC) &_arrow_ipc___feather___TableReader__version, 1}, 
+		{ "_arrow_ipc___feather___TableReader__num_rows", (DL_FUNC) &_arrow_ipc___feather___TableReader__num_rows, 1}, 
+		{ "_arrow_ipc___feather___TableReader__num_columns", (DL_FUNC) &_arrow_ipc___feather___TableReader__num_columns, 1}, 
+		{ "_arrow_ipc___feather___TableReader__GetColumnName", (DL_FUNC) &_arrow_ipc___feather___TableReader__GetColumnName, 2}, 
+		{ "_arrow_ipc___feather___TableReader__GetColumn", (DL_FUNC) &_arrow_ipc___feather___TableReader__GetColumn, 2}, 
+		{ "_arrow_ipc___feather___TableReader__Read", (DL_FUNC) &_arrow_ipc___feather___TableReader__Read, 2}, 
+		{ "_arrow_ipc___feather___TableReader__Open", (DL_FUNC) &_arrow_ipc___feather___TableReader__Open, 1}, 
+		{ "_arrow_Field__initialize", (DL_FUNC) &_arrow_Field__initialize, 3}, 
+		{ "_arrow_Field__ToString", (DL_FUNC) &_arrow_Field__ToString, 1}, 
+		{ "_arrow_Field__name", (DL_FUNC) &_arrow_Field__name, 1}, 
+		{ "_arrow_Field__Equals", (DL_FUNC) &_arrow_Field__Equals, 2}, 
+		{ "_arrow_Field__nullable", (DL_FUNC) &_arrow_Field__nullable, 1}, 
+		{ "_arrow_Field__type", (DL_FUNC) &_arrow_Field__type, 1}, 
+		{ "_arrow_io___Readable__Read", (DL_FUNC) &_arrow_io___Readable__Read, 2}, 
+		{ "_arrow_io___InputStream__Close", (DL_FUNC) &_arrow_io___InputStream__Close, 1}, 
+		{ "_arrow_io___OutputStream__Close", (DL_FUNC) &_arrow_io___OutputStream__Close, 1}, 
+		{ "_arrow_io___RandomAccessFile__GetSize", (DL_FUNC) &_arrow_io___RandomAccessFile__GetSize, 1}, 
+		{ "_arrow_io___RandomAccessFile__supports_zero_copy", (DL_FUNC) &_arrow_io___RandomAccessFile__supports_zero_copy, 1}, 
+		{ "_arrow_io___RandomAccessFile__Seek", (DL_FUNC) &_arrow_io___RandomAccessFile__Seek, 2}, 
+		{ "_arrow_io___RandomAccessFile__Tell", (DL_FUNC) &_arrow_io___RandomAccessFile__Tell, 1}, 
+		{ "_arrow_io___MemoryMappedFile__Create", (DL_FUNC) &_arrow_io___MemoryMappedFile__Create, 2}, 
+		{ "_arrow_io___MemoryMappedFile__Open", (DL_FUNC) &_arrow_io___MemoryMappedFile__Open, 2}, 
+		{ "_arrow_io___MemoryMappedFile__Resize", (DL_FUNC) &_arrow_io___MemoryMappedFile__Resize, 2}, 
+		{ "_arrow_io___ReadableFile__Open", (DL_FUNC) &_arrow_io___ReadableFile__Open, 1}, 
+		{ "_arrow_io___BufferReader__initialize", (DL_FUNC) &_arrow_io___BufferReader__initialize, 1}, 
+		{ "_arrow_io___Writable__write", (DL_FUNC) &_arrow_io___Writable__write, 2}, 
+		{ "_arrow_io___OutputStream__Tell", (DL_FUNC) &_arrow_io___OutputStream__Tell, 1}, 
+		{ "_arrow_io___FileOutputStream__Open", (DL_FUNC) &_arrow_io___FileOutputStream__Open, 1}, 
+		{ "_arrow_io___BufferOutputStream__Create", (DL_FUNC) &_arrow_io___BufferOutputStream__Create, 1}, 
+		{ "_arrow_io___BufferOutputStream__capacity", (DL_FUNC) &_arrow_io___BufferOutputStream__capacity, 1}, 
+		{ "_arrow_io___BufferOutputStream__Finish", (DL_FUNC) &_arrow_io___BufferOutputStream__Finish, 1}, 
+		{ "_arrow_io___BufferOutputStream__Tell", (DL_FUNC) &_arrow_io___BufferOutputStream__Tell, 1}, 
+		{ "_arrow_io___BufferOutputStream__Write", (DL_FUNC) &_arrow_io___BufferOutputStream__Write, 2}, 
+		{ "_arrow_io___MockOutputStream__initialize", (DL_FUNC) &_arrow_io___MockOutputStream__initialize, 0}, 
+		{ "_arrow_io___MockOutputStream__GetExtentBytesWritten", (DL_FUNC) &_arrow_io___MockOutputStream__GetExtentBytesWritten, 1}, 
+		{ "_arrow_io___FixedSizeBufferWriter__initialize", (DL_FUNC) &_arrow_io___FixedSizeBufferWriter__initialize, 1}, 
+		{ "_arrow_MemoryPool__default", (DL_FUNC) &_arrow_MemoryPool__default, 0}, 
+		{ "_arrow_MemoryPool__bytes_allocated", (DL_FUNC) &_arrow_MemoryPool__bytes_allocated, 1}, 
+		{ "_arrow_MemoryPool__max_memory", (DL_FUNC) &_arrow_MemoryPool__max_memory, 1}, 
+		{ "_arrow_ipc___Message__body_length", (DL_FUNC) &_arrow_ipc___Message__body_length, 1}, 
+		{ "_arrow_ipc___Message__metadata", (DL_FUNC) &_arrow_ipc___Message__metadata, 1}, 
+		{ "_arrow_ipc___Message__body", (DL_FUNC) &_arrow_ipc___Message__body, 1}, 
+		{ "_arrow_ipc___Message__Verify", (DL_FUNC) &_arrow_ipc___Message__Verify, 1}, 
+		{ "_arrow_ipc___Message__type", (DL_FUNC) &_arrow_ipc___Message__type, 1}, 
+		{ "_arrow_ipc___Message__Equals", (DL_FUNC) &_arrow_ipc___Message__Equals, 2}, 
+		{ "_arrow_ipc___ReadRecordBatch__Message__Schema", (DL_FUNC) &_arrow_ipc___ReadRecordBatch__Message__Schema, 2}, 
+		{ "_arrow_ipc___ReadSchema_InputStream", (DL_FUNC) &_arrow_ipc___ReadSchema_InputStream, 1}, 
+		{ "_arrow_ipc___MessageReader__Open", (DL_FUNC) &_arrow_ipc___MessageReader__Open, 1}, 
+		{ "_arrow_ipc___MessageReader__ReadNextMessage", (DL_FUNC) &_arrow_ipc___MessageReader__ReadNextMessage, 1}, 
+		{ "_arrow_ipc___ReadMessage", (DL_FUNC) &_arrow_ipc___ReadMessage, 1}, 
+		{ "_arrow_read_parquet_file", (DL_FUNC) &_arrow_read_parquet_file, 1}, 
+		{ "_arrow_RecordBatch__num_columns", (DL_FUNC) &_arrow_RecordBatch__num_columns, 1}, 
+		{ "_arrow_RecordBatch__num_rows", (DL_FUNC) &_arrow_RecordBatch__num_rows, 1}, 
+		{ "_arrow_RecordBatch__schema", (DL_FUNC) &_arrow_RecordBatch__schema, 1}, 
+		{ "_arrow_RecordBatch__columns", (DL_FUNC) &_arrow_RecordBatch__columns, 1}, 
+		{ "_arrow_RecordBatch__column", (DL_FUNC) &_arrow_RecordBatch__column, 2}, 
+		{ "_arrow_RecordBatch__from_dataframe", (DL_FUNC) &_arrow_RecordBatch__from_dataframe, 1}, 
+		{ "_arrow_RecordBatch__Equals", (DL_FUNC) &_arrow_RecordBatch__Equals, 2}, 
+		{ "_arrow_RecordBatch__RemoveColumn", (DL_FUNC) &_arrow_RecordBatch__RemoveColumn, 2}, 
+		{ "_arrow_RecordBatch__column_name", (DL_FUNC) &_arrow_RecordBatch__column_name, 2}, 
+		{ "_arrow_RecordBatch__names", (DL_FUNC) &_arrow_RecordBatch__names, 1}, 
+		{ "_arrow_RecordBatch__Slice1", (DL_FUNC) &_arrow_RecordBatch__Slice1, 2}, 
+		{ "_arrow_RecordBatch__Slice2", (DL_FUNC) &_arrow_RecordBatch__Slice2, 3}, 
+		{ "_arrow_ipc___SerializeRecordBatch__Raw", (DL_FUNC) &_arrow_ipc___SerializeRecordBatch__Raw, 1}, 
+		{ "_arrow_ipc___ReadRecordBatch__InputStream__Schema", (DL_FUNC) &_arrow_ipc___ReadRecordBatch__InputStream__Schema, 2}, 
+		{ "_arrow_RecordBatch__from_arrays", (DL_FUNC) &_arrow_RecordBatch__from_arrays, 2}, 
+		{ "_arrow_RecordBatchReader__schema", (DL_FUNC) &_arrow_RecordBatchReader__schema, 1}, 
+		{ "_arrow_RecordBatchReader__ReadNext", (DL_FUNC) &_arrow_RecordBatchReader__ReadNext, 1}, 
+		{ "_arrow_ipc___RecordBatchStreamReader__Open", (DL_FUNC) &_arrow_ipc___RecordBatchStreamReader__Open, 1}, 
+		{ "_arrow_ipc___RecordBatchStreamReader__batches", (DL_FUNC) &_arrow_ipc___RecordBatchStreamReader__batches, 1}, 
+		{ "_arrow_ipc___RecordBatchFileReader__schema", (DL_FUNC) &_arrow_ipc___RecordBatchFileReader__schema, 1}, 
+		{ "_arrow_ipc___RecordBatchFileReader__num_record_batches", (DL_FUNC) &_arrow_ipc___RecordBatchFileReader__num_record_batches, 1}, 
+		{ "_arrow_ipc___RecordBatchFileReader__ReadRecordBatch", (DL_FUNC) &_arrow_ipc___RecordBatchFileReader__ReadRecordBatch, 2}, 
+		{ "_arrow_ipc___RecordBatchFileReader__Open", (DL_FUNC) &_arrow_ipc___RecordBatchFileReader__Open, 1}, 
+		{ "_arrow_Table__from_RecordBatchFileReader", (DL_FUNC) &_arrow_Table__from_RecordBatchFileReader, 1}, 
+		{ "_arrow_Table__from_RecordBatchStreamReader", (DL_FUNC) &_arrow_Table__from_RecordBatchStreamReader, 1}, 
+		{ "_arrow_ipc___RecordBatchFileReader__batches", (DL_FUNC) &_arrow_ipc___RecordBatchFileReader__batches, 1}, 
+		{ "_arrow_ipc___RecordBatchWriter__WriteRecordBatch", (DL_FUNC) &_arrow_ipc___RecordBatchWriter__WriteRecordBatch, 2}, 
+		{ "_arrow_ipc___RecordBatchWriter__WriteTable", (DL_FUNC) &_arrow_ipc___RecordBatchWriter__WriteTable, 2}, 
+		{ "_arrow_ipc___RecordBatchWriter__Close", (DL_FUNC) &_arrow_ipc___RecordBatchWriter__Close, 1}, 
+		{ "_arrow_ipc___RecordBatchFileWriter__Open", (DL_FUNC) &_arrow_ipc___RecordBatchFileWriter__Open, 2}, 
+		{ "_arrow_ipc___RecordBatchStreamWriter__Open", (DL_FUNC) &_arrow_ipc___RecordBatchStreamWriter__Open, 2}, 
+		{ "_arrow_Table__from_dataframe", (DL_FUNC) &_arrow_Table__from_dataframe, 1}, 
+		{ "_arrow_Table__num_columns", (DL_FUNC) &_arrow_Table__num_columns, 1}, 
+		{ "_arrow_Table__num_rows", (DL_FUNC) &_arrow_Table__num_rows, 1}, 
+		{ "_arrow_Table__schema", (DL_FUNC) &_arrow_Table__schema, 1}, 
+		{ "_arrow_Table__column", (DL_FUNC) &_arrow_Table__column, 2}, 
+		{ "_arrow_Table__columns", (DL_FUNC) &_arrow_Table__columns, 1}, 
+		{ "_arrow_Table__from_dots", (DL_FUNC) &_arrow_Table__from_dots, 2}, 
+		{ "_arrow_GetCpuThreadPoolCapacity", (DL_FUNC) &_arrow_GetCpuThreadPoolCapacity, 0}, 
+		{ "_arrow_SetCpuThreadPoolCapacity", (DL_FUNC) &_arrow_SetCpuThreadPoolCapacity, 1}, 
+		{NULL, NULL, 0}
+};
+
+RcppExport void R_init_arrow(DllInfo* dll){
+  R_registerRoutines(dll, NULL, CallEntries, NULL, NULL);
+  R_useDynamicSymbols(dll, FALSE);
+}
+
diff --git a/r/src/arrow_types.h b/r/src/arrow_types.h
index f4dad1b..f2ea6dd 100644
--- a/r/src/arrow_types.h
+++ b/r/src/arrow_types.h
@@ -22,26 +22,24 @@
 #include <vector>
 
 #include <RcppCommon.h>
-
 #undef Free
-#include <arrow/api.h>
-#include <arrow/compute/api.h>
-#include <arrow/csv/reader.h>
-#include <arrow/io/compressed.h>
-#include <arrow/io/file.h>
-#include <arrow/io/memory.h>
-#include <arrow/ipc/feather.h>
-#include <arrow/ipc/reader.h>
-#include <arrow/ipc/writer.h>
-#include <arrow/type.h>
-#include <arrow/util/compression.h>
 
-#define STOP_IF(TEST, MSG)     \
-  do {                         \
-    if (TEST) Rcpp::stop(MSG); \
+namespace arrow {
+namespace r {
+struct symbols {
+  static SEXP units;
+  static SEXP xp;
+  static SEXP dot_Internal;
+  static SEXP inspect;
+};
+}  // namespace r
+}  // namespace arrow
+
+#define STOP_IF_NOT(TEST, MSG)    \
+  do {                            \
+    if (!(TEST)) Rcpp::stop(MSG); \
   } while (0)
 
-#define STOP_IF_NOT(TEST, MSG) STOP_IF(!(TEST), MSG)
 #define STOP_IF_NOT_OK(s) STOP_IF_NOT(s.ok(), s.ToString())
 
 template <typename T>
@@ -54,17 +52,6 @@ struct NoDelete {
   inline void operator()(T* ptr) {}
 };
 
-namespace arrow {
-namespace r {
-struct symbols {
-  static SEXP units;
-  static SEXP xp;
-  static SEXP dot_Internal;
-  static SEXP inspect;
-};
-}  // namespace r
-}  // namespace arrow
-
 namespace Rcpp {
 namespace internal {
 
@@ -116,6 +103,7 @@ struct wrap_type_traits<std::unique_ptr<T>> {
 };
 
 }  // namespace traits
+
 namespace internal {
 
 template <typename T>
@@ -125,19 +113,10 @@ template <typename T>
 inline SEXP wrap_dispatch(const T& x, Rcpp::traits::wrap_type_unique_ptr_tag);
 
 }  // namespace internal
-
 }  // namespace Rcpp
 
 #include <Rcpp.h>
 
-RCPP_EXPOSED_ENUM_NODECL(arrow::Type::type)
-RCPP_EXPOSED_ENUM_NODECL(arrow::DateUnit)
-RCPP_EXPOSED_ENUM_NODECL(arrow::TimeUnit::type)
-RCPP_EXPOSED_ENUM_NODECL(arrow::StatusCode)
-RCPP_EXPOSED_ENUM_NODECL(arrow::io::FileMode::type)
-RCPP_EXPOSED_ENUM_NODECL(arrow::ipc::Message::Type)
-RCPP_EXPOSED_ENUM_NODECL(arrow::Compression::type)
-
 namespace Rcpp {
 namespace internal {
 
@@ -177,14 +156,61 @@ inline constexpr Rbyte default_value<RAWSXP>() {
 
 }  // namespace Rcpp
 
+namespace arrow {
+namespace r {
+
+template <typename T>
+inline std::shared_ptr<T> extract(SEXP x) {
+  return Rcpp::ConstReferenceSmartPtrInputParameter<std::shared_ptr<T>>(x);
+}
+
+}  // namespace r
+}  // namespace arrow
+
+#if defined(ARROW_R_WITH_ARROW)
+#include <arrow/api.h>
+#include <arrow/compute/api.h>
+#include <arrow/csv/reader.h>
+#include <arrow/io/compressed.h>
+#include <arrow/io/file.h>
+#include <arrow/io/memory.h>
+#include <arrow/ipc/feather.h>
+#include <arrow/ipc/reader.h>
+#include <arrow/ipc/writer.h>
+#include <arrow/type.h>
+#include <arrow/util/compression.h>
+
+RCPP_EXPOSED_ENUM_NODECL(arrow::Type::type)
+RCPP_EXPOSED_ENUM_NODECL(arrow::DateUnit)
+RCPP_EXPOSED_ENUM_NODECL(arrow::TimeUnit::type)
+RCPP_EXPOSED_ENUM_NODECL(arrow::StatusCode)
+RCPP_EXPOSED_ENUM_NODECL(arrow::io::FileMode::type)
+RCPP_EXPOSED_ENUM_NODECL(arrow::ipc::Message::Type)
+RCPP_EXPOSED_ENUM_NODECL(arrow::Compression::type)
+
 SEXP ChunkedArray__as_vector(const std::shared_ptr<arrow::ChunkedArray>& chunked_array);
 SEXP Array__as_vector(const std::shared_ptr<arrow::Array>& array);
 std::shared_ptr<arrow::Array> Array__from_vector(SEXP x, SEXP type);
 std::shared_ptr<arrow::RecordBatch> RecordBatch__from_arrays(SEXP, SEXP);
+std::shared_ptr<arrow::RecordBatch> RecordBatch__from_dataframe(Rcpp::DataFrame tbl);
 
 namespace arrow {
 namespace r {
 
+std::shared_ptr<arrow::Array> Array__from_vector(
+    SEXP x, const std::shared_ptr<arrow::DataType>& type, bool type_infered);
+
+template <typename T>
+std::vector<std::shared_ptr<T>> List_to_shared_ptr_vector(SEXP x) {
+  std::vector<std::shared_ptr<T>> vec;
+  R_xlen_t n = Rf_xlength(x);
+  for (R_xlen_t i = 0; i < n; i++) {
+    Rcpp::ConstReferenceSmartPtrInputParameter<std::shared_ptr<T>> ptr(VECTOR_ELT(x, i));
+    vec.push_back(ptr);
+  }
+  return vec;
+}
+
 void inspect(SEXP obj);
 
 // the integer64 sentinel
@@ -203,23 +229,7 @@ class RBuffer : public MutableBuffer {
   Vec vec_;
 };
 
-template <typename T>
-inline std::shared_ptr<T> extract(SEXP x) {
-  return Rcpp::ConstReferenceSmartPtrInputParameter<std::shared_ptr<T>>(x);
-}
-
-template <typename T>
-std::vector<std::shared_ptr<T>> list_to_shared_ptr_vector(SEXP lst) {
-  R_xlen_t n = XLENGTH(lst);
-  std::vector<std::shared_ptr<T>> res(n);
-  for (R_xlen_t i = 0; i < n; i++) {
-    res[i] = extract<T>(VECTOR_ELT(lst, i));
-  }
-  return res;
-}
-
-std::shared_ptr<arrow::Array> Array__from_vector(
-    SEXP x, const std::shared_ptr<arrow::DataType>& type, bool type_infered);
-
 }  // namespace r
 }  // namespace arrow
+
+#endif
diff --git a/r/src/buffer.cpp b/r/src/buffer.cpp
index bc52e69..afc0d49 100644
--- a/r/src/buffer.cpp
+++ b/r/src/buffer.cpp
@@ -17,27 +17,29 @@
 
 #include "./arrow_types.h"
 
-// [[Rcpp::export]]
+#if defined(ARROW_R_WITH_ARROW)
+
+// [[arrow::export]]
 bool Buffer__is_mutable(const std::shared_ptr<arrow::Buffer>& buffer) {
   return buffer->is_mutable();
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 void Buffer__ZeroPadding(const std::shared_ptr<arrow::Buffer>& buffer) {
   buffer->ZeroPadding();
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 int64_t Buffer__capacity(const std::shared_ptr<arrow::Buffer>& buffer) {
   return buffer->capacity();
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 int64_t Buffer__size(const std::shared_ptr<arrow::Buffer>& buffer) {
   return buffer->size();
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 std::shared_ptr<arrow::Buffer> r___RBuffer__initialize(SEXP x) {
   switch (TYPEOF(x)) {
     case RAWSXP:
@@ -54,3 +56,5 @@ std::shared_ptr<arrow::Buffer> r___RBuffer__initialize(SEXP x) {
   }
   return nullptr;
 }
+
+#endif
diff --git a/r/src/chunkedarray.cpp b/r/src/chunkedarray.cpp
index dab1f68..296f514 100644
--- a/r/src/chunkedarray.cpp
+++ b/r/src/chunkedarray.cpp
@@ -20,46 +20,50 @@
 using Rcpp::List;
 using Rcpp::wrap;
 
-// [[Rcpp::export]]
+#if defined(ARROW_R_WITH_ARROW)
+
+// [[arrow::export]]
 int ChunkedArray__length(const std::shared_ptr<arrow::ChunkedArray>& chunked_array) {
   return chunked_array->length();
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 int ChunkedArray__null_count(const std::shared_ptr<arrow::ChunkedArray>& chunked_array) {
   return chunked_array->null_count();
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 int ChunkedArray__num_chunks(const std::shared_ptr<arrow::ChunkedArray>& chunked_array) {
   return chunked_array->num_chunks();
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 std::shared_ptr<arrow::Array> ChunkedArray__chunk(
     const std::shared_ptr<arrow::ChunkedArray>& chunked_array, int i) {
   return chunked_array->chunk(i);
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 List ChunkedArray__chunks(const std::shared_ptr<arrow::ChunkedArray>& chunked_array) {
   return wrap(chunked_array->chunks());
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 std::shared_ptr<arrow::DataType> ChunkedArray__type(
     const std::shared_ptr<arrow::ChunkedArray>& chunked_array) {
   return chunked_array->type();
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 std::shared_ptr<arrow::ChunkedArray> ChunkArray__Slice1(
     const std::shared_ptr<arrow::ChunkedArray>& chunked_array, int offset) {
   return chunked_array->Slice(offset);
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 std::shared_ptr<arrow::ChunkedArray> ChunkArray__Slice2(
     const std::shared_ptr<arrow::ChunkedArray>& chunked_array, int offset, int length) {
   return chunked_array->Slice(offset, length);
 }
+
+#endif
diff --git a/r/src/column.cpp b/r/src/column.cpp
index b6eff6b..026cb69 100644
--- a/r/src/column.cpp
+++ b/r/src/column.cpp
@@ -17,24 +17,28 @@
 
 #include "./arrow_types.h"
 
-// [[Rcpp::export]]
+#if defined(ARROW_R_WITH_ARROW)
+
+// [[arrow::export]]
 int Column__length(const std::shared_ptr<arrow::Column>& column) {
   return column->length();
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 int Column__null_count(const std::shared_ptr<arrow::Column>& column) {
   return column->null_count();
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 std::shared_ptr<arrow::DataType> Column__type(
     const std::shared_ptr<arrow::Column>& column) {
   return column->type();
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 std::shared_ptr<arrow::ChunkedArray> Column__data(
     const std::shared_ptr<arrow::Column>& column) {
   return column->data();
 }
+
+#endif
diff --git a/r/src/compression.cpp b/r/src/compression.cpp
index c9dadc3..0a4a6d3 100644
--- a/r/src/compression.cpp
+++ b/r/src/compression.cpp
@@ -17,14 +17,16 @@
 
 #include "./arrow_types.h"
 
-// [[Rcpp::export]]
+#if defined(ARROW_R_WITH_ARROW)
+
+// [[arrow::export]]
 std::unique_ptr<arrow::util::Codec> util___Codec__Create(arrow::Compression::type codec) {
   std::unique_ptr<arrow::util::Codec> out;
   STOP_IF_NOT_OK(arrow::util::Codec::Create(codec, &out));
   return out;
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 std::shared_ptr<arrow::io::CompressedOutputStream> io___CompressedOutputStream__Make(
     const std::unique_ptr<arrow::util::Codec>& codec,
     const std::shared_ptr<arrow::io::OutputStream>& raw) {
@@ -33,7 +35,7 @@ std::shared_ptr<arrow::io::CompressedOutputStream> io___CompressedOutputStream__
   return stream;
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 std::shared_ptr<arrow::io::CompressedInputStream> io___CompressedInputStream__Make(
     const std::unique_ptr<arrow::util::Codec>& codec,
     const std::shared_ptr<arrow::io::InputStream>& raw) {
@@ -41,3 +43,5 @@ std::shared_ptr<arrow::io::CompressedInputStream> io___CompressedInputStream__Ma
   STOP_IF_NOT_OK(arrow::io::CompressedInputStream::Make(codec.get(), raw, &stream));
   return stream;
 }
+
+#endif
diff --git a/r/src/compute.cpp b/r/src/compute.cpp
index 42b73ce..4e40910 100644
--- a/r/src/compute.cpp
+++ b/r/src/compute.cpp
@@ -17,7 +17,9 @@
 
 #include "./arrow_types.h"
 
-// [[Rcpp::export]]
+#if defined(ARROW_R_WITH_ARROW)
+
+// [[arrow::export]]
 std::shared_ptr<arrow::compute::CastOptions> compute___CastOptions__initialize(
     bool allow_int_overflow, bool allow_time_truncate, bool allow_float_truncate) {
   auto options = std::make_shared<arrow::compute::CastOptions>();
@@ -27,7 +29,7 @@ std::shared_ptr<arrow::compute::CastOptions> compute___CastOptions__initialize(
   return options;
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 std::shared_ptr<arrow::Array> Array__cast(
     const std::shared_ptr<arrow::Array>& array,
     const std::shared_ptr<arrow::DataType>& target_type,
@@ -38,7 +40,7 @@ std::shared_ptr<arrow::Array> Array__cast(
   return out;
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 std::shared_ptr<arrow::ChunkedArray> ChunkedArray__cast(
     const std::shared_ptr<arrow::ChunkedArray>& chunked_array,
     const std::shared_ptr<arrow::DataType>& target_type,
@@ -50,7 +52,7 @@ std::shared_ptr<arrow::ChunkedArray> ChunkedArray__cast(
   return out.chunked_array();
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 std::shared_ptr<arrow::RecordBatch> RecordBatch__cast(
     const std::shared_ptr<arrow::RecordBatch>& batch,
     const std::shared_ptr<arrow::Schema>& schema,
@@ -65,7 +67,7 @@ std::shared_ptr<arrow::RecordBatch> RecordBatch__cast(
   return arrow::RecordBatch::Make(schema, batch->num_rows(), std::move(columns));
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 std::shared_ptr<arrow::Table> Table__cast(
     const std::shared_ptr<arrow::Table>& table,
     const std::shared_ptr<arrow::Schema>& schema,
@@ -82,3 +84,5 @@ std::shared_ptr<arrow::Table> Table__cast(
 
   return arrow::Table::Make(schema, std::move(columns), table->num_rows());
 }
+
+#endif
diff --git a/r/src/csv.cpp b/r/src/csv.cpp
index 09580be..bfcbae7 100644
--- a/r/src/csv.cpp
+++ b/r/src/csv.cpp
@@ -20,7 +20,9 @@
 using Rcpp::CharacterVector;
 using Rcpp::List_;
 
-// [[Rcpp::export]]
+#if defined(ARROW_R_WITH_ARROW)
+
+// [[arrow::export]]
 std::shared_ptr<arrow::csv::ReadOptions> csv___ReadOptions__initialize(List_ options) {
   auto res =
       std::make_shared<arrow::csv::ReadOptions>(arrow::csv::ReadOptions::Defaults());
@@ -31,7 +33,7 @@ std::shared_ptr<arrow::csv::ReadOptions> csv___ReadOptions__initialize(List_ opt
 
 inline char get_char(CharacterVector x) { return CHAR(STRING_ELT(x, 0))[0]; }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 std::shared_ptr<arrow::csv::ParseOptions> csv___ParseOptions__initialize(List_ options) {
   auto res =
       std::make_shared<arrow::csv::ParseOptions>(arrow::csv::ParseOptions::Defaults());
@@ -46,7 +48,7 @@ std::shared_ptr<arrow::csv::ParseOptions> csv___ParseOptions__initialize(List_ o
   return res;
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 std::shared_ptr<arrow::csv::ConvertOptions> csv___ConvertOptions__initialize(
     List_ options) {
   auto res = std::make_shared<arrow::csv::ConvertOptions>(
@@ -55,7 +57,7 @@ std::shared_ptr<arrow::csv::ConvertOptions> csv___ConvertOptions__initialize(
   return res;
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 std::shared_ptr<arrow::csv::TableReader> csv___TableReader__Make(
     const std::shared_ptr<arrow::io::InputStream>& input,
     const std::shared_ptr<arrow::csv::ReadOptions>& read_options,
@@ -68,10 +70,12 @@ std::shared_ptr<arrow::csv::TableReader> csv___TableReader__Make(
   return table_reader;
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 std::shared_ptr<arrow::Table> csv___TableReader__Read(
     const std::shared_ptr<arrow::csv::TableReader>& table_reader) {
   std::shared_ptr<arrow::Table> table;
   STOP_IF_NOT_OK(table_reader->Read(&table));
   return table;
 }
+
+#endif
diff --git a/r/src/datatype.cpp b/r/src/datatype.cpp
index b0becb4..0ab881d 100644
--- a/r/src/datatype.cpp
+++ b/r/src/datatype.cpp
@@ -22,97 +22,99 @@ using Rcpp::List;
 using Rcpp::stop;
 using Rcpp::wrap;
 
-// [[Rcpp::export]]
+#if defined(ARROW_R_WITH_ARROW)
+
+// [[arrow::export]]
 bool shared_ptr_is_null(SEXP xp) {
   return reinterpret_cast<std::shared_ptr<void>*>(EXTPTR_PTR(xp))->get() == nullptr;
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 bool unique_ptr_is_null(SEXP xp) {
   return reinterpret_cast<std::unique_ptr<void>*>(EXTPTR_PTR(xp))->get() == nullptr;
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 std::shared_ptr<arrow::DataType> Int8__initialize() { return arrow::int8(); }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 std::shared_ptr<arrow::DataType> Int16__initialize() { return arrow::int16(); }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 std::shared_ptr<arrow::DataType> Int32__initialize() { return arrow::int32(); }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 std::shared_ptr<arrow::DataType> Int64__initialize() { return arrow::int64(); }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 std::shared_ptr<arrow::DataType> UInt8__initialize() { return arrow::uint8(); }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 std::shared_ptr<arrow::DataType> UInt16__initialize() { return arrow::uint16(); }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 std::shared_ptr<arrow::DataType> UInt32__initialize() { return arrow::uint32(); }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 std::shared_ptr<arrow::DataType> UInt64__initialize() { return arrow::uint64(); }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 std::shared_ptr<arrow::DataType> Float16__initialize() { return arrow::float16(); }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 std::shared_ptr<arrow::DataType> Float32__initialize() { return arrow::float32(); }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 std::shared_ptr<arrow::DataType> Float64__initialize() { return arrow::float64(); }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 std::shared_ptr<arrow::DataType> Boolean__initialize() { return arrow::boolean(); }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 std::shared_ptr<arrow::DataType> Utf8__initialize() { return arrow::utf8(); }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 std::shared_ptr<arrow::DataType> Date32__initialize() { return arrow::date32(); }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 std::shared_ptr<arrow::DataType> Date64__initialize() { return arrow::date64(); }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 std::shared_ptr<arrow::DataType> Null__initialize() { return arrow::null(); }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 std::shared_ptr<arrow::DataType> Decimal128Type__initialize(int32_t precision,
                                                             int32_t scale) {
   return arrow::decimal(precision, scale);
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 std::shared_ptr<arrow::DataType> FixedSizeBinary__initialize(int32_t byte_width) {
   return arrow::fixed_size_binary(byte_width);
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 std::shared_ptr<arrow::DataType> Timestamp__initialize1(arrow::TimeUnit::type unit) {
   return arrow::timestamp(unit);
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 std::shared_ptr<arrow::DataType> Timestamp__initialize2(arrow::TimeUnit::type unit,
                                                         const std::string& timezone) {
   return arrow::timestamp(unit, timezone);
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 std::shared_ptr<arrow::DataType> Time32__initialize(arrow::TimeUnit::type unit) {
   return arrow::time32(unit);
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 std::shared_ptr<arrow::DataType> Time64__initialize(arrow::TimeUnit::type unit) {
   return arrow::time64(unit);
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 SEXP list__(SEXP x) {
   if (Rf_inherits(x, "arrow::Field")) {
     Rcpp::ConstReferenceSmartPtrInputParameter<std::shared_ptr<arrow::Field>> field(x);
@@ -128,68 +130,58 @@ SEXP list__(SEXP x) {
   return R_NilValue;
 }
 
-template <typename T>
-std::vector<std::shared_ptr<T>> List_to_shared_ptr_vector(List x) {
-  std::vector<std::shared_ptr<T>> vec;
-  for (SEXP element : x) {
-    Rcpp::ConstReferenceSmartPtrInputParameter<std::shared_ptr<T>> ptr(element);
-    vec.push_back(ptr);
-  }
-  return vec;
-}
-
-// [[Rcpp::export]]
+// [[arrow::export]]
 std::shared_ptr<arrow::DataType> struct_(List fields) {
-  return arrow::struct_(List_to_shared_ptr_vector<arrow::Field>(fields));
+  return arrow::struct_(arrow::r::List_to_shared_ptr_vector<arrow::Field>(fields));
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 std::string DataType__ToString(const std::shared_ptr<arrow::DataType>& type) {
   return type->ToString();
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 std::string DataType__name(const std::shared_ptr<arrow::DataType>& type) {
   return type->name();
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 bool DataType__Equals(const std::shared_ptr<arrow::DataType>& lhs,
                       const std::shared_ptr<arrow::DataType>& rhs) {
   return lhs->Equals(*rhs);
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 int DataType__num_children(const std::shared_ptr<arrow::DataType>& type) {
   return type->num_children();
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 List DataType__children_pointer(const std::shared_ptr<arrow::DataType>& type) {
   return List(type->children().begin(), type->children().end());
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 arrow::Type::type DataType__id(const std::shared_ptr<arrow::DataType>& type) {
   return type->id();
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 std::shared_ptr<arrow::Schema> schema_(List fields) {
-  return arrow::schema(List_to_shared_ptr_vector<arrow::Field>(fields));
+  return arrow::schema(arrow::r::List_to_shared_ptr_vector<arrow::Field>(fields));
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 std::string Schema__ToString(const std::shared_ptr<arrow::Schema>& s) {
   return s->ToString();
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 int Schema__num_fields(const std::shared_ptr<arrow::Schema>& s) {
   return s->num_fields();
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 std::shared_ptr<arrow::Field> Schema__field(const std::shared_ptr<arrow::Schema>& s,
                                             int i) {
   if (i >= s->num_fields() || i < 0) {
@@ -199,7 +191,7 @@ std::shared_ptr<arrow::Field> Schema__field(const std::shared_ptr<arrow::Schema>
   return s->field(i);
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 CharacterVector Schema__names(const std::shared_ptr<arrow::Schema>& schema) {
   auto fields = schema->fields();
   return CharacterVector(
@@ -207,72 +199,74 @@ CharacterVector Schema__names(const std::shared_ptr<arrow::Schema>& schema) {
       [](const std::shared_ptr<arrow::Field>& field) { return field->name(); });
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 std::string ListType__ToString(const std::shared_ptr<arrow::ListType>& type) {
   return type->ToString();
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 int FixedWidthType__bit_width(const std::shared_ptr<arrow::FixedWidthType>& type) {
   return type->bit_width();
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 arrow::DateUnit DateType__unit(const std::shared_ptr<arrow::DateType>& type) {
   return type->unit();
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 arrow::TimeUnit::type TimeType__unit(const std::shared_ptr<arrow::TimeType>& type) {
   return type->unit();
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 int32_t DecimalType__precision(const std::shared_ptr<arrow::DecimalType>& type) {
   return type->precision();
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 int32_t DecimalType__scale(const std::shared_ptr<arrow::DecimalType>& type) {
   return type->scale();
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 std::string TimestampType__timezone(const std::shared_ptr<arrow::TimestampType>& type) {
   return type->timezone();
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 arrow::TimeUnit::type TimestampType__unit(
     const std::shared_ptr<arrow::TimestampType>& type) {
   return type->unit();
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 std::shared_ptr<arrow::DataType> DictionaryType__initialize(
     const std::shared_ptr<arrow::DataType>& index_type,
     const std::shared_ptr<arrow::DataType>& value_type, bool ordered) {
   return arrow::dictionary(index_type, value_type, ordered);
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 std::shared_ptr<arrow::DataType> DictionaryType__index_type(
     const std::shared_ptr<arrow::DictionaryType>& type) {
   return type->index_type();
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 std::shared_ptr<arrow::DataType> DictionaryType__value_type(
     const std::shared_ptr<arrow::DictionaryType>& type) {
   return type->value_type();
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 std::string DictionaryType__name(const std::shared_ptr<arrow::DictionaryType>& type) {
   return type->name();
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 bool DictionaryType__ordered(const std::shared_ptr<arrow::DictionaryType>& type) {
   return type->ordered();
 }
+
+#endif
diff --git a/r/src/feather.cpp b/r/src/feather.cpp
index d981b4d..2836ba8 100644
--- a/r/src/feather.cpp
+++ b/r/src/feather.cpp
@@ -17,35 +17,37 @@
 
 #include "./arrow_types.h"
 
+#if defined(ARROW_R_WITH_ARROW)
+
 // ---------- TableWriter
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 void ipc___feather___TableWriter__SetDescription(
     const std::unique_ptr<arrow::ipc::feather::TableWriter>& writer,
     const std::string& description) {
   writer->SetDescription(description);
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 void ipc___feather___TableWriter__SetNumRows(
     const std::unique_ptr<arrow::ipc::feather::TableWriter>& writer, int64_t num_rows) {
   writer->SetNumRows(num_rows);
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 void ipc___feather___TableWriter__Append(
     const std::unique_ptr<arrow::ipc::feather::TableWriter>& writer,
     const std::string& name, const std::shared_ptr<arrow::Array>& values) {
   STOP_IF_NOT_OK(writer->Append(name, *values));
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 void ipc___feather___TableWriter__Finalize(
     const std::unique_ptr<arrow::ipc::feather::TableWriter>& writer) {
   STOP_IF_NOT_OK(writer->Finalize());
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 std::unique_ptr<arrow::ipc::feather::TableWriter> ipc___feather___TableWriter__Open(
     const std::shared_ptr<arrow::io::OutputStream>& stream) {
   std::unique_ptr<arrow::ipc::feather::TableWriter> writer;
@@ -53,7 +55,7 @@ std::unique_ptr<arrow::ipc::feather::TableWriter> ipc___feather___TableWriter__O
   return writer;
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 void ipc___TableWriter__RecordBatch__WriteFeather(
     const std::unique_ptr<arrow::ipc::feather::TableWriter>& writer,
     const std::shared_ptr<arrow::RecordBatch>& batch) {
@@ -67,43 +69,43 @@ void ipc___TableWriter__RecordBatch__WriteFeather(
 
 // ----------- TableReader
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 std::string ipc___feather___TableReader__GetDescription(
     const std::unique_ptr<arrow::ipc::feather::TableReader>& reader) {
   return reader->GetDescription();
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 bool ipc___feather___TableReader__HasDescription(
     const std::unique_ptr<arrow::ipc::feather::TableReader>& reader) {
   return reader->HasDescription();
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 int ipc___feather___TableReader__version(
     const std::unique_ptr<arrow::ipc::feather::TableReader>& reader) {
   return reader->version();
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 int64_t ipc___feather___TableReader__num_rows(
     const std::unique_ptr<arrow::ipc::feather::TableReader>& reader) {
   return reader->num_rows();
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 int64_t ipc___feather___TableReader__num_columns(
     const std::unique_ptr<arrow::ipc::feather::TableReader>& reader) {
   return reader->num_columns();
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 std::string ipc___feather___TableReader__GetColumnName(
     const std::unique_ptr<arrow::ipc::feather::TableReader>& reader, int i) {
   return reader->GetColumnName(i);
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 std::shared_ptr<arrow::Column> ipc___feather___TableReader__GetColumn(
     const std::unique_ptr<arrow::ipc::feather::TableReader>& reader, int i) {
   std::shared_ptr<arrow::Column> column;
@@ -111,7 +113,7 @@ std::shared_ptr<arrow::Column> ipc___feather___TableReader__GetColumn(
   return column;
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 std::shared_ptr<arrow::Table> ipc___feather___TableReader__Read(
     const std::unique_ptr<arrow::ipc::feather::TableReader>& reader, SEXP columns) {
   std::shared_ptr<arrow::Table> table;
@@ -147,10 +149,12 @@ std::shared_ptr<arrow::Table> ipc___feather___TableReader__Read(
   return table;
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 std::unique_ptr<arrow::ipc::feather::TableReader> ipc___feather___TableReader__Open(
     const std::shared_ptr<arrow::io::RandomAccessFile>& stream) {
   std::unique_ptr<arrow::ipc::feather::TableReader> reader;
   STOP_IF_NOT_OK(arrow::ipc::feather::TableReader::Open(stream, &reader));
   return reader;
 }
+
+#endif
diff --git a/r/src/field.cpp b/r/src/field.cpp
index e864bac..249393f 100644
--- a/r/src/field.cpp
+++ b/r/src/field.cpp
@@ -17,35 +17,39 @@
 
 #include "./arrow_types.h"
 
-// [[Rcpp::export]]
+#if defined(ARROW_R_WITH_ARROW)
+
+// [[arrow::export]]
 std::shared_ptr<arrow::Field> Field__initialize(
     const std::string& name, const std::shared_ptr<arrow::DataType>& field,
     bool nullable = true) {
   return arrow::field(name, field, nullable);
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 std::string Field__ToString(const std::shared_ptr<arrow::Field>& field) {
   return field->ToString();
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 std::string Field__name(const std::shared_ptr<arrow::Field>& field) {
   return field->name();
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 bool Field__Equals(const std::shared_ptr<arrow::Field>& field,
                    const std::shared_ptr<arrow::Field>& other) {
   return field->Equals(other);
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 bool Field__nullable(const std::shared_ptr<arrow::Field>& field) {
   return field->nullable();
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 std::shared_ptr<arrow::DataType> Field__type(const std::shared_ptr<arrow::Field>& field) {
   return field->type();
 }
+
+#endif
diff --git a/r/src/io.cpp b/r/src/io.cpp
index 82c2b20..cf142f3 100644
--- a/r/src/io.cpp
+++ b/r/src/io.cpp
@@ -19,9 +19,11 @@
 
 using Rcpp::RawVector_;
 
+#if defined(ARROW_R_WITH_ARROW)
+
 // ------ arrow::io::Readable
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 std::shared_ptr<arrow::Buffer> io___Readable__Read(
     const std::shared_ptr<arrow::io::Readable>& x, int64_t nbytes) {
   std::shared_ptr<arrow::Buffer> buf;
@@ -31,21 +33,21 @@ std::shared_ptr<arrow::Buffer> io___Readable__Read(
 
 // ------ arrow::io::InputStream
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 void io___InputStream__Close(const std::shared_ptr<arrow::io::InputStream>& x) {
   STOP_IF_NOT_OK(x->Close());
 }
 
 // ------ arrow::io::OutputStream
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 void io___OutputStream__Close(const std::shared_ptr<arrow::io::OutputStream>& x) {
   STOP_IF_NOT_OK(x->Close());
 }
 
 // ------ arrow::io::RandomAccessFile
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 int64_t io___RandomAccessFile__GetSize(
     const std::shared_ptr<arrow::io::RandomAccessFile>& x) {
   int64_t out;
@@ -53,19 +55,19 @@ int64_t io___RandomAccessFile__GetSize(
   return out;
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 bool io___RandomAccessFile__supports_zero_copy(
     const std::shared_ptr<arrow::io::RandomAccessFile>& x) {
   return x->supports_zero_copy();
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 void io___RandomAccessFile__Seek(const std::shared_ptr<arrow::io::RandomAccessFile>& x,
                                  int64_t position) {
   STOP_IF_NOT_OK(x->Seek(position));
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 int64_t io___RandomAccessFile__Tell(
     const std::shared_ptr<arrow::io::RandomAccessFile>& x) {
   int64_t out;
@@ -75,7 +77,7 @@ int64_t io___RandomAccessFile__Tell(
 
 // ------ arrow::io::MemoryMappedFile
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 std::shared_ptr<arrow::io::MemoryMappedFile> io___MemoryMappedFile__Create(
     const std::string& path, int64_t size) {
   std::shared_ptr<arrow::io::MemoryMappedFile> out;
@@ -83,7 +85,7 @@ std::shared_ptr<arrow::io::MemoryMappedFile> io___MemoryMappedFile__Create(
   return out;
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 std::shared_ptr<arrow::io::MemoryMappedFile> io___MemoryMappedFile__Open(
     const std::string& path, arrow::io::FileMode::type mode) {
   std::shared_ptr<arrow::io::MemoryMappedFile> out;
@@ -91,7 +93,7 @@ std::shared_ptr<arrow::io::MemoryMappedFile> io___MemoryMappedFile__Open(
   return out;
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 void io___MemoryMappedFile__Resize(const std::shared_ptr<arrow::io::MemoryMappedFile>& x,
                                    int64_t size) {
   STOP_IF_NOT_OK(x->Resize(size));
@@ -99,7 +101,7 @@ void io___MemoryMappedFile__Resize(const std::shared_ptr<arrow::io::MemoryMapped
 
 // ------ arrow::io::ReadableFile
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 std::shared_ptr<arrow::io::ReadableFile> io___ReadableFile__Open(
     const std::string& path) {
   std::shared_ptr<arrow::io::ReadableFile> out;
@@ -109,7 +111,7 @@ std::shared_ptr<arrow::io::ReadableFile> io___ReadableFile__Open(
 
 // ------ arrow::io::BufferReader
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 std::shared_ptr<arrow::io::BufferReader> io___BufferReader__initialize(
     const std::shared_ptr<arrow::Buffer>& buffer) {
   return std::make_shared<arrow::io::BufferReader>(buffer);
@@ -117,7 +119,7 @@ std::shared_ptr<arrow::io::BufferReader> io___BufferReader__initialize(
 
 // ------- arrow::io::Writable
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 void io___Writable__write(const std::shared_ptr<arrow::io::Writable>& stream,
                           const std::shared_ptr<arrow::Buffer>& buf) {
   STOP_IF_NOT_OK(stream->Write(buf->data(), buf->size()));
@@ -125,7 +127,7 @@ void io___Writable__write(const std::shared_ptr<arrow::io::Writable>& stream,
 
 // ------- arrow::io::OutputStream
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 int64_t io___OutputStream__Tell(const std::shared_ptr<arrow::io::OutputStream>& stream) {
   int64_t position;
   STOP_IF_NOT_OK(stream->Tell(&position));
@@ -134,7 +136,7 @@ int64_t io___OutputStream__Tell(const std::shared_ptr<arrow::io::OutputStream>&
 
 // ------ arrow::io::FileOutputStream
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 std::shared_ptr<arrow::io::FileOutputStream> io___FileOutputStream__Open(
     const std::string& path) {
   std::shared_ptr<arrow::io::FileOutputStream> stream;
@@ -144,7 +146,7 @@ std::shared_ptr<arrow::io::FileOutputStream> io___FileOutputStream__Open(
 
 // ------ arrow::BufferOutputStream
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 std::shared_ptr<arrow::io::BufferOutputStream> io___BufferOutputStream__Create(
     int64_t initial_capacity) {
   std::shared_ptr<arrow::io::BufferOutputStream> stream;
@@ -153,13 +155,13 @@ std::shared_ptr<arrow::io::BufferOutputStream> io___BufferOutputStream__Create(
   return stream;
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 int64_t io___BufferOutputStream__capacity(
     const std::shared_ptr<arrow::io::BufferOutputStream>& stream) {
   return stream->capacity();
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 std::shared_ptr<arrow::Buffer> io___BufferOutputStream__Finish(
     const std::shared_ptr<arrow::io::BufferOutputStream>& stream) {
   std::shared_ptr<arrow::Buffer> buffer;
@@ -167,7 +169,7 @@ std::shared_ptr<arrow::Buffer> io___BufferOutputStream__Finish(
   return buffer;
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 int64_t io___BufferOutputStream__Tell(
     const std::shared_ptr<arrow::io::BufferOutputStream>& stream) {
   int64_t res;
@@ -175,7 +177,7 @@ int64_t io___BufferOutputStream__Tell(
   return res;
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 void io___BufferOutputStream__Write(
     const std::shared_ptr<arrow::io::BufferOutputStream>& stream, RawVector_ bytes) {
   STOP_IF_NOT_OK(stream->Write(bytes.begin(), bytes.size()));
@@ -183,12 +185,12 @@ void io___BufferOutputStream__Write(
 
 // ------ arrow::io::MockOutputStream
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 std::shared_ptr<arrow::io::MockOutputStream> io___MockOutputStream__initialize() {
   return std::make_shared<arrow::io::MockOutputStream>();
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 int64_t io___MockOutputStream__GetExtentBytesWritten(
     const std::shared_ptr<arrow::io::MockOutputStream>& stream) {
   return stream->GetExtentBytesWritten();
@@ -196,8 +198,10 @@ int64_t io___MockOutputStream__GetExtentBytesWritten(
 
 // ------ arrow::io::FixedSizeBufferWriter
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 std::shared_ptr<arrow::io::FixedSizeBufferWriter> io___FixedSizeBufferWriter__initialize(
     const std::shared_ptr<arrow::Buffer>& buffer) {
   return std::make_shared<arrow::io::FixedSizeBufferWriter>(buffer);
 }
+
+#endif
diff --git a/r/src/memorypool.cpp b/r/src/memorypool.cpp
index fa0bb4a..bd5cebf 100644
--- a/r/src/memorypool.cpp
+++ b/r/src/memorypool.cpp
@@ -16,19 +16,22 @@
 // under the License.
 
 #include "./arrow_types.h"
+#if defined(ARROW_R_WITH_ARROW)
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 std::shared_ptr<arrow::MemoryPool> MemoryPool__default() {
   return std::shared_ptr<arrow::MemoryPool>(arrow::default_memory_pool(),
                                             NoDelete<arrow::MemoryPool>());
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 int MemoryPool__bytes_allocated(const std::shared_ptr<arrow::MemoryPool>& pool) {
   return pool->bytes_allocated();
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 int MemoryPool__max_memory(const std::shared_ptr<arrow::MemoryPool>& pool) {
   return pool->max_memory();
 }
+
+#endif
diff --git a/r/src/message.cpp b/r/src/message.cpp
index cb8a3c9..2769458 100644
--- a/r/src/message.cpp
+++ b/r/src/message.cpp
@@ -17,41 +17,43 @@
 
 #include "./arrow_types.h"
 
-// [[Rcpp::export]]
+#if defined(ARROW_R_WITH_ARROW)
+
+// [[arrow::export]]
 int64_t ipc___Message__body_length(const std::unique_ptr<arrow::ipc::Message>& message) {
   return message->body_length();
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 std::shared_ptr<arrow::Buffer> ipc___Message__metadata(
     const std::unique_ptr<arrow::ipc::Message>& message) {
   return message->metadata();
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 std::shared_ptr<arrow::Buffer> ipc___Message__body(
     const std::unique_ptr<arrow::ipc::Message>& message) {
   return message->body();
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 int64_t ipc___Message__Verify(const std::unique_ptr<arrow::ipc::Message>& message) {
   return message->Verify();
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 arrow::ipc::Message::Type ipc___Message__type(
     const std::unique_ptr<arrow::ipc::Message>& message) {
   return message->type();
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 bool ipc___Message__Equals(const std::unique_ptr<arrow::ipc::Message>& x,
                            const std::unique_ptr<arrow::ipc::Message>& y) {
   return x->Equals(*y);
 }
 
-// [[Rcpp::export]]
+// [[arrow::export]]
 std::shared_ptr<arrow::RecordBatch> ipc___ReadRecordBatch__Message__Schema(
... 516 lines suppressed ...