You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by GitBox <gi...@apache.org> on 2022/11/01 19:08:29 UTC

[GitHub] [arrow-nanoarrow] paleolimbot opened a new pull request, #65: [R] Complete ptype inferences and array conversions

paleolimbot opened a new pull request, #65:
URL: https://github.com/apache/arrow-nanoarrow/pull/65

   Type conversions that were missing were:
   
   - datetime types: time32, time64, timestamp, duration
   - nested: list, large_list, fixed_size_list


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

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

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


[GitHub] [arrow-nanoarrow] paleolimbot commented on pull request #65: [R] Complete ptype inferences and array conversions

Posted by GitBox <gi...@apache.org>.
paleolimbot commented on PR #65:
URL: https://github.com/apache/arrow-nanoarrow/pull/65#issuecomment-1302277227

   Still needs testing with  more datasets, but these conversions are potentially much faster than the current arrow R package's:
   
   ``` r
   # remotes::install_github("apache/arrow-nanoarrow/r#65")
   library(nanoarrow)
   # latest master (i.e., with latest ALTREP improvement PR)
   library(arrow, warn.conflicts = FALSE)
   #> Some features are not enabled in this build of Arrow. Run `arrow_info()` for more information.
   library(nycflights13)
   
   flights <- nycflights13::flights
   # until nanoarrow converts datetimes
   flights$time_hour <- NULL
   
   
   flights_table <- as_arrow_table(flights)
   flights_array <- as_nanoarrow_array(flights_table)
   n <- nrow(flights)
   
   # with altrep, arrow is faster
   bench::mark(
     arrow_altrep = as.data.frame(as.data.frame(flights_table)),
     nanoarrow = as.data.frame(as.data.frame(flights_array))
   )
   #> # A tibble: 2 × 6
   #>   expression        min   median `itr/sec` mem_alloc `gc/sec`
   #>   <bch:expr>   <bch:tm> <bch:tm>     <dbl> <bch:byt>    <dbl>
   #> 1 arrow_altrep    323µs 333.66µs     2923.     847KB     30.7
   #> 2 nanoarrow      2.12ms   2.56ms      384.    25.8MB    370.
   
   # with materialization nanoarrow is much faster?
   bench::mark(
     arrow_altrep = as.data.frame(as.data.frame(flights_table))[n:1, ],
     nanoarrow = as.data.frame(as.data.frame(flights_array))[n:1, ],
     min_iterations = 5
   )
   #> # A tibble: 2 × 6
   #>   expression        min   median `itr/sec` mem_alloc `gc/sec`
   #>   <bch:expr>   <bch:tm> <bch:tm>     <dbl> <bch:byt>    <dbl>
   #> 1 arrow_altrep   80.3ms   80.3ms      12.5    42.5MB     74.7
   #> 2 nanoarrow      42.7ms     43ms      23.2    68.2MB    151.
   
   # without altrep, nanoarrow is much much much faster?
   withr::with_options(list(arrow.use_altrep = FALSE), {
     bench::mark(
       arrow_no_altrep = as.data.frame(as.data.frame(flights_table)),
       nanoarrow = as.data.frame(as.data.frame(flights_array)),
       min_iterations = 5
     )
   })
   #> # A tibble: 2 × 6
   #>   expression           min   median `itr/sec` mem_alloc `gc/sec`
   #>   <bch:expr>      <bch:tm> <bch:tm>     <dbl> <bch:byt>    <dbl>
   #> 1 arrow_no_altrep  20.95ms  21.56ms      46.6      36MB     25.1
   #> 2 nanoarrow         2.06ms   2.56ms     384.     25.7MB    136.
   ```
   
   <sup>Created on 2022-11-03 with [reprex v2.0.2](https://reprex.tidyverse.org)</sup>


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

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

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


[GitHub] [arrow-nanoarrow] romainfrancois commented on a diff in pull request #65: [R] Complete ptype inferences and array conversions

Posted by GitBox <gi...@apache.org>.
romainfrancois commented on code in PR #65:
URL: https://github.com/apache/arrow-nanoarrow/pull/65#discussion_r1024134238


##########
r/src/convert.c:
##########
@@ -0,0 +1,483 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#define R_NO_REMAP
+#include <R.h>
+#include <Rinternals.h>
+
+#include "nanoarrow.h"
+
+#include "array.h"
+#include "convert.h"
+#include "materialize.h"
+#include "schema.h"
+
+static R_xlen_t nanoarrow_vec_size(SEXP vec_sexp, struct PTypeView* ptype_view) {
+  if (ptype_view->vector_type == VECTOR_TYPE_DATA_FRAME) {
+    if (Rf_length(vec_sexp) > 0) {
+      // Avoid materializing the row.names if we can
+      return Rf_xlength(VECTOR_ELT(vec_sexp, 0));
+    } else {
+      return Rf_xlength(Rf_getAttrib(vec_sexp, R_RowNamesSymbol));

Review Comment:
   You might be able to get around what `Rf_getAttrib()` does with the "row.names" argument by using `.row_names_info()` or perhaps a function that goes through `ATTRIB()`. something like:
   
   ```r
   cpp11::cpp_function('
   SEXP row_names(SEXP x){ 
      for(SEXP atts = ATTRIB(x); atts != R_NilValue; atts = CDR(atts)){
         if (TAG(atts) == Rf_install("row.names")) return CAR(atts);
      }  
      return R_NilValue; 
   }')
   df <- data.frame(x = 1:10)
   row_names(df)
   ```
   
   ```
   > row_names(df)
   [1]  NA -10
   ```
   
   When this is of the form `c(NA, -n)` then `n` is the size. 



##########
r/src/convert_array.c:
##########
@@ -0,0 +1,219 @@
+// 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.
+
+#define R_NO_REMAP
+#include <R.h>
+#include <Rinternals.h>
+
+#include "nanoarrow.h"
+
+#include "altrep.h"
+#include "array.h"
+#include "array_view.h"
+#include "convert.h"
+
+// The common case of converting a single array into a single vector is
+// defined here, powered by the generic conversion available via
+// convert.h but special-casing the common case of "just use the defaults"
+// (i.e., no need to allocate a zero-size ptype) and returning ALTREP
+// where possible.
+
+// This calls nanoarrow::convert_array() (via a package helper) to try S3
+// dispatch to find a convert_array() method (or error if there
+// isn't one)
+static SEXP call_convert_array(SEXP array_xptr, SEXP ptype_sexp) {
+  SEXP ns = PROTECT(R_FindNamespace(Rf_mkString("nanoarrow")));
+  SEXP call =
+      PROTECT(Rf_lang3(Rf_install("convert_array_from_c"), array_xptr, ptype_sexp));
+  SEXP result = PROTECT(Rf_eval(call, ns));
+  UNPROTECT(3);
+  return result;
+}
+
+// Call stop_cant_convert_array(), which gives a more informative error
+// message than we can provide in a reasonable amount of C code here
+static void call_stop_cant_convert_array(SEXP array_xptr, enum VectorType type,
+                                         SEXP ptype_sexp) {
+  int n_protected = 2;

Review Comment:
   I don't think this should handle `UNPROTECT()`ing objects that were protected somewhere else. This should rather be the job of the function calling `call_stop_cant_convert_array()` and so here it should start at 0.  
   
   I assume that `stop_cant_convert_array` will call `stop()` eventually in the R side ? so the `UNPROTECT()` is not reached anyway, and the stack of protection is restored automatically by `.Call()`



##########
r/src/convert.c:
##########
@@ -0,0 +1,483 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#define R_NO_REMAP
+#include <R.h>
+#include <Rinternals.h>
+
+#include "nanoarrow.h"
+
+#include "array.h"
+#include "convert.h"
+#include "materialize.h"
+#include "schema.h"
+
+static R_xlen_t nanoarrow_vec_size(SEXP vec_sexp, struct PTypeView* ptype_view) {
+  if (ptype_view->vector_type == VECTOR_TYPE_DATA_FRAME) {
+    if (Rf_length(vec_sexp) > 0) {
+      // Avoid materializing the row.names if we can
+      return Rf_xlength(VECTOR_ELT(vec_sexp, 0));
+    } else {
+      return Rf_xlength(Rf_getAttrib(vec_sexp, R_RowNamesSymbol));
+    }
+  } else {
+    return Rf_xlength(vec_sexp);
+  }
+}
+
+static void finalize_converter(SEXP converter_xptr) {
+  struct RConverter* converter = (struct RConverter*)R_ExternalPtrAddr(converter_xptr);
+  if (converter != NULL) {
+    ArrowArrayViewReset(&converter->array_view);
+
+    if (converter->children != NULL) {
+      ArrowFree(converter->children);
+    }
+
+    ArrowFree(converter);
+  }
+}
+
+SEXP nanoarrow_converter_from_type(enum VectorType vector_type) {
+  struct RConverter* converter =
+      (struct RConverter*)ArrowMalloc(sizeof(struct RConverter));
+  if (converter == NULL) {
+    Rf_error("Failed to allocate RConverter");
+  }
+
+  // 0: ptype, 1: schema_xptr, 2: array_xptr, 3: children, 4: result
+  SEXP converter_shelter = PROTECT(Rf_allocVector(VECSXP, 5));
+  SEXP converter_xptr =
+      PROTECT(R_MakeExternalPtr(converter, R_NilValue, converter_shelter));
+  R_RegisterCFinalizer(converter_xptr, &finalize_converter);
+
+  ArrowArrayViewInit(&converter->array_view, NANOARROW_TYPE_UNINITIALIZED);
+  converter->schema_view.data_type = NANOARROW_TYPE_UNINITIALIZED;
+  converter->schema_view.storage_data_type = NANOARROW_TYPE_UNINITIALIZED;
+  converter->src.array_view = &converter->array_view;
+  converter->dst.vec_sexp = R_NilValue;
+  converter->options = NULL;
+  converter->error.message[0] = '\0';
+  converter->size = 0;
+  converter->capacity = 0;
+  converter->n_children = 0;
+  converter->children = NULL;
+
+  converter->ptype_view.vector_type = vector_type;
+  converter->ptype_view.ptype = R_NilValue;
+
+  switch (vector_type) {
+    case VECTOR_TYPE_NULL:
+      converter->ptype_view.sexp_type = NILSXP;
+      break;
+    case VECTOR_TYPE_LGL:
+      converter->ptype_view.sexp_type = LGLSXP;
+      break;
+    case VECTOR_TYPE_INT:
+      converter->ptype_view.sexp_type = INTSXP;
+      break;
+    case VECTOR_TYPE_DBL:
+      converter->ptype_view.sexp_type = REALSXP;
+      break;
+    case VECTOR_TYPE_CHR:
+      converter->ptype_view.sexp_type = STRSXP;
+      break;
+    default:
+      UNPROTECT(2);
+      return R_NilValue;
+  }
+
+  UNPROTECT(2);
+  return converter_xptr;
+}
+
+static enum RTimeUnits time_units_from_difftime(SEXP ptype) {
+  SEXP units_attr = Rf_getAttrib(ptype, Rf_install("units"));
+  if (units_attr == R_NilValue || TYPEOF(units_attr) != STRSXP ||
+      Rf_length(units_attr) != 1) {
+    Rf_error("Expected difftime 'units' attribute of type character(1)");
+  }
+
+  const char* dst_units = Rf_translateCharUTF8(STRING_ELT(units_attr, 0));
+  if (strcmp(dst_units, "secs") == 0) {
+    return R_TIME_UNIT_SECONDS;
+  } else if (strcmp(dst_units, "mins") == 0) {
+    return R_TIME_UNIT_MINUTES;
+  } else if (strcmp(dst_units, "hours") == 0) {
+    return R_TIME_UNIT_HOURS;
+  } else if (strcmp(dst_units, "days") == 0) {
+    return R_TIME_UNIT_DAYS;
+  } else if (strcmp(dst_units, "weeks") == 0) {
+    return R_TIME_UNIT_WEEKS;
+  } else {
+    Rf_error("Unexpected value for difftime 'units' attribute");
+    return R_TIME_UNIT_SECONDS;
+  }
+}
+
+static void set_converter_data_frame(SEXP converter_xptr, struct RConverter* converter,
+                                     SEXP ptype) {
+  converter->n_children = Rf_xlength(ptype);
+  converter->children = (struct RConverter**)ArrowMalloc(converter->n_children *
+                                                         sizeof(struct RConverter*));
+  if (converter->children == NULL) {
+    Rf_error("Failed to allocate converter children array");
+  }
+
+  SEXP child_converter_xptrs = PROTECT(Rf_allocVector(VECSXP, converter->n_children));
+
+  for (R_xlen_t i = 0; i < converter->n_children; i++) {
+    SEXP child_ptype = VECTOR_ELT(ptype, i);
+    SEXP child_converter = PROTECT(nanoarrow_converter_from_ptype(child_ptype));
+    converter->children[i] = (struct RConverter*)R_ExternalPtrAddr(child_converter);
+    SET_VECTOR_ELT(child_converter_xptrs, i, child_converter);
+    UNPROTECT(1);
+  }
+
+  SEXP converter_shelter = R_ExternalPtrProtected(converter_xptr);
+  SET_VECTOR_ELT(converter_shelter, 3, child_converter_xptrs);
+  UNPROTECT(1);
+}
+
+static void set_converter_list_of(SEXP converter_xptr, struct RConverter* converter,
+                                  SEXP ptype) {
+  SEXP child_ptype = Rf_getAttrib(ptype, Rf_install("ptype"));
+  if (child_ptype == R_NilValue) {
+    Rf_error("Expected attribute 'ptype' for conversion to list_of");
+  }
+
+  converter->children = (struct RConverter**)ArrowMalloc(1 * sizeof(struct RConverter*));
+  if (converter->children == NULL) {
+    Rf_error("Failed to allocate converter children array");
+  }
+  converter->n_children = 1;
+
+  SEXP child_converter_xptrs = PROTECT(Rf_allocVector(VECSXP, 1));
+  SEXP child_converter = PROTECT(nanoarrow_converter_from_ptype(child_ptype));
+  converter->children[0] = (struct RConverter*)R_ExternalPtrAddr(child_converter);
+  SET_VECTOR_ELT(child_converter_xptrs, 0, child_converter);
+
+  SEXP converter_shelter = R_ExternalPtrProtected(converter_xptr);
+  SET_VECTOR_ELT(converter_shelter, 3, child_converter_xptrs);
+  UNPROTECT(2);
+}
+
+static int set_converter_children_schema(SEXP converter_xptr, SEXP schema_xptr) {
+  struct RConverter* converter = (struct RConverter*)R_ExternalPtrAddr(converter_xptr);
+  SEXP converter_shelter = R_ExternalPtrProtected(converter_xptr);
+  struct ArrowSchema* schema = schema_from_xptr(schema_xptr);
+
+  if (schema->n_children != converter->n_children) {
+    ArrowErrorSet(&converter->error,
+                  "Expected schema with %ld children but got schema with %ld children",
+                  (long)converter->n_children, (long)schema->n_children);
+    return EINVAL;
+  }
+
+  SEXP child_converter_xptrs = VECTOR_ELT(converter_shelter, 3);
+
+  for (R_xlen_t i = 0; i < converter->n_children; i++) {
+    SEXP child_converter_xptr = VECTOR_ELT(child_converter_xptrs, i);
+    SEXP child_schema_xptr = PROTECT(borrow_schema_child_xptr(schema_xptr, i));
+    int result = nanoarrow_converter_set_schema(child_converter_xptr, child_schema_xptr);
+    UNPROTECT(1);
+    if (result != NANOARROW_OK) {
+      return result;
+    }
+  }
+
+  return NANOARROW_OK;
+}
+
+static int set_converter_children_array(SEXP converter_xptr, SEXP array_xptr) {
+  struct RConverter* converter = (struct RConverter*)R_ExternalPtrAddr(converter_xptr);
+  SEXP converter_shelter = R_ExternalPtrProtected(converter_xptr);
+  struct ArrowArray* array = array_from_xptr(array_xptr);
+
+  if (array->n_children != converter->n_children) {
+    ArrowErrorSet(&converter->error,
+                  "Expected array with %ld children but got array with %ld children",
+                  (long)converter->n_children, (long)array->n_children);
+    return EINVAL;
+  }
+
+  SEXP child_converter_xptrs = VECTOR_ELT(converter_shelter, 3);
+
+  for (R_xlen_t i = 0; i < converter->n_children; i++) {
+    SEXP child_converter_xptr = VECTOR_ELT(child_converter_xptrs, i);
+    SEXP child_array_xptr = PROTECT(borrow_array_child_xptr(array_xptr, i));
+    int result = nanoarrow_converter_set_array(child_converter_xptr, child_array_xptr);
+    UNPROTECT(1);
+    if (result != NANOARROW_OK) {
+      return result;
+    }
+  }
+
+  return NANOARROW_OK;
+}
+
+SEXP nanoarrow_converter_from_ptype(SEXP ptype) {
+  SEXP converter_xptr = PROTECT(nanoarrow_converter_from_type(VECTOR_TYPE_NULL));
+  SEXP converter_shelter = R_ExternalPtrProtected(converter_xptr);
+  struct RConverter* converter = (struct RConverter*)R_ExternalPtrAddr(converter_xptr);
+
+  if (Rf_isObject(ptype)) {
+    if (nanoarrow_ptype_is_data_frame(ptype)) {
+      converter->ptype_view.vector_type = VECTOR_TYPE_DATA_FRAME;
+      set_converter_data_frame(converter_xptr, converter, ptype);
+    } else if (Rf_inherits(ptype, "blob")) {
+      converter->ptype_view.vector_type = VECTOR_TYPE_BLOB;
+    } else if (Rf_inherits(ptype, "vctrs_list_of")) {
+      converter->ptype_view.vector_type = VECTOR_TYPE_LIST_OF;
+      set_converter_list_of(converter_xptr, converter, ptype);
+    } else if (Rf_inherits(ptype, "vctrs_unspecified")) {
+      converter->ptype_view.vector_type = VECTOR_TYPE_UNSPECIFIED;
+    } else if (Rf_inherits(ptype, "Date")) {
+      converter->ptype_view.vector_type = VECTOR_TYPE_DATE;
+      converter->ptype_view.r_time_units = R_TIME_UNIT_DAYS;
+    } else if (Rf_inherits(ptype, "POSIXct")) {
+      converter->ptype_view.vector_type = VECTOR_TYPE_POSIXCT;
+      converter->ptype_view.r_time_units = R_TIME_UNIT_SECONDS;
+    } else if (Rf_inherits(ptype, "difftime")) {
+      converter->ptype_view.vector_type = VECTOR_TYPE_DIFFTIME;
+      converter->ptype_view.r_time_units = time_units_from_difftime(ptype);
+    } else {
+      converter->ptype_view.vector_type = VECTOR_TYPE_OTHER;
+    }
+  } else {
+    switch (TYPEOF(ptype)) {
+      case LGLSXP:
+        converter->ptype_view.vector_type = VECTOR_TYPE_LGL;
+        break;
+      case INTSXP:
+        converter->ptype_view.vector_type = VECTOR_TYPE_INT;
+        break;
+      case REALSXP:
+        converter->ptype_view.vector_type = VECTOR_TYPE_DBL;
+        break;
+      case STRSXP:
+        converter->ptype_view.vector_type = VECTOR_TYPE_CHR;
+        break;
+      default:
+        converter->ptype_view.vector_type = VECTOR_TYPE_OTHER;
+        break;
+    }
+  }
+
+  converter->ptype_view.ptype = ptype;
+  converter->ptype_view.sexp_type = TYPEOF(ptype);
+  SET_VECTOR_ELT(converter_shelter, 0, ptype);
+
+  UNPROTECT(1);
+  return converter_xptr;
+}
+
+int nanoarrow_converter_set_schema(SEXP converter_xptr, SEXP schema_xptr) {
+  struct RConverter* converter = (struct RConverter*)R_ExternalPtrAddr(converter_xptr);
+  SEXP converter_shelter = R_ExternalPtrProtected(converter_xptr);
+  struct ArrowSchema* schema = schema_from_xptr(schema_xptr);
+  NANOARROW_RETURN_NOT_OK(
+      ArrowSchemaViewInit(&converter->schema_view, schema, &converter->error));
+
+  // TODO: Currently we error at the materialize stage if a conversion is not possible;
+  // however, at this stage we have all the information we need to calculate that.
+
+  // For extension types, warn that we are about to strip the extension type, as we don't
+  // have a mechanism for dealing with them yet
+  if (converter->schema_view.extension_name.n_bytes > 0) {
+    int64_t schema_chars = ArrowSchemaToString(schema, NULL, 0, 1);
+    SEXP fmt_shelter = PROTECT(Rf_allocVector(RAWSXP, schema_chars + 1));
+    ArrowSchemaToString(schema, (char*)RAW(fmt_shelter), schema_chars + 1, 1);
+    const char* schema_name = schema->name;
+    if (schema_name == NULL || schema_name[0] == '\0') {
+      Rf_warning("Converting unknown extension %s as storage type",
+                 (const char*)RAW(fmt_shelter));
+    } else {
+      Rf_warning("%s: Converting unknown extension %s as storage type", schema_name,
+                 (const char*)RAW(fmt_shelter));
+    }
+
+    UNPROTECT(1);
+  }
+
+  // Sub-par error for dictionary types until we have a way to deal with them
+  if (converter->schema_view.data_type == NANOARROW_TYPE_DICTIONARY) {
+    ArrowErrorSet(&converter->error,
+                  "Conversion to dictionary-encoded array is not supported");
+    return ENOTSUP;
+  }
+
+  SET_VECTOR_ELT(converter_shelter, 1, schema_xptr);
+
+  ArrowArrayViewReset(&converter->array_view);
+  SET_VECTOR_ELT(converter_shelter, 2, R_NilValue);
+  NANOARROW_RETURN_NOT_OK(
+      ArrowArrayViewInitFromSchema(&converter->array_view, schema, &converter->error));
+
+  if (converter->ptype_view.vector_type == VECTOR_TYPE_LIST_OF ||
+      converter->ptype_view.vector_type == VECTOR_TYPE_DATA_FRAME) {
+    set_converter_children_schema(converter_xptr, schema_xptr);
+  }
+
+  return NANOARROW_OK;
+}
+
+int nanoarrow_converter_set_array(SEXP converter_xptr, SEXP array_xptr) {
+  struct RConverter* converter = (struct RConverter*)R_ExternalPtrAddr(converter_xptr);
+  SEXP converter_shelter = R_ExternalPtrProtected(converter_xptr);
+  struct ArrowArray* array = array_from_xptr(array_xptr);
+  NANOARROW_RETURN_NOT_OK(
+      ArrowArrayViewSetArray(&converter->array_view, array, &converter->error));
+  SET_VECTOR_ELT(converter_shelter, 2, array_xptr);
+  converter->src.offset = 0;
+  converter->src.length = 0;
+
+  if (converter->ptype_view.vector_type == VECTOR_TYPE_LIST_OF ||
+      converter->ptype_view.vector_type == VECTOR_TYPE_DATA_FRAME) {
+    set_converter_children_array(converter_xptr, array_xptr);
+  }
+
+  return NANOARROW_OK;
+}
+
+void sync_after_converter_reallocate(SEXP converter_xptr, struct RConverter* converter,
+                                     SEXP result_sexp, R_xlen_t capacity) {
+  SEXP converter_shelter = R_ExternalPtrProtected(converter_xptr);
+  SET_VECTOR_ELT(converter_shelter, 4, result_sexp);
+
+  converter->dst.vec_sexp = result_sexp;
+  converter->dst.offset = 0;
+  converter->dst.length = 0;
+  converter->size = 0;
+  converter->capacity = capacity;
+
+  if (converter->ptype_view.vector_type == VECTOR_TYPE_DATA_FRAME) {
+    SEXP child_converters = VECTOR_ELT(converter_shelter, 3);
+    for (R_xlen_t i = 0; i < converter->n_children; i++) {
+      sync_after_converter_reallocate(VECTOR_ELT(child_converters, i),
+                                      converter->children[i], VECTOR_ELT(result_sexp, i),
+                                      capacity);
+    }
+  }
+}
+
+int nanoarrow_converter_reserve(SEXP converter_xptr, R_xlen_t additional_size) {
+  struct RConverter* converter = (struct RConverter*)R_ExternalPtrAddr(converter_xptr);
+  SEXP converter_shelter = R_ExternalPtrProtected(converter_xptr);
+  SEXP current_result = VECTOR_ELT(converter_shelter, 4);
+
+  if (current_result != R_NilValue) {
+    ArrowErrorSet(&converter->error, "Reallocation in converter is not implemented");
+    return ENOTSUP;
+  }
+
+  SEXP result_sexp;
+  if (converter->ptype_view.ptype != R_NilValue) {
+    result_sexp = PROTECT(
+        nanoarrow_materialize_realloc(converter->ptype_view.ptype, additional_size));
+  } else {
+    result_sexp =
+        PROTECT(nanoarrow_alloc_type(converter->ptype_view.vector_type, additional_size));
+  }
+
+  sync_after_converter_reallocate(converter_xptr, converter, result_sexp,
+                                  additional_size);
+  UNPROTECT(1);
+  return NANOARROW_OK;
+}
+
+R_xlen_t nanoarrow_converter_materialize_n(SEXP converter_xptr, R_xlen_t n) {
+  struct RConverter* converter = (struct RConverter*)R_ExternalPtrAddr(converter_xptr);
+  if ((converter->dst.offset + n) > converter->capacity) {
+    n = converter->capacity - converter->dst.offset;
+  }
+
+  if ((converter->src.offset + n) > converter->array_view.array->length) {
+    n = converter->array_view.array->length - converter->src.offset;
+  }
+
+  if (n == 0) {
+    return 0;
+  }
+
+  converter->src.length = converter->dst.length = n;
+  int result = nanoarrow_materialize(converter, converter_xptr);
+  if (result != NANOARROW_OK) {
+    ArrowErrorSet(&converter->error, "Error in nanoarrow_materialize()");
+    return 0;
+  }
+
+  converter->src.offset += n;
+  converter->dst.offset += n;
+  converter->size += n;
+  return n;
+}
+
+int nanoarrow_converter_materialize_all(SEXP converter_xptr) {
+  struct RConverter* converter = (struct RConverter*)R_ExternalPtrAddr(converter_xptr);
+  R_xlen_t remaining = converter->array_view.array->length;
+  NANOARROW_RETURN_NOT_OK(nanoarrow_converter_reserve(converter_xptr, remaining));
+  if (nanoarrow_converter_materialize_n(converter_xptr, remaining) != remaining) {
+    return ERANGE;
+  } else {
+    return NANOARROW_OK;
+  }
+}
+
+int nanoarrow_converter_finalize(SEXP converter_xptr) {
+  struct RConverter* converter = (struct RConverter*)R_ExternalPtrAddr(converter_xptr);
+  SEXP converter_shelter = R_ExternalPtrProtected(converter_xptr);
+  SEXP current_result = VECTOR_ELT(converter_shelter, 4);
+
+  // Materialize never called (e.g., empty stream)
+  if (current_result == R_NilValue) {
+    NANOARROW_RETURN_NOT_OK(nanoarrow_converter_reserve(converter_xptr, 0));
+    current_result = VECTOR_ELT(converter_shelter, 4);
+  }
+
+  // Check result size. A future implementation could also shrink the length
+  // or reallocate a shorter vector.
+  R_xlen_t current_result_size =
+      nanoarrow_vec_size(current_result, &converter->ptype_view);
+  if (current_result_size != converter->size) {
+    ArrowErrorSet(&converter->error,
+                  "Expected result of size %ld but got result of size %ld",
+                  (long)current_result_size, (long)converter->size);
+    return ENOTSUP;
+  }
+
+  return NANOARROW_OK;
+}
+
+SEXP nanoarrow_converter_result(SEXP converter_xptr) {
+  struct RConverter* converter = (struct RConverter*)R_ExternalPtrAddr(converter_xptr);
+  SEXP converter_shelter = R_ExternalPtrProtected(converter_xptr);
+  SEXP result = PROTECT(VECTOR_ELT(converter_shelter, 4));

Review Comment:
   I think the `PROTECT()` is redundant here, as `converter_shelter` would already be protected by `converter_xptr`



##########
r/src/convert_array.c:
##########
@@ -0,0 +1,219 @@
+// 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.
+
+#define R_NO_REMAP
+#include <R.h>
+#include <Rinternals.h>
+
+#include "nanoarrow.h"
+
+#include "altrep.h"
+#include "array.h"
+#include "array_view.h"
+#include "convert.h"
+
+// The common case of converting a single array into a single vector is
+// defined here, powered by the generic conversion available via
+// convert.h but special-casing the common case of "just use the defaults"
+// (i.e., no need to allocate a zero-size ptype) and returning ALTREP
+// where possible.
+
+// This calls nanoarrow::convert_array() (via a package helper) to try S3
+// dispatch to find a convert_array() method (or error if there
+// isn't one)
+static SEXP call_convert_array(SEXP array_xptr, SEXP ptype_sexp) {
+  SEXP ns = PROTECT(R_FindNamespace(Rf_mkString("nanoarrow")));
+  SEXP call =
+      PROTECT(Rf_lang3(Rf_install("convert_array_from_c"), array_xptr, ptype_sexp));
+  SEXP result = PROTECT(Rf_eval(call, ns));
+  UNPROTECT(3);
+  return result;
+}
+
+// Call stop_cant_convert_array(), which gives a more informative error
+// message than we can provide in a reasonable amount of C code here
+static void call_stop_cant_convert_array(SEXP array_xptr, enum VectorType type,
+                                         SEXP ptype_sexp) {
+  int n_protected = 2;
+  if (ptype_sexp == R_NilValue) {
+    ptype_sexp = PROTECT(nanoarrow_alloc_type(type, 0));
+    n_protected++;
+  }
+
+  SEXP ns = PROTECT(R_FindNamespace(Rf_mkString("nanoarrow")));
+  SEXP call =
+      PROTECT(Rf_lang3(Rf_install("stop_cant_convert_array"), array_xptr, ptype_sexp));
+  Rf_eval(call, ns);
+
+  UNPROTECT(n_protected);
+}
+
+static SEXP convert_array_default(SEXP array_xptr, enum VectorType vector_type,
+                                  SEXP ptype) {
+  SEXP converter_xptr;
+  if (ptype == R_NilValue) {
+    converter_xptr = PROTECT(nanoarrow_converter_from_type(vector_type));
+  } else {
+    converter_xptr = PROTECT(nanoarrow_converter_from_ptype(ptype));
+  }
+
+  if (nanoarrow_converter_set_schema(converter_xptr, array_xptr_get_schema(array_xptr)) !=
+      NANOARROW_OK) {
+    nanoarrow_converter_stop(converter_xptr);
+  }
+
+  if (nanoarrow_converter_set_array(converter_xptr, array_xptr) != NANOARROW_OK) {
+    nanoarrow_converter_stop(converter_xptr);
+  }
+
+  if (nanoarrow_converter_materialize_all(converter_xptr) != NANOARROW_OK) {
+    call_stop_cant_convert_array(array_xptr, vector_type, ptype);
+  }
+
+  if (nanoarrow_converter_finalize(converter_xptr) != NANOARROW_OK) {
+    nanoarrow_converter_stop(converter_xptr);
+  }
+
+  SEXP result = PROTECT(nanoarrow_converter_result(converter_xptr));
+  UNPROTECT(2);
+  return result;
+}
+
+static SEXP convert_array_chr(SEXP array_xptr) {
+  SEXP result = PROTECT(nanoarrow_c_make_altrep_chr(array_xptr));
+  if (result == R_NilValue) {
+    call_stop_cant_convert_array(array_xptr, VECTOR_TYPE_CHR, R_NilValue);
+  }
+  UNPROTECT(1);
+  return result;
+}
+
+SEXP nanoarrow_c_convert_array(SEXP array_xptr, SEXP ptype_sexp);
+
+static SEXP convert_array_data_frame(SEXP array_xptr, SEXP ptype_sexp) {
+  struct ArrowArray* array = array_from_xptr(array_xptr);
+  R_xlen_t n_col = array->n_children;
+  SEXP result = PROTECT(Rf_allocVector(VECSXP, n_col));
+
+  if (ptype_sexp == R_NilValue) {
+    SEXP result_names = PROTECT(Rf_allocVector(STRSXP, n_col));
+
+    for (R_xlen_t i = 0; i < n_col; i++) {
+      SEXP child_xptr = PROTECT(borrow_array_child_xptr(array_xptr, i));
+      SET_VECTOR_ELT(result, i, nanoarrow_c_convert_array(child_xptr, R_NilValue));
+      UNPROTECT(1);
+
+      struct ArrowSchema* schema = schema_from_array_xptr(child_xptr);
+      if (schema->name != NULL) {
+        SET_STRING_ELT(result_names, i, Rf_mkCharCE(schema->name, CE_UTF8));
+      } else {
+        SET_STRING_ELT(result_names, i, Rf_mkChar(""));
+      }
+    }
+
+    Rf_setAttrib(result, R_NamesSymbol, result_names);
+    Rf_setAttrib(result, R_ClassSymbol, Rf_mkString("data.frame"));
+    UNPROTECT(1);
+  } else {
+    if (n_col != Rf_xlength(ptype_sexp)) {
+      Rf_error("Expected data.frame() ptype with %ld column(s) but found %ld column(s)",
+               (long)n_col, (long)Rf_xlength(ptype_sexp));
+    }
+
+    for (R_xlen_t i = 0; i < n_col; i++) {
+      SEXP child_xptr = PROTECT(borrow_array_child_xptr(array_xptr, i));
+      SEXP child_ptype = VECTOR_ELT(ptype_sexp, i);
+      SET_VECTOR_ELT(result, i, nanoarrow_c_convert_array(child_xptr, child_ptype));
+      UNPROTECT(1);
+    }
+
+    Rf_setAttrib(result, R_NamesSymbol, Rf_getAttrib(ptype_sexp, R_NamesSymbol));
+    Rf_copyMostAttrib(ptype_sexp, result);
+  }
+
+  if (Rf_inherits(result, "data.frame")) {
+    SEXP rownames = PROTECT(Rf_allocVector(INTSXP, 2));
+    INTEGER(rownames)[0] = NA_INTEGER;
+    INTEGER(rownames)[1] = array->length;

Review Comment:
   I think both are supported, but `-array->length` is preferred



##########
r/src/materialize.c:
##########
@@ -21,258 +21,215 @@
 
 #include "nanoarrow.h"
 
-// Note: These conversions are not currently written for safety rather than
-// speed. We could make use of C++ templating to provide faster and/or more
-// readable conversions here with a C entry point.
-
-SEXP nanoarrow_materialize_lgl(struct ArrowArrayView* array_view) {
-  SEXP result_sexp = PROTECT(Rf_allocVector(LGLSXP, array_view->array->length));
-  int* result = LOGICAL(result_sexp);
-
-  // True for all the types supported here
-  const uint8_t* is_valid = array_view->buffer_views[0].data.as_uint8;
-  const uint8_t* data_buffer = array_view->buffer_views[1].data.as_uint8;
-
-  // Fill the buffer
-  switch (array_view->storage_type) {
-    case NANOARROW_TYPE_BOOL:
-      for (R_xlen_t i = 0; i < array_view->array->length; i++) {
-        result[i] = ArrowBitGet(data_buffer, i);
-      }
-
-      // Set any nulls to NA_LOGICAL
-      if (is_valid != NULL && array_view->array->null_count != 0) {
-        for (R_xlen_t i = 0; i < array_view->array->length; i++) {
-          if (!ArrowBitGet(is_valid, i)) {
-            result[i] = NA_LOGICAL;
-          }
-        }
-      }
+// Needed for the list_of materializer
+#include "convert.h"
+
+#include "materialize.h"
+#include "materialize_blob.h"
+#include "materialize_chr.h"
+#include "materialize_date.h"
+#include "materialize_dbl.h"
+#include "materialize_difftime.h"
+#include "materialize_int.h"
+#include "materialize_lgl.h"
+#include "materialize_posixct.h"
+#include "materialize_unspecified.h"
+
+SEXP nanoarrow_alloc_type(enum VectorType vector_type, R_xlen_t len) {
+  SEXP result;
+
+  switch (vector_type) {
+    case VECTOR_TYPE_LGL:
+      result = PROTECT(Rf_allocVector(LGLSXP, len));
       break;
-    case NANOARROW_TYPE_INT8:
-    case NANOARROW_TYPE_UINT8:
-    case NANOARROW_TYPE_INT16:
-    case NANOARROW_TYPE_UINT16:
-    case NANOARROW_TYPE_INT32:
-    case NANOARROW_TYPE_UINT32:
-    case NANOARROW_TYPE_INT64:
-    case NANOARROW_TYPE_UINT64:
-    case NANOARROW_TYPE_FLOAT:
-    case NANOARROW_TYPE_DOUBLE:
-      for (R_xlen_t i = 0; i < array_view->array->length; i++) {
-        result[i] = ArrowArrayViewGetIntUnsafe(array_view, i) != 0;
-      }
-
-      // Set any nulls to NA_LOGICAL
-      if (is_valid != NULL && array_view->array->null_count != 0) {
-        for (R_xlen_t i = 0; i < array_view->array->length; i++) {
-          if (!ArrowBitGet(is_valid, i)) {
-            result[i] = NA_LOGICAL;
-          }
-        }
-      }
+    case VECTOR_TYPE_INT:
+      result = PROTECT(Rf_allocVector(INTSXP, len));
+      break;
+    case VECTOR_TYPE_DBL:
+      result = PROTECT(Rf_allocVector(REALSXP, len));
+      break;
+    case VECTOR_TYPE_CHR:
+      result = PROTECT(Rf_allocVector(STRSXP, len));
       break;
-
     default:
-      UNPROTECT(1);
       return R_NilValue;
   }
 
   UNPROTECT(1);
-  return result_sexp;
+  return result;
 }
 
-SEXP nanoarrow_materialize_int(struct ArrowArrayView* array_view) {
-  SEXP result_sexp = PROTECT(Rf_allocVector(INTSXP, array_view->array->length));
-  int* result = INTEGER(result_sexp);
-  int64_t n_bad_values = 0;
-
-  // True for all the types supported here
-  const uint8_t* is_valid = array_view->buffer_views[0].data.as_uint8;
+int nanoarrow_ptype_is_data_frame(SEXP ptype) {
+  return Rf_isObject(ptype) && TYPEOF(ptype) == VECSXP &&
+         (Rf_inherits(ptype, "data.frame") ||
+          (Rf_xlength(ptype) > 0 && Rf_getAttrib(ptype, R_NamesSymbol) != R_NilValue));

Review Comment:
   This `getAttrib()` might materialize the row names vector, which can be avoided. 



##########
r/src/materialize_dbl.h:
##########
@@ -0,0 +1,124 @@
+// 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.
+
+#ifndef R_MATERIALIZE_DBL_H_INCLUDED
+#define R_MATERIALIZE_DBL_H_INCLUDED
+
+#include <R.h>
+#include <Rinternals.h>
+
+#include "materialize_common.h"
+#include "nanoarrow.h"
+
+// Fall back to arrow for decimal conversion via a package helper
+static inline void nanoarrow_materialize_decimal_to_dbl(struct RConverter* converter) {
+  // A unique situation where we don't want owning external pointers because we know
+  // these are protected for the duration of our call into R and because we don't want
+  // then to be garbage collected and invalidate the converter
+  SEXP array_xptr =
+      PROTECT(R_MakeExternalPtr(converter->array_view.array, R_NilValue, R_NilValue));
+  Rf_setAttrib(array_xptr, R_ClassSymbol, Rf_mkString("nanoarrow_array"));

Review Comment:
   I think `Rf_mkString("nanoarrow_array")` needs protection before it goes into `Rf_setAttrib()` 



##########
r/src/materialize.c:
##########
@@ -21,258 +21,215 @@
 
 #include "nanoarrow.h"
 
-// Note: These conversions are not currently written for safety rather than
-// speed. We could make use of C++ templating to provide faster and/or more
-// readable conversions here with a C entry point.
-
-SEXP nanoarrow_materialize_lgl(struct ArrowArrayView* array_view) {
-  SEXP result_sexp = PROTECT(Rf_allocVector(LGLSXP, array_view->array->length));
-  int* result = LOGICAL(result_sexp);
-
-  // True for all the types supported here
-  const uint8_t* is_valid = array_view->buffer_views[0].data.as_uint8;
-  const uint8_t* data_buffer = array_view->buffer_views[1].data.as_uint8;
-
-  // Fill the buffer
-  switch (array_view->storage_type) {
-    case NANOARROW_TYPE_BOOL:
-      for (R_xlen_t i = 0; i < array_view->array->length; i++) {
-        result[i] = ArrowBitGet(data_buffer, i);
-      }
-
-      // Set any nulls to NA_LOGICAL
-      if (is_valid != NULL && array_view->array->null_count != 0) {
-        for (R_xlen_t i = 0; i < array_view->array->length; i++) {
-          if (!ArrowBitGet(is_valid, i)) {
-            result[i] = NA_LOGICAL;
-          }
-        }
-      }
+// Needed for the list_of materializer
+#include "convert.h"
+
+#include "materialize.h"
+#include "materialize_blob.h"
+#include "materialize_chr.h"
+#include "materialize_date.h"
+#include "materialize_dbl.h"
+#include "materialize_difftime.h"
+#include "materialize_int.h"
+#include "materialize_lgl.h"
+#include "materialize_posixct.h"
+#include "materialize_unspecified.h"
+
+SEXP nanoarrow_alloc_type(enum VectorType vector_type, R_xlen_t len) {
+  SEXP result;
+
+  switch (vector_type) {
+    case VECTOR_TYPE_LGL:
+      result = PROTECT(Rf_allocVector(LGLSXP, len));
       break;
-    case NANOARROW_TYPE_INT8:
-    case NANOARROW_TYPE_UINT8:
-    case NANOARROW_TYPE_INT16:
-    case NANOARROW_TYPE_UINT16:
-    case NANOARROW_TYPE_INT32:
-    case NANOARROW_TYPE_UINT32:
-    case NANOARROW_TYPE_INT64:
-    case NANOARROW_TYPE_UINT64:
-    case NANOARROW_TYPE_FLOAT:
-    case NANOARROW_TYPE_DOUBLE:
-      for (R_xlen_t i = 0; i < array_view->array->length; i++) {
-        result[i] = ArrowArrayViewGetIntUnsafe(array_view, i) != 0;
-      }
-
-      // Set any nulls to NA_LOGICAL
-      if (is_valid != NULL && array_view->array->null_count != 0) {
-        for (R_xlen_t i = 0; i < array_view->array->length; i++) {
-          if (!ArrowBitGet(is_valid, i)) {
-            result[i] = NA_LOGICAL;
-          }
-        }
-      }
+    case VECTOR_TYPE_INT:
+      result = PROTECT(Rf_allocVector(INTSXP, len));
+      break;
+    case VECTOR_TYPE_DBL:
+      result = PROTECT(Rf_allocVector(REALSXP, len));
+      break;
+    case VECTOR_TYPE_CHR:
+      result = PROTECT(Rf_allocVector(STRSXP, len));
       break;
-
     default:
-      UNPROTECT(1);
       return R_NilValue;
   }
 
   UNPROTECT(1);
-  return result_sexp;
+  return result;
 }
 
-SEXP nanoarrow_materialize_int(struct ArrowArrayView* array_view) {
-  SEXP result_sexp = PROTECT(Rf_allocVector(INTSXP, array_view->array->length));
-  int* result = INTEGER(result_sexp);
-  int64_t n_bad_values = 0;
-
-  // True for all the types supported here
-  const uint8_t* is_valid = array_view->buffer_views[0].data.as_uint8;
+int nanoarrow_ptype_is_data_frame(SEXP ptype) {
+  return Rf_isObject(ptype) && TYPEOF(ptype) == VECSXP &&
+         (Rf_inherits(ptype, "data.frame") ||
+          (Rf_xlength(ptype) > 0 && Rf_getAttrib(ptype, R_NamesSymbol) != R_NilValue));
+}
 
-  // Fill the buffer
-  switch (array_view->storage_type) {
-    case NANOARROW_TYPE_INT32:
-      memcpy(result,
-             array_view->buffer_views[1].data.as_int32 + array_view->array->offset,
-             array_view->array->length * sizeof(int32_t));
+SEXP nanoarrow_materialize_realloc(SEXP ptype, R_xlen_t len) {
+  SEXP result;
 
-      // Set any nulls to NA_INTEGER
-      if (is_valid != NULL && array_view->array->null_count != 0) {
-        for (R_xlen_t i = 0; i < array_view->array->length; i++) {
-          if (!ArrowBitGet(is_valid, i)) {
-            result[i] = NA_INTEGER;
-          }
-        }
-      }
-      break;
-    case NANOARROW_TYPE_BOOL:
-    case NANOARROW_TYPE_INT8:
-    case NANOARROW_TYPE_UINT8:
-    case NANOARROW_TYPE_INT16:
-    case NANOARROW_TYPE_UINT16:
-      // No need to bounds check for these types
-      for (R_xlen_t i = 0; i < array_view->array->length; i++) {
-        result[i] = ArrowArrayViewGetIntUnsafe(array_view, i);
+  if (Rf_isObject(ptype)) {
+    if (nanoarrow_ptype_is_data_frame(ptype)) {
+      R_xlen_t num_cols = Rf_xlength(ptype);
+      result = PROTECT(Rf_allocVector(VECSXP, num_cols));
+      for (R_xlen_t i = 0; i < num_cols; i++) {
+        SET_VECTOR_ELT(result, i,
+                       nanoarrow_materialize_realloc(VECTOR_ELT(ptype, i), len));
       }
 
-      // Set any nulls to NA_INTEGER
-      if (is_valid != NULL && array_view->array->null_count != 0) {
-        for (R_xlen_t i = 0; i < array_view->array->length; i++) {
-          if (!ArrowBitGet(is_valid, i)) {
-            result[i] = NA_INTEGER;
-          }
-        }
+      // Set attributes from ptype
+      Rf_setAttrib(result, R_NamesSymbol, Rf_getAttrib(ptype, R_NamesSymbol));
+      Rf_copyMostAttrib(ptype, result);
+
+      // ...except rownames
+      if (Rf_inherits(ptype, "data.frame")) {
+        SEXP rownames = PROTECT(Rf_allocVector(INTSXP, 2));
+        INTEGER(rownames)[0] = NA_INTEGER;
+        INTEGER(rownames)[1] = len;

Review Comment:
   ```suggestion
           INTEGER(rownames)[1] = -len;
   ```



##########
r/src/convert_array.c:
##########
@@ -0,0 +1,219 @@
+// 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.
+
+#define R_NO_REMAP
+#include <R.h>
+#include <Rinternals.h>
+
+#include "nanoarrow.h"
+
+#include "altrep.h"
+#include "array.h"
+#include "array_view.h"
+#include "convert.h"
+
+// The common case of converting a single array into a single vector is
+// defined here, powered by the generic conversion available via
+// convert.h but special-casing the common case of "just use the defaults"
+// (i.e., no need to allocate a zero-size ptype) and returning ALTREP
+// where possible.
+
+// This calls nanoarrow::convert_array() (via a package helper) to try S3
+// dispatch to find a convert_array() method (or error if there
+// isn't one)
+static SEXP call_convert_array(SEXP array_xptr, SEXP ptype_sexp) {
+  SEXP ns = PROTECT(R_FindNamespace(Rf_mkString("nanoarrow")));

Review Comment:
   Or maybe the result of  `R_FindNamespace()` could be cached. e.g. this caches a few namespace environment in `dplyr`: https://github.com/tidyverse/dplyr/blob/935cd11694786ac82748d82f9e19600566b8fbcb/src/init.cpp#L76



##########
r/src/convert_array.c:
##########
@@ -0,0 +1,219 @@
+// 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.
+
+#define R_NO_REMAP
+#include <R.h>
+#include <Rinternals.h>
+
+#include "nanoarrow.h"
+
+#include "altrep.h"
+#include "array.h"
+#include "array_view.h"
+#include "convert.h"
+
+// The common case of converting a single array into a single vector is
+// defined here, powered by the generic conversion available via
+// convert.h but special-casing the common case of "just use the defaults"
+// (i.e., no need to allocate a zero-size ptype) and returning ALTREP
+// where possible.
+
+// This calls nanoarrow::convert_array() (via a package helper) to try S3
+// dispatch to find a convert_array() method (or error if there
+// isn't one)
+static SEXP call_convert_array(SEXP array_xptr, SEXP ptype_sexp) {
+  SEXP ns = PROTECT(R_FindNamespace(Rf_mkString("nanoarrow")));

Review Comment:
   `Rf_mkString("nanoarrow")` should be protected, because `R_FindNamespace` might allocate, and then trigger a gc ... 
   
   Or perhaps, since it's used in several places, it could be cached. 



##########
r/src/materialize_dbl.h:
##########
@@ -0,0 +1,124 @@
+// 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.
+
+#ifndef R_MATERIALIZE_DBL_H_INCLUDED
+#define R_MATERIALIZE_DBL_H_INCLUDED
+
+#include <R.h>
+#include <Rinternals.h>
+
+#include "materialize_common.h"
+#include "nanoarrow.h"
+
+// Fall back to arrow for decimal conversion via a package helper
+static inline void nanoarrow_materialize_decimal_to_dbl(struct RConverter* converter) {
+  // A unique situation where we don't want owning external pointers because we know
+  // these are protected for the duration of our call into R and because we don't want
+  // then to be garbage collected and invalidate the converter
+  SEXP array_xptr =
+      PROTECT(R_MakeExternalPtr(converter->array_view.array, R_NilValue, R_NilValue));
+  Rf_setAttrib(array_xptr, R_ClassSymbol, Rf_mkString("nanoarrow_array"));
+  SEXP schema_xptr =
+      PROTECT(R_MakeExternalPtr(converter->schema_view.schema, R_NilValue, R_NilValue));
+  Rf_setAttrib(schema_xptr, R_ClassSymbol, Rf_mkString("nanoarrow_schema"));

Review Comment:
   same for `Rf_mkString("nanoarrow_schema")`



##########
r/src/materialize_difftime.h:
##########
@@ -0,0 +1,92 @@
+// 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.
+
+#ifndef R_MATERIALIZE_DIFFTIME_H_INCLUDED
+#define R_MATERIALIZE_DIFFTIME_H_INCLUDED
+
+#include <R.h>
+#include <Rinternals.h>
+
+#include "materialize_common.h"
+#include "materialize_dbl.h"
+#include "nanoarrow.h"
+
+static inline int nanoarrow_materialize_difftime(struct RConverter* converter) {

Review Comment:
   It's a bit odd these live in a header file. 



##########
r/src/materialize.c:
##########
@@ -21,258 +21,215 @@
 
 #include "nanoarrow.h"
 
-// Note: These conversions are not currently written for safety rather than
-// speed. We could make use of C++ templating to provide faster and/or more
-// readable conversions here with a C entry point.
-
-SEXP nanoarrow_materialize_lgl(struct ArrowArrayView* array_view) {
-  SEXP result_sexp = PROTECT(Rf_allocVector(LGLSXP, array_view->array->length));
-  int* result = LOGICAL(result_sexp);
-
-  // True for all the types supported here
-  const uint8_t* is_valid = array_view->buffer_views[0].data.as_uint8;
-  const uint8_t* data_buffer = array_view->buffer_views[1].data.as_uint8;
-
-  // Fill the buffer
-  switch (array_view->storage_type) {
-    case NANOARROW_TYPE_BOOL:
-      for (R_xlen_t i = 0; i < array_view->array->length; i++) {
-        result[i] = ArrowBitGet(data_buffer, i);
-      }
-
-      // Set any nulls to NA_LOGICAL
-      if (is_valid != NULL && array_view->array->null_count != 0) {
-        for (R_xlen_t i = 0; i < array_view->array->length; i++) {
-          if (!ArrowBitGet(is_valid, i)) {
-            result[i] = NA_LOGICAL;
-          }
-        }
-      }
+// Needed for the list_of materializer
+#include "convert.h"
+
+#include "materialize.h"
+#include "materialize_blob.h"
+#include "materialize_chr.h"
+#include "materialize_date.h"
+#include "materialize_dbl.h"
+#include "materialize_difftime.h"
+#include "materialize_int.h"
+#include "materialize_lgl.h"
+#include "materialize_posixct.h"
+#include "materialize_unspecified.h"
+
+SEXP nanoarrow_alloc_type(enum VectorType vector_type, R_xlen_t len) {
+  SEXP result;
+
+  switch (vector_type) {
+    case VECTOR_TYPE_LGL:
+      result = PROTECT(Rf_allocVector(LGLSXP, len));

Review Comment:
   maybe this can be simplified to `return Rf_allocVector(LGLSXP, len);` i.e. this would not need the `PROTECT()` / `UNPROTECT()`



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

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

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


[GitHub] [arrow-nanoarrow] paleolimbot merged pull request #65: [R] Complete ptype inferences and array conversions

Posted by GitBox <gi...@apache.org>.
paleolimbot merged PR #65:
URL: https://github.com/apache/arrow-nanoarrow/pull/65


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

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

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


[GitHub] [arrow-nanoarrow] paleolimbot commented on a diff in pull request #65: [R] Complete ptype inferences and array conversions

Posted by GitBox <gi...@apache.org>.
paleolimbot commented on code in PR #65:
URL: https://github.com/apache/arrow-nanoarrow/pull/65#discussion_r1026511358


##########
r/src/convert.c:
##########
@@ -0,0 +1,483 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#define R_NO_REMAP
+#include <R.h>
+#include <Rinternals.h>
+
+#include "nanoarrow.h"
+
+#include "array.h"
+#include "convert.h"
+#include "materialize.h"
+#include "schema.h"
+
+static R_xlen_t nanoarrow_vec_size(SEXP vec_sexp, struct PTypeView* ptype_view) {
+  if (ptype_view->vector_type == VECTOR_TYPE_DATA_FRAME) {
+    if (Rf_length(vec_sexp) > 0) {
+      // Avoid materializing the row.names if we can
+      return Rf_xlength(VECTOR_ELT(vec_sexp, 0));
+    } else {
+      return Rf_xlength(Rf_getAttrib(vec_sexp, R_RowNamesSymbol));

Review Comment:
   Thanks! I played with this a little bit and if I go this route I have to implement inspecting for `c(NA, nrow)`, `c(NA, -nrow)` myself and I'm a little worried I will mess this up. It looks like rownames are sufficiently ALTREP even when expanded in recent R (although I don't know how far back this goes) such that computing the length shouldn't be an expensive operation?
   
   ``` r
   df <- nanoarrow:::new_data_frame(x = 1:1e9, 1e9)
   .Internal(inspect(df))
   #> @141c96758 13 INTSXP g0c0 [OBJ,REF(2),ATT]  wrapper [srt=-2147483648,no_na=0]
   #>   @141c93060 13 INTSXP g0c0 [REF(65535)]  1 : 1000000000 (compact)
   #> ATTRIB:
   #>   @141c96720 02 LISTSXP g0c0 [REF(1)] 
   #>     TAG: @15780c9f0 01 SYMSXP g1c0 [MARK,REF(65535),LCK,gp=0x4000] "row.names" (has value)
   #>     @141d29098 13 INTSXP g0c1 [REF(1)] (len=2, tl=0) -2147483648,1000000000
   #>     TAG: @15780d1d0 01 SYMSXP g1c0 [MARK,REF(38209),LCK,gp=0x6000] "class" (has value)
   #>     @141d29290 16 STRSXP g0c1 [REF(65535)] (len=1, tl=0)
   #>       @157891e08 09 CHARSXP g1c2 [MARK,REF(576),gp=0x61,ATT] [ASCII] [cached] "data.frame"
   str(attr(df, "row.names"))
   #>  int [1:1000000000] 1 2 3 4 5 6 7 8 9 10 ...
   bench::mark(attr(df, "row.names"))
   #> # A tibble: 1 × 6
   #>   expression                 min   median `itr/sec` mem_alloc `gc/sec`
   #>   <bch:expr>            <bch:tm> <bch:tm>     <dbl> <bch:byt>    <dbl>
   #> 1 attr(df, "row.names")     41ns     82ns  8860040.        0B        0
   ```
   
   <sup>Created on 2022-11-18 with [reprex v2.0.2](https://reprex.tidyverse.org)</sup>



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

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

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


[GitHub] [arrow-nanoarrow] paleolimbot commented on a diff in pull request #65: [R] Complete ptype inferences and array conversions

Posted by GitBox <gi...@apache.org>.
paleolimbot commented on code in PR #65:
URL: https://github.com/apache/arrow-nanoarrow/pull/65#discussion_r1026761896


##########
r/src/materialize_difftime.h:
##########
@@ -0,0 +1,92 @@
+// 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.
+
+#ifndef R_MATERIALIZE_DIFFTIME_H_INCLUDED
+#define R_MATERIALIZE_DIFFTIME_H_INCLUDED
+
+#include <R.h>
+#include <Rinternals.h>
+
+#include "materialize_common.h"
+#include "materialize_dbl.h"
+#include "nanoarrow.h"
+
+static inline int nanoarrow_materialize_difftime(struct RConverter* converter) {

Review Comment:
   Agreed - I moved them out of materialize.c because that file was getting very crowded but this solution is far from elegant. Most of these conversions would benefit from some optimization for specific pairs of conversions which may be a good opportunity to split out the .c files.



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

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

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