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

[arrow] branch master updated: ARROW-5503 [R]: add read_json()

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 0b552a0  ARROW-5503 [R]: add read_json()
0b552a0 is described below

commit 0b552a0795db43c0bfbcb65be032793c8172e2df
Author: Romain Francois <ro...@rstudio.com>
AuthorDate: Wed Jun 12 13:27:17 2019 +0200

    ARROW-5503 [R]: add read_json()
    
    ``` r
    library(arrow, warn.conflicts = FALSE)
    
    tf <- tempfile()
    writeLines('
        { "hello": 3.5, "world": false, "yo": "thing" }
        { "hello": 3.25, "world": null }
        { "hello": 3.125, "world": null, "yo": "\u5fcd" }
        { "hello": 0.0, "world": true, "yo": null }
    ', tf)
    
    tab <- read_json_arrow(tf)
    tab
    #> arrow::Table
    tab$schema
    #> arrow::Schema
    #> hello: double
    #> world: bool
    #> yo: string
    as_tibble(tab)
    #> # A tibble: 4 x 3
    #>   hello world yo
    #>   <dbl> <lgl> <chr>
    #> 1  3.5  FALSE thing
    #> 2  3.25 NA    <NA>
    #> 3  3.12 NA    忍
    #> 4  0    TRUE  <NA>
    ```
    
    <sup>Created on 2019-06-11 by the [reprex package](https://reprex.tidyverse.org) (v0.3.0.9000)</sup>
    
    Author: Romain Francois <ro...@rstudio.com>
    
    Closes #4518 from romainfrancois/ARROW-5503/read_json and squashes the following commits:
    
    0314b6a0 <Romain Francois> #include <arrow/...> should be protected by ARROW_R_WITH_ARROW
    14cf419e <Romain Francois> + read_json_arrow(as_tibble=)
    d469467a <Romain Francois> use ]
    57d4d076 <Romain Francois> + read_json_arrow()
---
 r/DESCRIPTION                     |   1 +
 r/NAMESPACE                       |   9 +++
 r/R/arrowExports.R                |  16 ++++
 r/R/json.R                        | 159 ++++++++++++++++++++++++++++++++++++++
 r/man/arrow__json__TableReader.Rd |  18 +++++
 r/man/json_parse_options.Rd       |  14 ++++
 r/man/json_read_options.Rd        |  16 ++++
 r/man/json_table_reader.Rd        |  21 +++++
 r/man/read_json_arrow.Rd          |  16 ++++
 r/src/arrowExports.cpp            |  66 ++++++++++++++++
 r/src/arrow_types.h               |   1 +
 r/src/json.cpp                    |  63 +++++++++++++++
 r/tests/testthat/test-json.R      | 106 +++++++++++++++++++++++++
 13 files changed, 506 insertions(+)

diff --git a/r/DESCRIPTION b/r/DESCRIPTION
index d566f74..58a208c 100644
--- a/r/DESCRIPTION
+++ b/r/DESCRIPTION
@@ -67,6 +67,7 @@ Collate:
     'csv.R'
     'dictionary.R'
     'feather.R'
+    'json.R'
     'memory_pool.R'
     'message.R'
     'parquet.R'
diff --git a/r/NAMESPACE b/r/NAMESPACE
index bc4f677..13071b9 100644
--- a/r/NAMESPACE
+++ b/r/NAMESPACE
@@ -53,6 +53,11 @@ S3method(csv_table_reader,default)
 S3method(csv_table_reader,fs_path)
 S3method(dim,"arrow::RecordBatch")
 S3method(dim,"arrow::Table")
+S3method(json_table_reader,"arrow::io::InputStream")
+S3method(json_table_reader,"arrow::json::TableReader")
+S3method(json_table_reader,character)
+S3method(json_table_reader,default)
+S3method(json_table_reader,fs_path)
 S3method(length,"arrow::Array")
 S3method(names,"arrow::RecordBatch")
 S3method(print,"arrow-enum")
@@ -132,6 +137,9 @@ export(int16)
 export(int32)
 export(int64)
 export(int8)
+export(json_parse_options)
+export(json_read_options)
+export(json_table_reader)
 export(list_of)
 export(mmap_create)
 export(mmap_open)
@@ -139,6 +147,7 @@ export(null)
 export(read_arrow)
 export(read_csv_arrow)
 export(read_feather)
+export(read_json_arrow)
 export(read_message)
 export(read_parquet)
 export(read_record_batch)
diff --git a/r/R/arrowExports.R b/r/R/arrowExports.R
index 6359f90..52ff492 100644
--- a/r/R/arrowExports.R
+++ b/r/R/arrowExports.R
@@ -612,6 +612,22 @@ io___FixedSizeBufferWriter__initialize <- function(buffer){
     .Call(`_arrow_io___FixedSizeBufferWriter__initialize` , buffer)
 }
 
+json___ReadOptions__initialize <- function(options){
+    .Call(`_arrow_json___ReadOptions__initialize` , options)
+}
+
+json___ParseOptions__initialize <- function(options){
+    .Call(`_arrow_json___ParseOptions__initialize` , options)
+}
+
+json___TableReader__Make <- function(input, read_options, parse_options){
+    .Call(`_arrow_json___TableReader__Make` , input, read_options, parse_options)
+}
+
+json___TableReader__Read <- function(table_reader){
+    .Call(`_arrow_json___TableReader__Read` , table_reader)
+}
+
 MemoryPool__default <- function(){
     .Call(`_arrow_MemoryPool__default` )
 }
diff --git a/r/R/json.R b/r/R/json.R
new file mode 100644
index 0000000..2de8b94
--- /dev/null
+++ b/r/R/json.R
@@ -0,0 +1,159 @@
+# 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.
+
+#' @include R6.R
+
+#' @include R6.R
+#'
+#' @title class arrow::json::TableReader
+#'
+#' @usage NULL
+#' @format NULL
+#' @docType class
+#'
+#' @section Methods:
+#'
+#' - `Read()` : read the json file as an [arrow::Table][arrow__Table]
+#'
+#' @rdname arrow__json__TableReader
+#' @name arrow__json__TableReader
+`arrow::json::TableReader` <- R6Class("arrow::json::TableReader", inherit = `arrow::Object`,
+  public = list(
+    Read = function() shared_ptr(`arrow::Table`, json___TableReader__Read(self))
+  )
+)
+
+`arrow::json::ReadOptions` <- R6Class("arrow::json::ReadOptions", inherit = `arrow::Object`)
+`arrow::json::ParseOptions` <- R6Class("arrow::json::ParseOptions", inherit = `arrow::Object`)
+
+#' read options for the json reader
+#'
+#' @param use_threads Whether to use the global CPU thread pool
+#' @param block_size Block size we request from the IO layer; also determines the size of chunks when use_threads is `TRUE`. NB: if false, input must end with an empty line
+#'
+#' @export
+json_read_options <- function(use_threads = TRUE, block_size = 1048576L) {
+  shared_ptr(`arrow::json::ReadOptions`, json___ReadOptions__initialize(
+    list(
+      use_threads = use_threads,
+      block_size = block_size
+    )
+  ))
+}
+
+#' Parsing options
+#'
+#' @param newlines_in_values Whether objects may be printed across multiple lines (for example pretty printed),
+#'
+#' @export
+json_parse_options <- function(
+  newlines_in_values = FALSE
+){
+  shared_ptr(`arrow::json::ParseOptions`, json___ParseOptions__initialize(
+    list(
+      newlines_in_values = newlines_in_values
+    )
+  ))
+}
+
+#' Json table reader
+#'
+#' @param file file
+#' @param read_options, see [json_read_options()]
+#' @param parse_options, see [json_parse_options()]
+#' @param ... additional parameters.
+#'
+#' @export
+json_table_reader <- function(file,
+  read_options = json_read_options(),
+  parse_options = json_parse_options(),
+  ...
+){
+  UseMethod("json_table_reader")
+}
+
+#' @importFrom rlang abort
+#' @export
+json_table_reader.default <- function(file,
+  read_options = json_read_options(),
+  parse_options = json_parse_options(),
+  ...
+) {
+  abort("unsupported")
+}
+
+#' @export
+`json_table_reader.character` <- function(file,
+  read_options = json_read_options(),
+  parse_options = json_parse_options(),
+  ...
+){
+  json_table_reader(fs::path_abs(file),
+    read_options = read_options,
+    parse_options = parse_options,
+    ...
+  )
+}
+
+#' @export
+`json_table_reader.fs_path` <- function(file,
+  read_options = json_read_options(),
+  parse_options = json_parse_options(),
+  ...
+){
+  json_table_reader(ReadableFile(file),
+    read_options = read_options,
+    parse_options = parse_options,
+    ...
+  )
+}
+
+#' @export
+`json_table_reader.arrow::io::InputStream` <- function(file,
+  read_options = json_read_options(),
+  parse_options = json_parse_options(),
+  ...
+){
+  shared_ptr(`arrow::json::TableReader`,
+    json___TableReader__Make(file, read_options, parse_options)
+  )
+}
+
+#' @export
+`json_table_reader.arrow::json::TableReader` <- function(file,
+  read_options = json_read_options(),
+  parse_options = json_parse_options(),
+  ...
+){
+  file
+}
+
+#' Read json file into an arrow::Table
+#'
+#' Use [arrow::json::TableReader][arrow__json__TableReader] from [json_table_reader()]
+#'
+#' @param ... Used to construct an arrow::json::TableReader
+#' @param as_tibble convert the [arrow::Table][arrow__Table] to a data frame
+#'
+#' @export
+read_json_arrow <- function(..., as_tibble = TRUE) {
+  tab <- json_table_reader(...)$Read()
+  if (isTRUE(as_tibble)) {
+    tab <- as_tibble(tab)
+  }
+  tab
+}
diff --git a/r/man/arrow__json__TableReader.Rd b/r/man/arrow__json__TableReader.Rd
new file mode 100644
index 0000000..58515a4
--- /dev/null
+++ b/r/man/arrow__json__TableReader.Rd
@@ -0,0 +1,18 @@
+% Generated by roxygen2: do not edit by hand
+% Please edit documentation in R/json.R
+\docType{class}
+\name{arrow__json__TableReader}
+\alias{arrow__json__TableReader}
+\alias{arrow::json::TableReader}
+\title{class arrow::json::TableReader}
+\description{
+class arrow::json::TableReader
+}
+\section{Methods}{
+
+\itemize{
+\item \code{Read()} : read the json file as an \link[=arrow__Table]{arrow::Table}
+}
+}
+
+\keyword{datasets}
diff --git a/r/man/json_parse_options.Rd b/r/man/json_parse_options.Rd
new file mode 100644
index 0000000..c4c4636
--- /dev/null
+++ b/r/man/json_parse_options.Rd
@@ -0,0 +1,14 @@
+% Generated by roxygen2: do not edit by hand
+% Please edit documentation in R/json.R
+\name{json_parse_options}
+\alias{json_parse_options}
+\title{Parsing options}
+\usage{
+json_parse_options(newlines_in_values = FALSE)
+}
+\arguments{
+\item{newlines_in_values}{Whether objects may be printed across multiple lines (for example pretty printed),}
+}
+\description{
+Parsing options
+}
diff --git a/r/man/json_read_options.Rd b/r/man/json_read_options.Rd
new file mode 100644
index 0000000..f2eb7bc
--- /dev/null
+++ b/r/man/json_read_options.Rd
@@ -0,0 +1,16 @@
+% Generated by roxygen2: do not edit by hand
+% Please edit documentation in R/json.R
+\name{json_read_options}
+\alias{json_read_options}
+\title{read options for the json reader}
+\usage{
+json_read_options(use_threads = TRUE, block_size = 1048576L)
+}
+\arguments{
+\item{use_threads}{Whether to use the global CPU thread pool}
+
+\item{block_size}{Block size we request from the IO layer; also determines the size of chunks when use_threads is \code{TRUE}. NB: if false, input must end with an empty line}
+}
+\description{
+read options for the json reader
+}
diff --git a/r/man/json_table_reader.Rd b/r/man/json_table_reader.Rd
new file mode 100644
index 0000000..b2816c2
--- /dev/null
+++ b/r/man/json_table_reader.Rd
@@ -0,0 +1,21 @@
+% Generated by roxygen2: do not edit by hand
+% Please edit documentation in R/json.R
+\name{json_table_reader}
+\alias{json_table_reader}
+\title{Json table reader}
+\usage{
+json_table_reader(file, read_options = json_read_options(),
+  parse_options = json_parse_options(), ...)
+}
+\arguments{
+\item{file}{file}
+
+\item{read_options, }{see \code{\link[=json_read_options]{json_read_options()}}}
+
+\item{parse_options, }{see \code{\link[=json_parse_options]{json_parse_options()}}}
+
+\item{...}{additional parameters.}
+}
+\description{
+Json table reader
+}
diff --git a/r/man/read_json_arrow.Rd b/r/man/read_json_arrow.Rd
new file mode 100644
index 0000000..aa98b69
--- /dev/null
+++ b/r/man/read_json_arrow.Rd
@@ -0,0 +1,16 @@
+% Generated by roxygen2: do not edit by hand
+% Please edit documentation in R/json.R
+\name{read_json_arrow}
+\alias{read_json_arrow}
+\title{Read json file into an arrow::Table}
+\usage{
+read_json_arrow(..., as_tibble = TRUE)
+}
+\arguments{
+\item{...}{Used to construct an arrow::json::TableReader}
+
+\item{as_tibble}{convert the \link[=arrow__Table]{arrow::Table} to a data frame}
+}
+\description{
+Use \link[=arrow__json__TableReader]{arrow::json::TableReader} from \code{\link[=json_table_reader]{json_table_reader()}}
+}
diff --git a/r/src/arrowExports.cpp b/r/src/arrowExports.cpp
index 37e9a3d..f16179b 100644
--- a/r/src/arrowExports.cpp
+++ b/r/src/arrowExports.cpp
@@ -2352,6 +2352,68 @@ RcppExport SEXP _arrow_io___FixedSizeBufferWriter__initialize(SEXP buffer_sexp){
 }
 #endif
 
+// json.cpp
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::json::ReadOptions> json___ReadOptions__initialize(List_ options);
+RcppExport SEXP _arrow_json___ReadOptions__initialize(SEXP options_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<List_>::type options(options_sexp);
+	return Rcpp::wrap(json___ReadOptions__initialize(options));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_json___ReadOptions__initialize(SEXP options_sexp){
+	Rf_error("Cannot call json___ReadOptions__initialize(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// json.cpp
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::json::ParseOptions> json___ParseOptions__initialize(List_ options);
+RcppExport SEXP _arrow_json___ParseOptions__initialize(SEXP options_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<List_>::type options(options_sexp);
+	return Rcpp::wrap(json___ParseOptions__initialize(options));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_json___ParseOptions__initialize(SEXP options_sexp){
+	Rf_error("Cannot call json___ParseOptions__initialize(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// json.cpp
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::json::TableReader> json___TableReader__Make(const std::shared_ptr<arrow::io::InputStream>& input, const std::shared_ptr<arrow::json::ReadOptions>& read_options, const std::shared_ptr<arrow::json::ParseOptions>& parse_options);
+RcppExport SEXP _arrow_json___TableReader__Make(SEXP input_sexp, SEXP read_options_sexp, SEXP parse_options_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::io::InputStream>&>::type input(input_sexp);
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::json::ReadOptions>&>::type read_options(read_options_sexp);
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::json::ParseOptions>&>::type parse_options(parse_options_sexp);
+	return Rcpp::wrap(json___TableReader__Make(input, read_options, parse_options));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_json___TableReader__Make(SEXP input_sexp, SEXP read_options_sexp, SEXP parse_options_sexp){
+	Rf_error("Cannot call json___TableReader__Make(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
+// json.cpp
+#if defined(ARROW_R_WITH_ARROW)
+std::shared_ptr<arrow::Table> json___TableReader__Read(const std::shared_ptr<arrow::json::TableReader>& table_reader);
+RcppExport SEXP _arrow_json___TableReader__Read(SEXP table_reader_sexp){
+BEGIN_RCPP
+	Rcpp::traits::input_parameter<const std::shared_ptr<arrow::json::TableReader>&>::type table_reader(table_reader_sexp);
+	return Rcpp::wrap(json___TableReader__Read(table_reader));
+END_RCPP
+}
+#else
+RcppExport SEXP _arrow_json___TableReader__Read(SEXP table_reader_sexp){
+	Rf_error("Cannot call json___TableReader__Read(). Please use arrow::install_arrow() to install required runtime libraries. ");
+}
+#endif
+
 // memorypool.cpp
 #if defined(ARROW_R_WITH_ARROW)
 std::shared_ptr<arrow::MemoryPool> MemoryPool__default();
@@ -3363,6 +3425,10 @@ static const R_CallMethodDef CallEntries[] = {
 		{ "_arrow_io___MockOutputStream__initialize", (DL_FUNC) &_arrow_io___MockOutputStream__initialize, 0}, 
 		{ "_arrow_io___MockOutputStream__GetExtentBytesWritten", (DL_FUNC) &_arrow_io___MockOutputStream__GetExtentBytesWritten, 1}, 
 		{ "_arrow_io___FixedSizeBufferWriter__initialize", (DL_FUNC) &_arrow_io___FixedSizeBufferWriter__initialize, 1}, 
+		{ "_arrow_json___ReadOptions__initialize", (DL_FUNC) &_arrow_json___ReadOptions__initialize, 1}, 
+		{ "_arrow_json___ParseOptions__initialize", (DL_FUNC) &_arrow_json___ParseOptions__initialize, 1}, 
+		{ "_arrow_json___TableReader__Make", (DL_FUNC) &_arrow_json___TableReader__Make, 3}, 
+		{ "_arrow_json___TableReader__Read", (DL_FUNC) &_arrow_json___TableReader__Read, 1}, 
 		{ "_arrow_MemoryPool__default", (DL_FUNC) &_arrow_MemoryPool__default, 0}, 
 		{ "_arrow_MemoryPool__bytes_allocated", (DL_FUNC) &_arrow_MemoryPool__bytes_allocated, 1}, 
 		{ "_arrow_MemoryPool__max_memory", (DL_FUNC) &_arrow_MemoryPool__max_memory, 1}, 
diff --git a/r/src/arrow_types.h b/r/src/arrow_types.h
index f2ea6dd..867a89f 100644
--- a/r/src/arrow_types.h
+++ b/r/src/arrow_types.h
@@ -179,6 +179,7 @@ inline std::shared_ptr<T> extract(SEXP x) {
 #include <arrow/ipc/writer.h>
 #include <arrow/type.h>
 #include <arrow/util/compression.h>
+#include <arrow/json/reader.h>
 
 RCPP_EXPOSED_ENUM_NODECL(arrow::Type::type)
 RCPP_EXPOSED_ENUM_NODECL(arrow::DateUnit)
diff --git a/r/src/json.cpp b/r/src/json.cpp
new file mode 100644
index 0000000..0479121
--- /dev/null
+++ b/r/src/json.cpp
@@ -0,0 +1,63 @@
+// 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.
+
+#include "./arrow_types.h"
+#if defined(ARROW_R_WITH_ARROW)
+
+#include <arrow/json/reader.h>
+
+using Rcpp::CharacterVector;
+using Rcpp::List_;
+
+// [[arrow::export]]
+std::shared_ptr<arrow::json::ReadOptions> json___ReadOptions__initialize(List_ options) {
+  auto res =
+      std::make_shared<arrow::json::ReadOptions>(arrow::json::ReadOptions::Defaults());
+  res->use_threads = options["use_threads"];
+  res->block_size = options["block_size"];
+  return res;
+}
+
+// [[arrow::export]]
+std::shared_ptr<arrow::json::ParseOptions> json___ParseOptions__initialize(
+    List_ options) {
+  auto res =
+      std::make_shared<arrow::json::ParseOptions>(arrow::json::ParseOptions::Defaults());
+  res->newlines_in_values = options["newlines_in_values"];
+  return res;
+}
+
+// [[arrow::export]]
+std::shared_ptr<arrow::json::TableReader> json___TableReader__Make(
+    const std::shared_ptr<arrow::io::InputStream>& input,
+    const std::shared_ptr<arrow::json::ReadOptions>& read_options,
+    const std::shared_ptr<arrow::json::ParseOptions>& parse_options) {
+  std::shared_ptr<arrow::json::TableReader> table_reader;
+  STOP_IF_NOT_OK(arrow::json::TableReader::Make(
+      arrow::default_memory_pool(), input, *read_options, *parse_options, &table_reader));
+  return table_reader;
+}
+
+// [[arrow::export]]
+std::shared_ptr<arrow::Table> json___TableReader__Read(
+    const std::shared_ptr<arrow::json::TableReader>& table_reader) {
+  std::shared_ptr<arrow::Table> table;
+  STOP_IF_NOT_OK(table_reader->Read(&table));
+  return table;
+}
+
+#endif
diff --git a/r/tests/testthat/test-json.R b/r/tests/testthat/test-json.R
new file mode 100644
index 0000000..627e445
--- /dev/null
+++ b/r/tests/testthat/test-json.R
@@ -0,0 +1,106 @@
+# 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.
+
+context("arrow::json::TableReader")
+
+test_that("Can read json file with scalars columns (ARROW-5503)", {
+  tf <- tempfile()
+  writeLines('
+    { "hello": 3.5, "world": false, "yo": "thing" }
+    { "hello": 3.25, "world": null }
+    { "hello": 3.125, "world": null, "yo": "\u5fcd" }
+    { "hello": 0.0, "world": true, "yo": null }
+  ', tf)
+
+  tab1 <- read_json_arrow(tf, as_tibble = FALSE)
+  tab2 <- read_json_arrow(mmap_open(tf), as_tibble = FALSE)
+  tab3 <- read_json_arrow(ReadableFile(tf), as_tibble = FALSE)
+
+  expect_equal(tab1, tab2)
+  expect_equal(tab1, tab3)
+
+  expect_equal(
+    tab1$schema,
+    schema(hello = float64(), world = boolean(), yo = utf8())
+  )
+  tib <- as_tibble(tab1)
+  expect_equal(tib$hello, c(3.5, 3.25, 3.125, 0))
+  expect_equal(tib$world, c(FALSE, NA, NA, TRUE))
+  expect_equal(tib$yo, c("thing", NA, "\u5fcd", NA))
+
+  unlink(tf)
+})
+
+test_that("read_json_arrow() converts to tibble", {
+  tf <- tempfile()
+  writeLines('
+    { "hello": 3.5, "world": false, "yo": "thing" }
+    { "hello": 3.25, "world": null }
+    { "hello": 3.125, "world": null, "yo": "\u5fcd" }
+    { "hello": 0.0, "world": true, "yo": null }
+  ', tf)
+
+  tab1 <- read_json_arrow(tf)
+  tab2 <- read_json_arrow(mmap_open(tf))
+  tab3 <- read_json_arrow(ReadableFile(tf))
+
+  expect_is(tab1, "tbl_df")
+  expect_is(tab2, "tbl_df")
+  expect_is(tab3, "tbl_df")
+
+  expect_equal(tab1, tab2)
+  expect_equal(tab1, tab3)
+
+  expect_equal(tab1$hello, c(3.5, 3.25, 3.125, 0))
+  expect_equal(tab1$world, c(FALSE, NA, NA, TRUE))
+  expect_equal(tab1$yo, c("thing", NA, "\u5fcd", NA))
+
+  unlink(tf)
+})
+
+test_that("Can read json file with nested columns (ARROW-5503)", {
+  tf <- tempfile()
+  writeLines('
+    { "hello": 3.5, "world": false, "yo": "thing", "arr": [1, 2, 3], "nuf": {} }
+    { "hello": 3.25, "world": null, "arr": [2], "nuf": null }
+    { "hello": 3.125, "world": null, "yo": "\u5fcd", "arr": [], "nuf": { "ps": 78 } }
+    { "hello": 0.0, "world": true, "yo": null, "arr": null, "nuf": { "ps": 90 } }
+  ', tf)
+
+  tab1 <- read_json_arrow(tf, as_tibble = FALSE)
+  tab2 <- read_json_arrow(mmap_open(tf), as_tibble = FALSE)
+  tab3 <- read_json_arrow(ReadableFile(tf), as_tibble = FALSE)
+
+  expect_equal(tab1, tab2)
+  expect_equal(tab1, tab3)
+
+  expect_equal(
+    tab1$schema,
+    schema(
+      hello = float64(),
+      world = boolean(),
+      yo = utf8(),
+      arr = list_of(int64()),
+      nuf = struct(ps = int64())
+    )
+  )
+  # cannot yet test list and struct types in R api
+  # tib <- as_tibble(tab1)
+
+  unlink(tf)
+})
+