You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by pa...@apache.org on 2022/10/26 14:10:05 UTC

[arrow] branch master updated: ARROW-17187: [R] Improve lazy ALTREP implementation for String (#14271)

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

paleolimbot 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 21564cf398 ARROW-17187: [R] Improve lazy ALTREP implementation for String (#14271)
21564cf398 is described below

commit 21564cf3981a1da0662ccd495320a5127b693a1f
Author: Dewey Dunnington <de...@voltrondata.com>
AuthorDate: Wed Oct 26 11:09:58 2022 -0300

    ARROW-17187: [R] Improve lazy ALTREP implementation for String (#14271)
    
    This PR is a more permanent fix for ARROW-16578 (#13415), which noted some very bad performance when calling `unique()` in a character vector created by us. In benchmarking a few other things and implementing some simpler altrep classes around the C Data interface (https://github.com/apache/arrow-nanoarrow/pull/50), I found a few other issues...notably, that calling `head()` (or accessing a single string element) would materialize an entire string vector (rather than just materializin [...]
    
    ``` r
    library(arrow, warn.conflicts = FALSE)
    
    big_string <- as.character(runif(1e6))
    big_string_altrep <- as.vector(as_arrow_array(big_string))
    
    bench::mark(head(big_string), head(big_string_altrep))[c("expression", "mem_alloc")]
    #> # A tibble: 2 × 2
    #>   expression              mem_alloc
    #>   <bch:expr>              <bch:byt>
    #> 1 head(big_string)               0B
    #> 2 head(big_string_altrep)    7.63MB
    ```
    
    For the original STRING_ELT issue, the previous method may have been slow for a few reasons:
    
    - First, it used `BEGIN_CPP11` and `END_CPP11`, which stack-allocates an 8 kb error message array (which is fast but not trivial).
    - Second, it stack-allocates an `RStringViewer` for every string access. This is probably not that bad, but because it's not trivially destructible it couldn't be used without `BEGIN_CPP11`/`END_CPP11`, which allocates more than needed. I statically allocate one of these to solve this.
    - Third, it was calling `GetOption()` (via creating the string viewer) on every string access. This is a little tricky to work around...I opted to read the option whenever we create an ALTREP vector. This means that technically the value read when an individual value materializes could be out of date; however, in the case that a user does `options(arrow.strip_nuls = TRUE)` it will be true for the rest of the session.
    - Fourth, it `unwind_protect`ed for every string access (which is fast but not trivial). I fixed this by statically allocating one `RStringViewer` (so that memory does not leak from a longjmp in the frame) and removing the cpp11 calls from the Elt method (since these would throw exceptions where a longjmp is expected by the calling frame).
    
    This PR also releases the `std::shared_ptr<ChunkedArray>` when we materialize our ALTREP vectors. We use it for length/sum/min/max implementations and for roundtripping but in many cases it's not used and is just taking up memory that could be released and used for something else. I ran ursabot's benchmarks to make sure this doesn't slow things down, and it doesn't (at least for our existing benchmarks).
    
    In changing the above bits, I ran test coverage and noticed that there were parts of altrep.cpp that weren't tested by our test suite. Testing ALTREP is hard because it's almost completely undocumented and where exactly it gets used in the R API changes with each release. I added some helper methods and tests that more accurately reflect the C/C++ usage of ALTREP.
    
    Finally, I noticed that we were doing a "from the beginning" (O(N squred)) search rather than a binary search O(log(n)) in some places to resolve an array/index in the array. This has a non-trivial effect on arrays with many chunks:
    
    Before this PR:
    
    ``` r
    library(arrow, warn.conflicts = FALSE)
    #> Some features are not enabled in this build of Arrow. Run `arrow_info()` for more information.
    
    chunks <- lapply(1:1000, function(i) runif(1e3))
    altrep_one_chunk <- as.vector(ChunkedArray$create(unlist(chunks)))
    altrep_many_chunks <- as.vector(ChunkedArray$create(!!!chunks))
    random_indices <- sample(seq_len(1e6), 1e6, replace = FALSE)
    
    bench::mark(
      altrep_one_chunk[random_indices],
      altrep_many_chunks[random_indices]
    )
    #> Warning: Some expressions had a GC in every iteration; so filtering is disabled.
    #> # A tibble: 2 × 6
    #>   expression                              min   median `itr/sec` mem_a…¹ gc/se…²
    #>   <bch:expr>                         <bch:tm> <bch:tm>     <dbl> <bch:b>   <dbl>
    #> 1 altrep_one_chunk[random_indices]    68.99ms  72.05ms    12.5    7.63MB  16.1
    #> 2 altrep_many_chunks[random_indices]    4.36s    4.36s     0.229  7.63MB   0.459
    #> # … with abbreviated variable names ¹​mem_alloc, ²​`gc/sec`
    ```
    
    After this PR:
    
    ``` r
    library(arrow, warn.conflicts = FALSE)
    
    chunks <- lapply(1:1000, function(i) runif(1e3))
    altrep_one_chunk <- as.vector(ChunkedArray$create(unlist(chunks)))
    altrep_many_chunks <- as.vector(ChunkedArray$create(!!!chunks))
    random_indices <- sample(seq_len(1e6), 1e6, replace = FALSE)
    
    bench::mark(
      altrep_one_chunk[random_indices],
      altrep_many_chunks[random_indices]
    )
    #> Warning: Some expressions had a GC in every iteration; so filtering is disabled.
    #> # A tibble: 2 × 6
    #>   expression                              min   median `itr/sec` mem_a…¹ gc/se…²
    #>   <bch:expr>                         <bch:tm> <bch:tm>     <dbl> <bch:b>   <dbl>
    #> 1 altrep_one_chunk[random_indices]     64.7ms   71.5ms     13.1   7.63MB   16.8
    #> 2 altrep_many_chunks[random_indices]  113.1ms  117.1ms      7.85  7.63MB    9.82
    #> # … with abbreviated variable names ¹​mem_alloc, ²​`gc/sec`
    ```
    
    I solved this by using Arrow's `ChunkResolver`. This resulted in a small change in the ALTREP implementation: it is now an external pointer around a custom `ArrowAltrepData` class that contains a `shared_ptr<ChunkedArray>` and a `ChunkResolver` (before it was just an external pointer to a `shared_ptr<ChunkedArray>`).
    
    Lead-authored-by: Dewey Dunnington <de...@fishandwhistle.net>
    Co-authored-by: Dewey Dunnington <de...@voltrondata.com>
    Signed-off-by: Dewey Dunnington <de...@voltrondata.com>
---
 r/R/arrowExports.R                  |  28 ++-
 r/src/altrep.cpp                    | 416 ++++++++++++++++++++++++++++--------
 r/src/array.cpp                     |   5 +-
 r/src/array_to_vector.cpp           |   4 +-
 r/src/arrowExports.cpp              |  66 +++++-
 r/tests/testthat/test-RecordBatch.R |  21 +-
 r/tests/testthat/test-altrep.R      | 268 ++++++++++++++++++++++-
 7 files changed, 684 insertions(+), 124 deletions(-)

diff --git a/r/R/arrowExports.R b/r/R/arrowExports.R
index c42fca00b5..ac68540d6c 100644
--- a/r/R/arrowExports.R
+++ b/r/R/arrowExports.R
@@ -1,13 +1,33 @@
 # Generated by using data-raw/codegen.R -> do not edit by hand
 
-test_SET_STRING_ELT <- function(s) {
-  invisible(.Call(`_arrow_test_SET_STRING_ELT`, s))
-}
-
 is_arrow_altrep <- function(x) {
   .Call(`_arrow_is_arrow_altrep`, x)
 }
 
+test_arrow_altrep_set_string_elt <- function(x, i, value) {
+  invisible(.Call(`_arrow_test_arrow_altrep_set_string_elt`, x, i, value))
+}
+
+test_arrow_altrep_is_materialized <- function(x) {
+  .Call(`_arrow_test_arrow_altrep_is_materialized`, x)
+}
+
+test_arrow_altrep_force_materialize <- function(x) {
+  .Call(`_arrow_test_arrow_altrep_force_materialize`, x)
+}
+
+test_arrow_altrep_copy_by_element <- function(x) {
+  .Call(`_arrow_test_arrow_altrep_copy_by_element`, x)
+}
+
+test_arrow_altrep_copy_by_region <- function(x, region_size) {
+  .Call(`_arrow_test_arrow_altrep_copy_by_region`, x, region_size)
+}
+
+test_arrow_altrep_copy_by_dataptr <- function(x) {
+  .Call(`_arrow_test_arrow_altrep_copy_by_dataptr`, x)
+}
+
 Array__Slice1 <- function(array, offset) {
   .Call(`_arrow_Array__Slice1`, array, offset)
 }
diff --git a/r/src/altrep.cpp b/r/src/altrep.cpp
index 8e19d13a0e..cace7b13aa 100644
--- a/r/src/altrep.cpp
+++ b/r/src/altrep.cpp
@@ -18,6 +18,7 @@
 #include "./arrow_types.h"
 
 #include <arrow/array.h>
+#include <arrow/chunk_resolver.h>
 #include <arrow/chunked_array.h>
 #include <arrow/compute/api.h>
 #include <arrow/util/bitmap_reader.h>
@@ -79,35 +80,33 @@ void DeletePointer(std::shared_ptr<T>* ptr) {
 template <typename T>
 using Pointer = cpp11::external_pointer<std::shared_ptr<T>, DeletePointer<T>>;
 
-// the ChunkedArray that is being wrapped by the altrep object
-const std::shared_ptr<ChunkedArray>& GetChunkedArray(SEXP alt) {
-  return *Pointer<ChunkedArray>(R_altrep_data1(alt));
-}
+class ArrowAltrepData {
+ public:
+  explicit ArrowAltrepData(const std::shared_ptr<ChunkedArray>& chunked_array)
+      : chunked_array_(chunked_array), resolver_(chunked_array->chunks()) {}
 
-struct ArrayResolve {
-  ArrayResolve(const std::shared_ptr<ChunkedArray>& chunked_array, int64_t i) {
-    for (int idx_chunk = 0; idx_chunk < chunked_array->num_chunks(); idx_chunk++) {
-      std::shared_ptr<Array> chunk = chunked_array->chunk(idx_chunk);
-      auto chunk_size = chunk->length();
-      if (i < chunk_size) {
-        index_ = i;
-        array_ = chunk;
-        position_ = idx_chunk;
-        break;
-      }
+  const std::shared_ptr<ChunkedArray>& chunked_array() { return chunked_array_; }
 
-      i -= chunk_size;
-    }
+  arrow::internal::ChunkLocation locate(int64_t index) {
+    return resolver_.Resolve(index);
   }
 
-  std::shared_ptr<Array> array_;
-  int64_t index_ = 0;
-  int64_t position_ = 0;
+ private:
+  std::shared_ptr<ChunkedArray> chunked_array_;
+  arrow::internal::ChunkResolver resolver_;
 };
 
+// the ChunkedArray that is being wrapped by the altrep object
+const std::shared_ptr<ChunkedArray>& GetChunkedArray(SEXP alt) {
+  auto array_data =
+      reinterpret_cast<ArrowAltrepData*>(R_ExternalPtrAddr(R_altrep_data1(alt)));
+  return array_data->chunked_array();
+}
+
 // base class for all altrep vectors
 //
-// data1: the Array as an external pointer.
+// data1: the Array as an external pointer; becomes NULL when
+//        materialization is needed.
 // data2: starts as NULL, and becomes a standard R vector with the same
 //        data if necessary: if materialization is needed, e.g. if we need
 //        to access its data pointer, with DATAPTR().
@@ -117,7 +116,7 @@ struct AltrepVectorBase {
   static SEXP Make(const std::shared_ptr<ChunkedArray>& chunked_array) {
     SEXP alt = R_new_altrep(
         Impl::class_t,
-        Pointer<ChunkedArray>(new std::shared_ptr<ChunkedArray>(chunked_array)),
+        external_pointer<ArrowAltrepData>(new ArrowAltrepData(chunked_array)),
         R_NilValue);
     MARK_NOT_MUTABLE(alt);
 
@@ -128,20 +127,39 @@ struct AltrepVectorBase {
   // standard R vector with the same data as the array.
   static bool IsMaterialized(SEXP alt) { return !Rf_isNull(Impl::Representation(alt)); }
 
-  static R_xlen_t Length(SEXP alt) { return GetChunkedArray(alt)->length(); }
+  static R_xlen_t Length(SEXP alt) {
+    if (IsMaterialized(alt)) {
+      return Rf_xlength(Representation(alt));
+    } else {
+      return GetChunkedArray(alt)->length();
+    }
+  }
+
+  static int No_NA(SEXP alt) {
+    if (IsMaterialized(alt)) {
+      return false;
+    }
 
-  static int No_NA(SEXP alt) { return GetChunkedArray(alt)->null_count() == 0; }
+    return GetChunkedArray(alt)->null_count() == 0;
+  }
 
   static int Is_sorted(SEXP alt) { return UNKNOWN_SORTEDNESS; }
 
   // What gets printed on .Internal(inspect(<the altrep object>))
   static Rboolean Inspect(SEXP alt, int pre, int deep, int pvec,
                           void (*inspect_subtree)(SEXP, int, int, int)) {
-    const auto& chunked_array = GetChunkedArray(alt);
-    Rprintf("arrow::ChunkedArray<%p, %s, %d chunks, %d nulls> len=%d\n",
-            chunked_array.get(), chunked_array->type()->ToString().c_str(),
-            chunked_array->num_chunks(), chunked_array->null_count(),
-            chunked_array->length());
+    SEXP data_class_sym = CAR(ATTRIB(ALTREP_CLASS(alt)));
+    const char* class_name = CHAR(PRINTNAME(data_class_sym));
+
+    if (IsMaterialized(alt)) {
+      Rprintf("materialized %s len=%d\n", class_name, Rf_xlength(Representation(alt)));
+    } else {
+      const auto& chunked_array = GetChunkedArray(alt);
+      Rprintf("%s<%p, %s, %d chunks, %d nulls> len=%d\n", class_name, chunked_array.get(),
+              chunked_array->type()->ToString().c_str(), chunked_array->num_chunks(),
+              chunked_array->null_count(), chunked_array->length());
+    }
+
     return TRUE;
   }
 
@@ -183,16 +201,14 @@ struct AltrepVectorPrimitive : public AltrepVectorBase<AltrepVectorPrimitive<sex
 
   // Force materialization. After calling this, the data2 slot of the altrep
   // object contains a standard R vector with the same data, with
-  // R sentinels where the Array has nulls.
-  //
-  // The Array remains available so that it can be used by Length(), Min(), etc ...
+  // R sentinels where the Array has nulls. This method also releases the
+  // reference to the original ChunkedArray.
   static SEXP Materialize(SEXP alt) {
     if (!IsMaterialized(alt)) {
       auto size = Base::Length(alt);
 
-      // create an immutable standard R vector
+      // create a standard R vector
       SEXP copy = PROTECT(Rf_allocVector(sexp_type, size));
-      MARK_NOT_MUTABLE(copy);
 
       // copy the data from the array, through Get_region
       Get_region(alt, 0, size, reinterpret_cast<c_type*>(DATAPTR(copy)));
@@ -200,6 +216,11 @@ struct AltrepVectorPrimitive : public AltrepVectorBase<AltrepVectorPrimitive<sex
       // store as data2, this is now considered materialized
       SetRepresentation(alt, copy);
 
+      // we no longer need the original ChunkedArray (keeping it alive uses more
+      // memory than is required, since our methods can now use the
+      // materialized array)
+      R_set_altrep_data1(alt, R_NilValue);
+
       UNPROTECT(1);
     }
     return Representation(alt);
@@ -239,25 +260,20 @@ struct AltrepVectorPrimitive : public AltrepVectorBase<AltrepVectorPrimitive<sex
     }
 
     // Otherwise we have to materialize and hand the pointer to data2
-    //
-    // NOTE: this returns the DATAPTR() of data2 even in the case writeable = TRUE
-    //
-    // which is risky because C(++) clients of this object might
-    // modify data2, and therefore make it diverge from the data of the Array,
-    // but the object was marked as immutable on creation, so doing this is
-    // disregarding the R api.
-    //
-    // Simply stop() when `writeable = TRUE` is too strong, e.g. this fails
-    // identical() which calls DATAPTR() even though DATAPTR_RO() would
-    // be enough
     return DATAPTR(Materialize(alt));
   }
 
   // The value at position i
   static c_type Elt(SEXP alt, R_xlen_t i) {
-    ArrayResolve resolve(GetChunkedArray(alt), i);
-    auto array = resolve.array_;
-    auto j = resolve.index_;
+    if (IsMaterialized(alt)) {
+      return reinterpret_cast<c_type*>(DATAPTR(Representation(alt)))[i];
+    }
+
+    auto altrep_data =
+        reinterpret_cast<ArrowAltrepData*>(R_ExternalPtrAddr(R_altrep_data1(alt)));
+    auto resolve = altrep_data->locate(i);
+    const auto& array = altrep_data->chunked_array()->chunk(resolve.chunk_index);
+    auto j = resolve.index_in_chunk;
 
     return array->IsNull(j) ? cpp11::na<c_type>()
                             : array->data()->template GetValues<c_type>(1)[j];
@@ -279,7 +295,7 @@ struct AltrepVectorPrimitive : public AltrepVectorBase<AltrepVectorPrimitive<sex
     // do a second pass to force the R sentinels for where the
     // array has nulls
     //
-    // This only materialize the region, into buf. Not the entire vector.
+    // This only materializes the region into buf (not the entire vector).
     auto slice = GetChunkedArray(alt)->Slice(i, n);
     R_xlen_t ncopy = slice->length();
 
@@ -318,6 +334,10 @@ struct AltrepVectorPrimitive : public AltrepVectorBase<AltrepVectorPrimitive<sex
 
   template <bool Min>
   static SEXP MinMax(SEXP alt, Rboolean narm) {
+    if (IsMaterialized(alt)) {
+      return nullptr;
+    }
+
     using data_type = typename std::conditional<sexp_type == REALSXP, double, int>::type;
     using scalar_type =
         typename std::conditional<sexp_type == INTSXP, Int32Scalar, DoubleScalar>::type;
@@ -327,7 +347,13 @@ struct AltrepVectorPrimitive : public AltrepVectorBase<AltrepVectorPrimitive<sex
     auto n = chunked_array->length();
     auto null_count = chunked_array->null_count();
     if ((na_rm || n == 0) && null_count == n) {
-      return Rf_ScalarReal(Min ? R_PosInf : R_NegInf);
+      if (Min) {
+        Rf_warning("no non-missing arguments to min; returning Inf");
+        return Rf_ScalarReal(R_PosInf);
+      } else {
+        Rf_warning("no non-missing arguments to max; returning -Inf");
+        return Rf_ScalarReal(R_NegInf);
+      }
     }
     if (!na_rm && null_count > 0) {
       return cpp11::as_sexp(cpp11::na<data_type>());
@@ -350,6 +376,10 @@ struct AltrepVectorPrimitive : public AltrepVectorBase<AltrepVectorPrimitive<sex
   static SEXP Max(SEXP alt, Rboolean narm) { return MinMax<false>(alt, narm); }
 
   static SEXP Sum(SEXP alt, Rboolean narm) {
+    if (IsMaterialized(alt)) {
+      return nullptr;
+    }
+
     using data_type = typename std::conditional<sexp_type == REALSXP, double, int>::type;
 
     const auto& chunked_array = GetChunkedArray(alt);
@@ -389,6 +419,14 @@ struct AltrepFactor : public AltrepVectorBase<AltrepFactor> {
   using Base = AltrepVectorBase<AltrepFactor>;
   using Base::IsMaterialized;
 
+  static R_xlen_t Length(SEXP alt) {
+    if (IsMaterialized(alt)) {
+      return Rf_xlength(Representation(alt));
+    } else {
+      return GetChunkedArray(alt)->length();
+    }
+  }
+
   // redefining because data2 is a paired list with the representation as the
   // first node: the CAR
   static SEXP Representation(SEXP alt) { return CAR(R_altrep_data2(alt)); }
@@ -452,7 +490,7 @@ struct AltrepFactor : public AltrepVectorBase<AltrepFactor> {
 
     // the chunked array as data1
     SEXP data1 =
-        PROTECT(Pointer<ChunkedArray>(new std::shared_ptr<ChunkedArray>(chunked_array)));
+        PROTECT(external_pointer<ArrowAltrepData>(new ArrowAltrepData(chunked_array)));
 
     // a pairlist with the representation in the first node
     SEXP data2 = PROTECT(Rf_list2(R_NilValue,  // representation, empty at first
@@ -487,7 +525,9 @@ struct AltrepFactor : public AltrepVectorBase<AltrepFactor> {
 
       // store as data2, this is now considered materialized
       SetRepresentation(alt, copy);
-      MARK_NOT_MUTABLE(copy);
+
+      // remove the ChunkedArray reference
+      R_set_altrep_data1(alt, R_NilValue);
 
       UNPROTECT(1);
     }
@@ -522,9 +562,12 @@ struct AltrepFactor : public AltrepVectorBase<AltrepFactor> {
       return INTEGER_ELT(Representation(alt), i);
     }
 
-    ArrayResolve resolve(GetChunkedArray(alt), i);
-    auto array = resolve.array_;
-    auto j = resolve.index_;
+    auto altrep_data =
+        reinterpret_cast<ArrowAltrepData*>(R_ExternalPtrAddr(R_altrep_data1(alt)));
+    auto resolve = altrep_data->locate(i);
+
+    const auto& array = altrep_data->chunked_array()->chunk(resolve.chunk_index);
+    auto j = resolve.index_in_chunk;
 
     if (!array->IsNull(j)) {
       const auto& indices =
@@ -532,7 +575,7 @@ struct AltrepFactor : public AltrepVectorBase<AltrepFactor> {
 
       if (WasUnified(alt)) {
         const auto* transpose_data = reinterpret_cast<const int32_t*>(
-            GetArrayTransposed(alt, resolve.position_)->data());
+            GetArrayTransposed(alt, resolve.chunk_index)->data());
 
         switch (indices->type_id()) {
           case Type::UINT8:
@@ -693,11 +736,19 @@ struct AltrepVectorString : public AltrepVectorBase<AltrepVectorString<Type>> {
   using Base::Representation;
   using Base::SetRepresentation;
 
-  // Helper class to convert to R strings
+  static SEXP Make(const std::shared_ptr<ChunkedArray>& chunked_array) {
+    string_viewer().set_strip_out_nuls(GetBoolOption("arrow.skip_nul", false));
+    return Base::Make(chunked_array);
+  }
+
+  // Helper class to convert to R strings. We declare one of these for the
+  // class to avoid having to stack-allocate one for every STRING_ELT call.
   struct RStringViewer {
-    RStringViewer()
-        : strip_out_nuls_(GetBoolOption("arrow.skip_nul", false)),
-          nul_was_stripped_(false) {}
+    RStringViewer() : strip_out_nuls_(false), nul_was_stripped_(false) {}
+
+    void reset_null_was_stripped() { nul_was_stripped_ = false; }
+
+    void set_strip_out_nuls(bool strip_out_nuls) { strip_out_nuls_ = strip_out_nuls; }
 
     // convert the i'th string of the Array to an R string (CHARSXP)
     SEXP Convert(size_t i) {
@@ -777,20 +828,37 @@ struct AltrepVectorString : public AltrepVectorBase<AltrepVectorString<Type>> {
     std::shared_ptr<Array> array_;
     const StringArrayType* string_array_;
     std::string stripped_string_;
-    const bool strip_out_nuls_;
+    bool strip_out_nuls_;
     bool nul_was_stripped_;
     std::string_view view_;
   };
 
-  // Get a single string, as a CHARSXP SEXP from data2.
-  // Materialize if not done so yet, given that it is
-  // likely that there will be another call from R if there is a call (e.g. unique()),
-  // and getting a string from Array is much more costly than from data2.
+  // Get a single string as a CHARSXP SEXP
   static SEXP Elt(SEXP alt, R_xlen_t i) {
-    if (!Base::IsMaterialized(alt)) {
-      Materialize(alt);
+    if (Base::IsMaterialized(alt)) {
+      return STRING_ELT(Representation(alt), i);
+    }
+
+    auto altrep_data =
+        reinterpret_cast<ArrowAltrepData*>(R_ExternalPtrAddr(R_altrep_data1(alt)));
+    auto resolve = altrep_data->locate(i);
+    const auto& array = altrep_data->chunked_array()->chunk(resolve.chunk_index);
+    auto j = resolve.index_in_chunk;
+
+    SEXP s = NA_STRING;
+    RStringViewer& r_string_viewer = string_viewer();
+    r_string_viewer.SetArray(array);
+    // Note: we don't check GetBoolOption("arrow.skip_nul", false) here
+    // because it is too expensive to do so. We do set this value whenever
+    // an altrep string; however, there is a chance that this value could
+    // be out of date by the time a value in the vector is accessed.
+    r_string_viewer.reset_null_was_stripped();
+    s = r_string_viewer.Convert(j);
+    if (r_string_viewer.nul_was_stripped()) {
+      Rf_warning("Stripping '\\0' (nul) from character vector");
     }
-    return STRING_ELT(Representation(alt), i);
+
+    return s;
   }
 
   static void* Dataptr(SEXP alt, Rboolean writeable) { return DATAPTR(Materialize(alt)); }
@@ -800,41 +868,35 @@ struct AltrepVectorString : public AltrepVectorBase<AltrepVectorString<Type>> {
       return Representation(alt);
     }
 
-    BEGIN_CPP11
-
     const auto& chunked_array = GetChunkedArray(alt);
     SEXP data2 = PROTECT(Rf_allocVector(STRSXP, chunked_array->length()));
     MARK_NOT_MUTABLE(data2);
 
-    RStringViewer r_string_viewer;
-
-    // r_string_viewer.Convert() might jump so we have to
-    // wrap it in unwind_protect() to:
-    // - correctly destruct the C++ objects
-    // - resume the unwinding
-    cpp11::unwind_protect([&]() {
-      R_xlen_t i = 0;
-      for (const auto& array : chunked_array->chunks()) {
-        r_string_viewer.SetArray(array);
+    R_xlen_t i = 0;
+    RStringViewer& r_string_viewer = string_viewer();
+    r_string_viewer.reset_null_was_stripped();
+    r_string_viewer.set_strip_out_nuls(GetBoolOption("arrow.skip_nul", false));
+    for (const auto& array : chunked_array->chunks()) {
+      r_string_viewer.SetArray(array);
 
-        auto ni = array->length();
-        for (R_xlen_t j = 0; j < ni; j++, i++) {
-          SET_STRING_ELT(data2, i, r_string_viewer.Convert(j));
-        }
+      auto ni = array->length();
+      for (R_xlen_t j = 0; j < ni; j++, i++) {
+        SET_STRING_ELT(data2, i, r_string_viewer.Convert(j));
       }
+    }
 
-      if (r_string_viewer.nul_was_stripped()) {
-        cpp11::warning("Stripping '\\0' (nul) from character vector");
-      }
-    });
+    if (r_string_viewer.nul_was_stripped()) {
+      Rf_warning("Stripping '\\0' (nul) from character vector");
+    }
 
     // only set to data2 if all the values have been converted
     SetRepresentation(alt, data2);
     UNPROTECT(1);  // data2
 
-    return data2;
+    // remove reference to chunked array
+    R_set_altrep_data1(alt, R_NilValue);
 
-    END_CPP11
+    return data2;
   }
 
   static const void* Dataptr_or_null(SEXP alt) {
@@ -847,6 +909,11 @@ struct AltrepVectorString : public AltrepVectorBase<AltrepVectorString<Type>> {
   static void Set_elt(SEXP alt, R_xlen_t i, SEXP v) {
     Rf_error("ALTSTRING objects of type <arrow::array_string_vector> are immutable");
   }
+
+  static RStringViewer& string_viewer() {
+    static RStringViewer string_viewer;
+    return string_viewer;
+  }
 };
 
 template <typename Type>
@@ -986,7 +1053,7 @@ bool is_arrow_altrep(SEXP x) {
 }
 
 std::shared_ptr<ChunkedArray> vec_to_arrow_altrep_bypass(SEXP x) {
-  if (is_arrow_altrep(x)) {
+  if (is_arrow_altrep(x) && R_altrep_data1(x) != R_NilValue) {
     return GetChunkedArray(x);
   }
 
@@ -1019,7 +1086,170 @@ std::shared_ptr<ChunkedArray> vec_to_arrow_altrep_bypass(SEXP x) { return nullpt
 #endif
 
 // [[arrow::export]]
-void test_SET_STRING_ELT(SEXP s) { SET_STRING_ELT(s, 0, Rf_mkChar("forbidden")); }
+bool is_arrow_altrep(cpp11::sexp x) { return arrow::r::altrep::is_arrow_altrep(x); }
+
+// [[arrow::export]]
+void test_arrow_altrep_set_string_elt(sexp x, int i, std::string value) {
+  if (!is_arrow_altrep(x)) {
+    stop("x is not arrow ALTREP");
+  }
+
+  SET_STRING_ELT(x, i, Rf_mkChar(value.c_str()));
+}
+
+// [[arrow::export]]
+logicals test_arrow_altrep_is_materialized(sexp x) {
+  if (!is_arrow_altrep(x)) {
+    return logicals({NA_LOGICAL});
+  }
+
+  sexp data_class_sym = CAR(ATTRIB(ALTREP_CLASS(x)));
+  std::string class_name(CHAR(PRINTNAME(data_class_sym)));
+
+  int result = NA_LOGICAL;
+  if (class_name == "arrow::array_dbl_vector") {
+    result = arrow::r::altrep::AltrepVectorPrimitive<REALSXP>::IsMaterialized(x);
+  } else if (class_name == "arrow::array_int_vector") {
+    result = arrow::r::altrep::AltrepVectorPrimitive<INTSXP>::IsMaterialized(x);
+  } else if (class_name == "arrow::array_string_vector") {
+    result = arrow::r::altrep::AltrepVectorString<arrow::StringType>::IsMaterialized(x);
+  } else if (class_name == "arrow::array_large_string_vector") {
+    result =
+        arrow::r::altrep::AltrepVectorString<arrow::LargeStringType>::IsMaterialized(x);
+  } else if (class_name == "arrow::array_factor") {
+    result = arrow::r::altrep::AltrepFactor::IsMaterialized(x);
+  }
+
+  return logicals({result});
+}
+
+// [[arrow::export]]
+bool test_arrow_altrep_force_materialize(sexp x) {
+  if (!is_arrow_altrep(x)) {
+    stop("x is not arrow ALTREP");
+  }
+
+  bool already_materialized = as_cpp<bool>(test_arrow_altrep_is_materialized(x));
+  if (already_materialized) {
+    stop("x is already materialized");
+  }
+
+  sexp data_class_sym = CAR(ATTRIB(ALTREP_CLASS(x)));
+  std::string class_name(CHAR(PRINTNAME(data_class_sym)));
+
+  if (class_name == "arrow::array_dbl_vector") {
+    arrow::r::altrep::AltrepVectorPrimitive<REALSXP>::Materialize(x);
+  } else if (class_name == "arrow::array_int_vector") {
+    arrow::r::altrep::AltrepVectorPrimitive<INTSXP>::Materialize(x);
+  } else if (class_name == "arrow::array_string_vector") {
+    arrow::r::altrep::AltrepVectorString<arrow::StringType>::Materialize(x);
+  } else if (class_name == "arrow::array_large_string_vector") {
+    arrow::r::altrep::AltrepVectorString<arrow::LargeStringType>::Materialize(x);
+  } else if (class_name == "arrow::array_factor") {
+    arrow::r::altrep::AltrepFactor::Materialize(x);
+  } else {
+    return false;
+  }
+
+  return true;
+}
+
+// [[arrow::export]]
+sexp test_arrow_altrep_copy_by_element(sexp x) {
+  if (!is_arrow_altrep(x)) {
+    stop("x is not arrow ALTREP");
+  }
+
+  R_xlen_t n = Rf_xlength(x);
+
+  if (TYPEOF(x) == INTSXP) {
+    writable::integers out(Rf_xlength(x));
+    for (R_xlen_t i = 0; i < n; i++) {
+      out[i] = INTEGER_ELT(x, i);
+    }
+    return out;
+  } else if (TYPEOF(x) == REALSXP) {
+    writable::doubles out(Rf_xlength(x));
+    for (R_xlen_t i = 0; i < n; i++) {
+      out[i] = REAL_ELT(x, i);
+    }
+    return out;
+  } else if (TYPEOF(x) == STRSXP) {
+    writable::strings out(Rf_xlength(x));
+    for (R_xlen_t i = 0; i < n; i++) {
+      out[i] = STRING_ELT(x, i);
+    }
+    return out;
+  } else {
+    return R_NilValue;
+  }
+}
 
 // [[arrow::export]]
-bool is_arrow_altrep(SEXP x) { return arrow::r::altrep::is_arrow_altrep(x); }
+sexp test_arrow_altrep_copy_by_region(sexp x, R_xlen_t region_size) {
+  if (!is_arrow_altrep(x)) {
+    stop("x is not arrow ALTREP");
+  }
+
+  R_xlen_t n = Rf_xlength(x);
+
+  if (TYPEOF(x) == INTSXP) {
+    writable::integers out(Rf_xlength(x));
+    writable::integers buf_shelter(region_size);
+    int* buf = INTEGER(buf_shelter);
+    for (R_xlen_t i = 0; i < n; i++) {
+      if ((i % region_size) == 0) {
+        INTEGER_GET_REGION(x, i, region_size, buf);
+      }
+      out[i] = buf[i % region_size];
+    }
+    return out;
+  } else if (TYPEOF(x) == REALSXP) {
+    writable::doubles out(Rf_xlength(x));
+    writable::doubles buf_shelter(region_size);
+    double* buf = REAL(buf_shelter);
+    for (R_xlen_t i = 0; i < n; i++) {
+      if ((i % region_size) == 0) {
+        REAL_GET_REGION(x, i, region_size, buf);
+      }
+      out[i] = buf[i % region_size];
+    }
+    return out;
+  } else {
+    return R_NilValue;
+  }
+}
+
+// [[arrow::export]]
+sexp test_arrow_altrep_copy_by_dataptr(sexp x) {
+  if (!is_arrow_altrep(x)) {
+    stop("x is not arrow ALTREP");
+  }
+
+  R_xlen_t n = Rf_xlength(x);
+
+  if (TYPEOF(x) == INTSXP) {
+    writable::integers out(Rf_xlength(x));
+    int* ptr = reinterpret_cast<int*>(DATAPTR(x));
+    for (R_xlen_t i = 0; i < n; i++) {
+      out[i] = ptr[i];
+    }
+    return out;
+  } else if (TYPEOF(x) == REALSXP) {
+    writable::doubles out(Rf_xlength(x));
+    double* ptr = reinterpret_cast<double*>(DATAPTR(x));
+    for (R_xlen_t i = 0; i < n; i++) {
+      out[i] = ptr[i];
+    }
+    return out;
+  } else if (TYPEOF(x) == STRSXP) {
+    writable::strings out(Rf_xlength(x));
+    SEXP* ptr = reinterpret_cast<SEXP*>(DATAPTR(x));
+    for (R_xlen_t i = 0; i < n; i++) {
+      out[i] = ptr[i];
+    }
+    return out;
+  } else {
+    return R_NilValue;
+  }
+}
diff --git a/r/src/array.cpp b/r/src/array.cpp
index 8ba0c569ea..608c0d5108 100644
--- a/r/src/array.cpp
+++ b/r/src/array.cpp
@@ -69,7 +69,10 @@ void arrow::r::validate_slice_length(R_xlen_t length, int64_t available) {
     cpp11::stop("Slice 'length' cannot be negative");
   }
   if (length > available) {
-    cpp11::warning("Slice 'length' greater than available length");
+    // For an unknown reason, cpp11::warning() crashes here; however, this
+    // should throw an exception if Rf_warning() jumps, so we need
+    // cpp11::safe[]().
+    cpp11::safe[Rf_warning]("Slice 'length' greater than available length");
   }
 }
 
diff --git a/r/src/array_to_vector.cpp b/r/src/array_to_vector.cpp
index d7c51e7935..9cf539bb41 100644
--- a/r/src/array_to_vector.cpp
+++ b/r/src/array_to_vector.cpp
@@ -365,7 +365,7 @@ struct Converter_String : public Converter {
     }
 
     if (nul_was_stripped) {
-      cpp11::warning("Stripping '\\0' (nul) from character vector");
+      cpp11::safe[Rf_warning]("Stripping '\\0' (nul) from character vector");
     }
 
     return Status::OK();
@@ -706,7 +706,7 @@ class Converter_Dictionary : public Converter {
     // Alternative: preserve the logical type of the dictionary values
     // (e.g. if dict is timestamp, return a POSIXt R vector, not factor)
     if (dictionary_->type_id() != Type::STRING) {
-      cpp11::warning("Coercing dictionary values to R character factor levels");
+      cpp11::safe[Rf_warning]("Coercing dictionary values to R character factor levels");
     }
 
     SEXP vec = PROTECT(Converter::Convert(dictionary_));
diff --git a/r/src/arrowExports.cpp b/r/src/arrowExports.cpp
index cde8795c9f..996251a05b 100644
--- a/r/src/arrowExports.cpp
+++ b/r/src/arrowExports.cpp
@@ -5,20 +5,63 @@
 #include "./arrow_types.h"
 
 // altrep.cpp
-void test_SET_STRING_ELT(SEXP s);
-extern "C" SEXP _arrow_test_SET_STRING_ELT(SEXP s_sexp){
+bool is_arrow_altrep(cpp11::sexp x);
+extern "C" SEXP _arrow_is_arrow_altrep(SEXP x_sexp){
 BEGIN_CPP11
-	arrow::r::Input<SEXP>::type s(s_sexp);
-	test_SET_STRING_ELT(s);
+	arrow::r::Input<cpp11::sexp>::type x(x_sexp);
+	return cpp11::as_sexp(is_arrow_altrep(x));
+END_CPP11
+}
+// altrep.cpp
+void test_arrow_altrep_set_string_elt(sexp x, int i, std::string value);
+extern "C" SEXP _arrow_test_arrow_altrep_set_string_elt(SEXP x_sexp, SEXP i_sexp, SEXP value_sexp){
+BEGIN_CPP11
+	arrow::r::Input<sexp>::type x(x_sexp);
+	arrow::r::Input<int>::type i(i_sexp);
+	arrow::r::Input<std::string>::type value(value_sexp);
+	test_arrow_altrep_set_string_elt(x, i, value);
 	return R_NilValue;
 END_CPP11
 }
 // altrep.cpp
-bool is_arrow_altrep(SEXP x);
-extern "C" SEXP _arrow_is_arrow_altrep(SEXP x_sexp){
+logicals test_arrow_altrep_is_materialized(sexp x);
+extern "C" SEXP _arrow_test_arrow_altrep_is_materialized(SEXP x_sexp){
 BEGIN_CPP11
-	arrow::r::Input<SEXP>::type x(x_sexp);
-	return cpp11::as_sexp(is_arrow_altrep(x));
+	arrow::r::Input<sexp>::type x(x_sexp);
+	return cpp11::as_sexp(test_arrow_altrep_is_materialized(x));
+END_CPP11
+}
+// altrep.cpp
+bool test_arrow_altrep_force_materialize(sexp x);
+extern "C" SEXP _arrow_test_arrow_altrep_force_materialize(SEXP x_sexp){
+BEGIN_CPP11
+	arrow::r::Input<sexp>::type x(x_sexp);
+	return cpp11::as_sexp(test_arrow_altrep_force_materialize(x));
+END_CPP11
+}
+// altrep.cpp
+sexp test_arrow_altrep_copy_by_element(sexp x);
+extern "C" SEXP _arrow_test_arrow_altrep_copy_by_element(SEXP x_sexp){
+BEGIN_CPP11
+	arrow::r::Input<sexp>::type x(x_sexp);
+	return cpp11::as_sexp(test_arrow_altrep_copy_by_element(x));
+END_CPP11
+}
+// altrep.cpp
+sexp test_arrow_altrep_copy_by_region(sexp x, R_xlen_t region_size);
+extern "C" SEXP _arrow_test_arrow_altrep_copy_by_region(SEXP x_sexp, SEXP region_size_sexp){
+BEGIN_CPP11
+	arrow::r::Input<sexp>::type x(x_sexp);
+	arrow::r::Input<R_xlen_t>::type region_size(region_size_sexp);
+	return cpp11::as_sexp(test_arrow_altrep_copy_by_region(x, region_size));
+END_CPP11
+}
+// altrep.cpp
+sexp test_arrow_altrep_copy_by_dataptr(sexp x);
+extern "C" SEXP _arrow_test_arrow_altrep_copy_by_dataptr(SEXP x_sexp){
+BEGIN_CPP11
+	arrow::r::Input<sexp>::type x(x_sexp);
+	return cpp11::as_sexp(test_arrow_altrep_copy_by_dataptr(x));
 END_CPP11
 }
 // array.cpp
@@ -5234,8 +5277,13 @@ static const R_CallMethodDef CallEntries[] = {
 		{ "_s3_available", (DL_FUNC)& _s3_available, 0 },
 		{ "_gcs_available", (DL_FUNC)& _gcs_available, 0 },
 		{ "_json_available", (DL_FUNC)& _json_available, 0 },
-		{ "_arrow_test_SET_STRING_ELT", (DL_FUNC) &_arrow_test_SET_STRING_ELT, 1}, 
 		{ "_arrow_is_arrow_altrep", (DL_FUNC) &_arrow_is_arrow_altrep, 1}, 
+		{ "_arrow_test_arrow_altrep_set_string_elt", (DL_FUNC) &_arrow_test_arrow_altrep_set_string_elt, 3}, 
+		{ "_arrow_test_arrow_altrep_is_materialized", (DL_FUNC) &_arrow_test_arrow_altrep_is_materialized, 1}, 
+		{ "_arrow_test_arrow_altrep_force_materialize", (DL_FUNC) &_arrow_test_arrow_altrep_force_materialize, 1}, 
+		{ "_arrow_test_arrow_altrep_copy_by_element", (DL_FUNC) &_arrow_test_arrow_altrep_copy_by_element, 1}, 
+		{ "_arrow_test_arrow_altrep_copy_by_region", (DL_FUNC) &_arrow_test_arrow_altrep_copy_by_region, 2}, 
+		{ "_arrow_test_arrow_altrep_copy_by_dataptr", (DL_FUNC) &_arrow_test_arrow_altrep_copy_by_dataptr, 1}, 
 		{ "_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}, 
diff --git a/r/tests/testthat/test-RecordBatch.R b/r/tests/testthat/test-RecordBatch.R
index 6b79325934..7e7084542d 100644
--- a/r/tests/testthat/test-RecordBatch.R
+++ b/r/tests/testthat/test-RecordBatch.R
@@ -642,14 +642,19 @@ test_that("Handling string data with embedded nuls", {
   batch_with_nul$b <- batch_with_nul$b$cast(utf8())
 
   withr::with_options(list(arrow.skip_nul = TRUE), {
-    expect_warning(
-      expect_equal(
-        as.data.frame(batch_with_nul)$b,
-        c("person", "woman", "man", "camera", "tv"),
-        ignore_attr = TRUE
-      ),
-      "Stripping '\\0' (nul) from character vector",
-      fixed = TRUE
+    # Because expect_equal() may call identical(x, y) more than once,
+    # the string with a nul may be created more than once and multiple
+    # warnings may be issued.
+    suppressWarnings(
+      expect_warning(
+        expect_equal(
+          as.data.frame(batch_with_nul)$b,
+          c("person", "woman", "man", "camera", "tv"),
+          ignore_attr = TRUE
+        ),
+        "Stripping '\\0' (nul) from character vector",
+        fixed = TRUE
+      )
     )
   })
 })
diff --git a/r/tests/testthat/test-altrep.R b/r/tests/testthat/test-altrep.R
index cd1d841c42..4eb316d446 100644
--- a/r/tests/testthat/test-altrep.R
+++ b/r/tests/testthat/test-altrep.R
@@ -17,8 +17,34 @@
 
 skip_on_r_older_than("3.6")
 
-test_that("is_arrow_altrep() does not include base altrep", {
+test_that("altrep test functions do not include base altrep", {
   expect_false(is_arrow_altrep(1:10))
+  expect_identical(test_arrow_altrep_is_materialized(1:10), NA)
+  expect_error(
+    test_arrow_altrep_force_materialize(1:10),
+    "is not arrow ALTREP"
+  )
+  expect_error(
+    test_arrow_altrep_copy_by_element(1:10),
+    "is not arrow ALTREP"
+  )
+  expect_error(
+    test_arrow_altrep_copy_by_region(1:10, 1024),
+    "is not arrow ALTREP"
+  )
+  expect_error(
+    test_arrow_altrep_copy_by_dataptr(1:10),
+    "is not arrow ALTREP"
+  )
+})
+
+test_that(".Internal(inspect()) prints out Arrow altrep info", {
+  withr::local_options(list(arrow.use_altrep = TRUE))
+  alt <- as.vector(Array$create(1:1000))
+
+  expect_output(.Internal(inspect(alt)), "\\] arrow::array_int_vector")
+  expect_true(test_arrow_altrep_force_materialize(alt))
+  expect_output(.Internal(inspect(alt)), "materialized arrow::array_int_vector")
 })
 
 test_that("altrep vectors from int32 and dbl arrays with no nulls", {
@@ -54,6 +80,54 @@ test_that("altrep vectors from int32 and dbl arrays with no nulls", {
   expect_false(is_arrow_altrep(as.vector(v_dbl$Slice(1))))
 })
 
+test_that("element access methods for int32 ALTREP with no nulls", {
+  withr::local_options(list(arrow.use_altrep = TRUE))
+  original <- 1:1000
+  v_int <- Array$create(original)
+  altrep <- as.vector(v_int)
+  expect_false(test_arrow_altrep_is_materialized(altrep))
+
+  # altrep-aware iterating should not materialize
+  expect_identical(test_arrow_altrep_copy_by_element(altrep), original)
+  expect_identical(test_arrow_altrep_copy_by_region(altrep, 123), original)
+  expect_false(test_arrow_altrep_is_materialized(altrep))
+
+  # because there are no nulls, DATAPTR() does not materialize
+  expect_identical(test_arrow_altrep_copy_by_dataptr(altrep), original)
+  expect_false(test_arrow_altrep_is_materialized(altrep))
+
+  # test element access after forcing materialization
+  expect_true(test_arrow_altrep_force_materialize(altrep))
+  expect_true(test_arrow_altrep_is_materialized(altrep))
+  expect_identical(test_arrow_altrep_copy_by_element(altrep), original)
+  expect_identical(test_arrow_altrep_copy_by_region(altrep, 123), original)
+  expect_identical(test_arrow_altrep_copy_by_dataptr(altrep), original)
+})
+
+test_that("element access methods for double ALTREP with no nulls", {
+  withr::local_options(list(arrow.use_altrep = TRUE))
+  original <- as.double(1:1000)
+  v_dbl <- Array$create(original)
+  altrep <- as.vector(v_dbl)
+  expect_false(test_arrow_altrep_is_materialized(altrep))
+
+  # altrep-aware iterating should not materialize
+  expect_identical(test_arrow_altrep_copy_by_element(altrep), original)
+  expect_identical(test_arrow_altrep_copy_by_region(altrep, 123), original)
+  expect_false(test_arrow_altrep_is_materialized(altrep))
+
+  # because there are no nulls, DATAPTR() does not materialize
+  expect_identical(test_arrow_altrep_copy_by_dataptr(altrep), original)
+  expect_false(test_arrow_altrep_is_materialized(altrep))
+
+  # test element access after forcing materialization
+  expect_true(test_arrow_altrep_force_materialize(altrep))
+  expect_true(test_arrow_altrep_is_materialized(altrep))
+  expect_identical(test_arrow_altrep_copy_by_element(altrep), original)
+  expect_identical(test_arrow_altrep_copy_by_region(altrep, 123), original)
+  expect_identical(test_arrow_altrep_copy_by_dataptr(altrep), original)
+})
+
 test_that("altrep vectors from int32 and dbl arrays with nulls", {
   withr::local_options(list(arrow.use_altrep = TRUE))
   v_int <- Array$create(c(1L, NA, 3L))
@@ -75,7 +149,6 @@ test_that("altrep vectors from int32 and dbl arrays with nulls", {
   expect_true(is_arrow_altrep(as.vector(c_int$Slice(2))))
   expect_true(is_arrow_altrep(as.vector(c_dbl$Slice(2))))
 
-  # chunked array with 2 chunks cannot be altrep
   c_int <- ChunkedArray$create(0L, c(1L, NA, 3L))
   c_dbl <- ChunkedArray$create(0, c(1, NA, 3))
   expect_equal(c_int$num_chunks, 2L)
@@ -87,6 +160,123 @@ test_that("altrep vectors from int32 and dbl arrays with nulls", {
   expect_true(is_arrow_altrep(as.vector(c_dbl$Slice(3))))
 })
 
+test_that("element access methods for int32 ALTREP with nulls", {
+  withr::local_options(list(arrow.use_altrep = TRUE))
+  original <- c(NA, 1:1000)
+  v_int <- Array$create(original)
+  altrep <- as.vector(v_int)
+  expect_false(test_arrow_altrep_is_materialized(altrep))
+
+  # altrep-aware iterating should not materialize
+  expect_identical(test_arrow_altrep_copy_by_element(altrep), original)
+  expect_identical(test_arrow_altrep_copy_by_region(altrep, 123), original)
+  expect_false(test_arrow_altrep_is_materialized(altrep))
+
+  # because there are no nulls, DATAPTR() does not materialize
+  expect_identical(test_arrow_altrep_copy_by_dataptr(altrep), original)
+  expect_true(test_arrow_altrep_is_materialized(altrep))
+
+  # test element access after materialization
+  expect_true(test_arrow_altrep_is_materialized(altrep))
+  expect_identical(test_arrow_altrep_copy_by_element(altrep), original)
+  expect_identical(test_arrow_altrep_copy_by_region(altrep, 123), original)
+  expect_identical(test_arrow_altrep_copy_by_dataptr(altrep), original)
+})
+
+test_that("element access methods for double ALTREP with nulls", {
+  withr::local_options(list(arrow.use_altrep = TRUE))
+  original <- as.double(c(NA, 1:1000))
+  v_dbl <- Array$create(original)
+  altrep <- as.vector(v_dbl)
+  expect_false(test_arrow_altrep_is_materialized(altrep))
+
+  # altrep-aware iterating should not materialize
+  expect_identical(test_arrow_altrep_copy_by_element(altrep), original)
+  expect_identical(test_arrow_altrep_copy_by_region(altrep, 123), original)
+  expect_false(test_arrow_altrep_is_materialized(altrep))
+
+  # because there are no nulls, DATAPTR() does not materialize
+  expect_identical(test_arrow_altrep_copy_by_dataptr(altrep), original)
+  expect_true(test_arrow_altrep_is_materialized(altrep))
+
+  # test element access after materialization
+  expect_true(test_arrow_altrep_is_materialized(altrep))
+  expect_identical(test_arrow_altrep_copy_by_element(altrep), original)
+  expect_identical(test_arrow_altrep_copy_by_region(altrep, 123), original)
+  expect_identical(test_arrow_altrep_copy_by_dataptr(altrep), original)
+})
+
+test_that("altrep vectors from string arrays", {
+  withr::local_options(list(arrow.use_altrep = TRUE))
+  v_chr <- Array$create(c("one", NA, "three"))
+  c_chr <- ChunkedArray$create(c("one", NA, "three"))
+
+  expect_true(is_arrow_altrep(as.vector(v_chr)))
+  expect_true(is_arrow_altrep(as.vector(v_chr$Slice(1))))
+  expect_true(is_arrow_altrep(as.vector(c_chr)))
+  expect_true(is_arrow_altrep(as.vector(c_chr$Slice(1))))
+
+  expect_true(is_arrow_altrep(as.vector(v_chr$Slice(2))))
+  expect_true(is_arrow_altrep(as.vector(c_chr$Slice(2))))
+
+  c_chr <- ChunkedArray$create("zero", c("one", NA, "three"))
+  expect_equal(c_chr$num_chunks, 2L)
+
+  expect_true(is_arrow_altrep(as.vector(c_chr)))
+  expect_true(is_arrow_altrep(as.vector(c_chr$Slice(3))))
+})
+
+test_that("can't SET_STRING_ELT() on character ALTREP", {
+  withr::local_options(list(arrow.use_altrep = TRUE))
+  alt <- as.vector(Array$create(c("one", "two", "three")))
+  expect_error(
+    test_arrow_altrep_set_string_elt(alt, 0, "value"),
+    "are immutable"
+  )
+})
+
+test_that("element access methods for character ALTREP", {
+  withr::local_options(list(arrow.use_altrep = TRUE))
+  original <- as.character(c(NA, 1:1000))
+  v_chr <- Array$create(original)
+  altrep <- as.vector(v_chr)
+  expect_false(test_arrow_altrep_is_materialized(altrep))
+
+  # altrep-aware iterating should not materialize
+  expect_identical(test_arrow_altrep_copy_by_element(altrep), original)
+  expect_false(test_arrow_altrep_is_materialized(altrep))
+
+  # DATAPTR() should always materialize for strings
+  expect_identical(test_arrow_altrep_copy_by_dataptr(altrep), original)
+  expect_true(test_arrow_altrep_is_materialized(altrep))
+
+  # test element access after materialization
+  expect_true(test_arrow_altrep_is_materialized(altrep))
+  expect_identical(test_arrow_altrep_copy_by_element(altrep), original)
+  expect_identical(test_arrow_altrep_copy_by_dataptr(altrep), original)
+})
+
+test_that("element access methods for character ALTREP from large_utf8()", {
+  withr::local_options(list(arrow.use_altrep = TRUE))
+  original <- as.character(c(NA, 1:1000))
+  v_chr <- Array$create(original, type = large_utf8())
+  altrep <- as.vector(v_chr)
+  expect_false(test_arrow_altrep_is_materialized(altrep))
+
+  # altrep-aware iterating should not materialize
+  expect_identical(test_arrow_altrep_copy_by_element(altrep), original)
+  expect_false(test_arrow_altrep_is_materialized(altrep))
+
+  # DATAPTR() should always materialize for strings
+  expect_identical(test_arrow_altrep_copy_by_dataptr(altrep), original)
+  expect_true(test_arrow_altrep_is_materialized(altrep))
+
+  # test element access after materialization
+  expect_true(test_arrow_altrep_is_materialized(altrep))
+  expect_identical(test_arrow_altrep_copy_by_element(altrep), original)
+  expect_identical(test_arrow_altrep_copy_by_dataptr(altrep), original)
+})
+
 test_that("empty vectors are not altrep", {
   withr::local_options(list(arrow.use_altrep = TRUE))
   v_int <- Array$create(integer())
@@ -149,20 +339,28 @@ test_that("as.data.frame(<Table>, <RecordBatch>) can create altrep vectors", {
 })
 
 expect_altrep_roundtrip <- function(x, fn, ..., .expect_warning = NA) {
+  # check altrep Array
   alt <- Array$create(x)$as_vector()
-
   expect_true(is_arrow_altrep(alt))
   expect_warning(
-    expect_identical(fn(x, ...), fn(alt, ...)), .expect_warning
+    expect_identical(fn(alt, ...), suppressWarnings(fn(x, ...))), .expect_warning
   )
-  expect_true(is_arrow_altrep(alt))
+  expect_false(test_arrow_altrep_is_materialized(alt))
 
+  # check altrep ChunkedArray
   alt2 <- ChunkedArray$create(x, x)$as_vector()
   expect_true(is_arrow_altrep(alt2))
   expect_warning(
-    expect_identical(fn(c(x, x), ...), fn(alt2, ...)), .expect_warning
+    expect_identical(fn(alt2, ...), suppressWarnings(fn(c(x, x), ...))), .expect_warning
+  )
+  expect_false(test_arrow_altrep_is_materialized(alt2))
+
+  # Check materialized altrep
+  alt3 <- Array$create(x)$as_vector()
+  expect_true(test_arrow_altrep_force_materialize(alt3))
+  expect_warning(
+    expect_identical(fn(alt3, ...), suppressWarnings(fn(x, ...))), .expect_warning
   )
-  expect_true(is_arrow_altrep(alt2))
 }
 
 test_that("altrep min/max/sum identical to R versions for double", {
@@ -231,17 +429,20 @@ test_that("altrep vectors handle serialization", {
   ints <- c(1L, 2L, NA_integer_)
   dbls <- c(1, 2, NA_real_)
   strs <- c("un", "deux", NA_character_)
+  fctrs <- as.factor(strs)
 
   expect_identical(ints, unserialize(serialize(Array$create(ints)$as_vector(), NULL)))
   expect_identical(dbls, unserialize(serialize(Array$create(dbls)$as_vector(), NULL)))
   expect_identical(strs, unserialize(serialize(Array$create(strs)$as_vector(), NULL)))
   expect_identical(strs, unserialize(serialize(Array$create(strs, large_utf8())$as_vector(), NULL)))
+  expect_identical(fctrs, unserialize(serialize(Array$create(fctrs)$as_vector(), NULL)))
 })
 
 test_that("altrep vectors handle coercion", {
   ints <- c(1L, 2L, NA_integer_)
   dbls <- c(1, 2, NA_real_)
   strs <- c("1", "2", NA_character_)
+  fctrs <- as.factor(strs)
 
   expect_identical(ints, as.integer(Array$create(dbls)$as_vector()))
   expect_identical(ints, as.integer(Array$create(strs)$as_vector()))
@@ -251,6 +452,9 @@ test_that("altrep vectors handle coercion", {
 
   expect_identical(strs, as.character(Array$create(ints)$as_vector()))
   expect_identical(strs, as.character(Array$create(dbls)$as_vector()))
+
+  expect_identical(fctrs, as.factor(Array$create(fctrs)$as_vector()))
+  expect_identical(strs, as.character(Array$create(fctrs)$as_vector()))
 })
 
 test_that("columns of struct types may be altrep", {
@@ -353,6 +557,56 @@ test_that("dictionaries chunked arrays are made altrep", {
   expect_equal(as.integer(f), c(1L, 2L, 3L, 4L, 1L, NA_integer_, 5L))
 })
 
+test_that("element access methods for ALTREP factors", {
+  index_types <- list(int8(), uint8(), int16(), uint16(), int32(), uint32())
+
+  for (index_type in index_types) {
+    # without unification
+    int_indices <- c(1L, 2L, 4L, 3L, 1L, NA_integer_, 5L)
+    x <- ChunkedArray$create(
+      factor(c("a", "b"), levels = letters[1:5]),
+      factor(c("d", "c", "a", NA, "e"), levels = letters[1:5]),
+      type = dictionary(index_type, string())
+    )
+    f <- x$as_vector()
+    expect_true(is_arrow_altrep(f))
+    # This may fail interactively because str() currently
+    # calls unclass(f), which calls our duplicate method
+    expect_false(test_arrow_altrep_is_materialized(f))
+
+    expect_identical(test_arrow_altrep_copy_by_element(f), int_indices)
+    expect_identical(test_arrow_altrep_copy_by_region(f, 3), int_indices)
+    expect_false(test_arrow_altrep_is_materialized(f))
+
+    expect_identical(test_arrow_altrep_copy_by_dataptr(f), int_indices)
+    expect_true(test_arrow_altrep_is_materialized(f))
+
+    expect_identical(test_arrow_altrep_copy_by_element(f), int_indices)
+    expect_identical(test_arrow_altrep_copy_by_region(f, 3), int_indices)
+
+    # with unification
+    int_indices <- c(1L, 2L, 3L, 4L, 1L, NA_integer_, 5L)
+    x <- ChunkedArray$create(
+      factor(c("a", "b"), levels = c("a", "b")),
+      factor(c("d", "c", "a", NA, "e"), levels = c("d", "c", "a", "e")),
+      type = dictionary(index_type, string())
+    )
+    f <- x$as_vector()
+    expect_true(is_arrow_altrep(f))
+    expect_false(test_arrow_altrep_is_materialized(f))
+
+    expect_identical(test_arrow_altrep_copy_by_element(f), int_indices)
+    expect_identical(test_arrow_altrep_copy_by_region(f, 3), int_indices)
+    expect_false(test_arrow_altrep_is_materialized(f))
+
+    expect_identical(test_arrow_altrep_copy_by_dataptr(f), int_indices)
+    expect_true(test_arrow_altrep_is_materialized(f))
+
+    expect_identical(test_arrow_altrep_copy_by_element(f), int_indices)
+    expect_identical(test_arrow_altrep_copy_by_region(f, 3), int_indices)
+  }
+})
+
 
 test_that("R checks for bounds", {
   v_int <- Array$create(c(1, 2, 3))$as_vector()