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 2020/10/08 00:23:46 UTC

[GitHub] [arrow] bkietz opened a new pull request #8389: ARROW-8296: [C++][Dataset] Add IpcFileWriteOptions

bkietz opened a new pull request #8389:
URL: https://github.com/apache/arrow/pull/8389


   


----------------------------------------------------------------
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.

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



[GitHub] [arrow] bkietz commented on a change in pull request #8389: ARROW-8296: [C++][Dataset] Add IpcFileWriteOptions

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #8389:
URL: https://github.com/apache/arrow/pull/8389#discussion_r502558340



##########
File path: r/src/recordbatchreader.cpp
##########
@@ -74,6 +76,12 @@ int ipc___RecordBatchFileReader__num_record_batches(
   return reader->num_record_batches();
 }
 
+// [[arrow::export]]
+cpp11::strings ipc___RecordBatchFileReader__metadata(

Review comment:
       I'll revert this, then




----------------------------------------------------------------
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.

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



[GitHub] [arrow] nealrichardson commented on a change in pull request #8389: ARROW-8296: [C++][Dataset] Add IpcFileWriteOptions

Posted by GitBox <gi...@apache.org>.
nealrichardson commented on a change in pull request #8389:
URL: https://github.com/apache/arrow/pull/8389#discussion_r502656073



##########
File path: r/tests/testthat/test-dataset.R
##########
@@ -943,10 +943,42 @@ test_that("Dataset writing: from RecordBatch", {
   )
 })
 
+test_that("Writing a dataset: Ipc format options & compression", {
+  skip_on_os("windows") # https://issues.apache.org/jira/browse/ARROW-9651
+  ds <- open_dataset(csv_dir, partitioning = "part", format = "csv")
+  dst_dir <- make_temp_dir()
+
+  metadata <- c(hello = "world", eh = "!")
+  codec <- NULL
+  if (codec_is_available("zstd")) {
+    codec <- Codec$create("zstd")
+  }
+
+  write_dataset(ds, dst_dir, format = "feather", metadata = metadata, codec = codec)
+  expect_true(dir.exists(dst_dir))
+
+  file <- ds$filesystem$OpenInputStream(paste(dst_dir, dir(dst_dir)[[1]], sep = "/"))
+  expect_equivalent(metadata, RecordBatchFileReader$create(file)$metadata)

Review comment:
       What is footer metadata for? 
   
   Would it make sense/be ok with you to defer wiring up footer metadata in R for this release? I don't understand it enough to advise on how to expose it in R in a way that is useful and not confusing.




----------------------------------------------------------------
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.

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



[GitHub] [arrow] nealrichardson commented on a change in pull request #8389: ARROW-8296: [C++][Dataset] Add IpcFileWriteOptions

Posted by GitBox <gi...@apache.org>.
nealrichardson commented on a change in pull request #8389:
URL: https://github.com/apache/arrow/pull/8389#discussion_r502591443



##########
File path: r/R/dataset-format.R
##########
@@ -139,6 +139,13 @@ FileWriteOptions <- R6Class("FileWriteOptions", inherit = ArrowObject,
         dataset___ParquetFileWriteOptions__update(self,
             ParquetWriterProperties$create(...),
             ParquetArrowWriterProperties$create(...))
+      } else if (self$type == "ipc") {
+        args <- list(...)
+        dataset___IpcFileWriteOptions__update(self,
+            get_ipc_use_legacy_format(args$use_legacy_format),
+            args$codec,

Review comment:
       so "uncompressed" codec is a nullptr? that means you can have `std::shared_ptr<arrow::Codec>` in the `dataset___IpcFileWriteOptions__update` signature, and then check for nullptr (and not R NULL)?




----------------------------------------------------------------
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.

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



[GitHub] [arrow] nealrichardson commented on a change in pull request #8389: ARROW-8296: [C++][Dataset] Add IpcFileWriteOptions

Posted by GitBox <gi...@apache.org>.
nealrichardson commented on a change in pull request #8389:
URL: https://github.com/apache/arrow/pull/8389#discussion_r502545157



##########
File path: r/src/arrow_metadata.h
##########
@@ -0,0 +1,38 @@
+// 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/util/key_value_metadata.h>
+
+inline std::shared_ptr<arrow::KeyValueMetadata> KeyValueMetadata__Make(

Review comment:
       Two thoughts:
   
   1. We already have KeyValueMetadata support in R, so if these helpers are necessary, I'd expect to see them used in those other places too.
   2. It looks like these functions are opposites, one takes a named character vector and turns it into arrow::KVM, and the other takes KVM and makes a named character vector. Can we make the function names more obvious about that?

##########
File path: r/tests/testthat/test-dataset.R
##########
@@ -943,10 +943,42 @@ test_that("Dataset writing: from RecordBatch", {
   )
 })
 
+test_that("Writing a dataset: Ipc format options & compression", {
+  skip_on_os("windows") # https://issues.apache.org/jira/browse/ARROW-9651
+  ds <- open_dataset(csv_dir, partitioning = "part", format = "csv")
+  dst_dir <- make_temp_dir()
+
+  metadata <- c(hello = "world", eh = "!")
+  codec <- NULL
+  if (codec_is_available("zstd")) {
+    codec <- Codec$create("zstd")
+  }
+
+  write_dataset(ds, dst_dir, format = "feather", metadata = metadata, codec = codec)
+  expect_true(dir.exists(dst_dir))
+
+  file <- ds$filesystem$OpenInputStream(paste(dst_dir, dir(dst_dir)[[1]], sep = "/"))
+  expect_equivalent(metadata, RecordBatchFileReader$create(file)$metadata)

Review comment:
       ```suggestion
     file <- dir(dst_dir, full.names = TRUE)[[1]]
     expect_equivalent(read_feather(file, as_data_frame = FALSE)$metadata, metadata)
   ```

##########
File path: r/configure
##########
@@ -201,7 +201,10 @@ else
 fi
 
 # Write to Makevars
-sed -e "s|@cflags@|$PKG_CFLAGS|" -e "s|@libs@|$PKG_LIBS|" src/Makevars.in > src/Makevars
+sed -e "s|@cflags@|$PKG_CFLAGS|" \

Review comment:
       Please revert this change and the one in Makevars.in. This will provoke the wrath of CRAN (and it should cause at least one nightly build failure).

##########
File path: r/src/recordbatchreader.cpp
##########
@@ -74,6 +76,12 @@ int ipc___RecordBatchFileReader__num_record_batches(
   return reader->num_record_batches();
 }
 
+// [[arrow::export]]
+cpp11::strings ipc___RecordBatchFileReader__metadata(

Review comment:
       It's cool that you added this but it's probably not necessary; see below for a simpler test.

##########
File path: r/src/arrow_cpp11.h
##########
@@ -157,19 +157,42 @@ struct ns {
   static SEXP arrow;
 };
 
+// Specialize this struct to define a default value to be used when NULL is given.

Review comment:
       Why is this change necessary?

##########
File path: r/R/dataset-format.R
##########
@@ -139,6 +139,13 @@ FileWriteOptions <- R6Class("FileWriteOptions", inherit = ArrowObject,
         dataset___ParquetFileWriteOptions__update(self,
             ParquetWriterProperties$create(...),
             ParquetArrowWriterProperties$create(...))
+      } else if (self$type == "ipc") {
+        args <- list(...)
+        dataset___IpcFileWriteOptions__update(self,
+            get_ipc_use_legacy_format(args$use_legacy_format),
+            args$codec,

Review comment:
       I think a better approach for `codec` would be to allow you to specify the parameters like `write_feather` (though fine if you want to provide your own Codec I guess). so something like this:
   
   ```suggestion
               args$codec %||% Codec$create(
                 args$compression %||% "uncompressed", 
                 args$compression_level %||% NA
               ),
   ```
   
   Perhaps this lets you get rid of your funky `nil_value` helper too? With this change, you're always providing a Codec to C++, never `NULL`. 

##########
File path: r/R/dataset-write.R
##########
@@ -44,7 +44,8 @@
 #' @param filesystem A [FileSystem] where the dataset should be written if it is a
 #' string file path; default is the local file system
 #' @param ... additional format-specific arguments. For available Parquet
-#' options, see [write_parquet()].
+#' options, see [write_parquet()]. For available Feather options, see
+#' [RecordBatchFileWriter$create()].

Review comment:
       I don't think that doc link will work, and if you were to do `make doc`, this will probably fail check due to a bad xref. The options also aren't well documented there, nor are they all exposed in write_feather, so it might be best to enumerate them here.

##########
File path: r/tests/testthat/test-dataset.R
##########
@@ -943,10 +943,42 @@ test_that("Dataset writing: from RecordBatch", {
   )
 })
 
+test_that("Writing a dataset: Ipc format options & compression", {
+  skip_on_os("windows") # https://issues.apache.org/jira/browse/ARROW-9651
+  ds <- open_dataset(csv_dir, partitioning = "part", format = "csv")
+  dst_dir <- make_temp_dir()
+
+  metadata <- c(hello = "world", eh = "!")
+  codec <- NULL
+  if (codec_is_available("zstd")) {
+    codec <- Codec$create("zstd")
+  }
+
+  write_dataset(ds, dst_dir, format = "feather", metadata = metadata, codec = codec)
+  expect_true(dir.exists(dst_dir))
+
+  file <- ds$filesystem$OpenInputStream(paste(dst_dir, dir(dst_dir)[[1]], sep = "/"))
+  expect_equivalent(metadata, RecordBatchFileReader$create(file)$metadata)
+
+  new_ds <- open_dataset(dst_dir, format = "feather")
+  expect_equivalent(
+    new_ds %>%
+      select(string = chr, integer = int) %>%
+      filter(integer > 6 & integer < 11) %>%
+      collect() %>%
+      summarize(mean = mean(integer)),
+    df1 %>%
+      select(string = chr, integer = int) %>%
+      filter(integer > 6) %>%
+      summarize(mean = mean(integer))
+  )
+})
+
 test_that("Writing a dataset: Parquet format options", {
   skip_on_os("windows") # https://issues.apache.org/jira/browse/ARROW-9651
   ds <- open_dataset(csv_dir, partitioning = "part", format = "csv")
   dst_dir <- make_temp_dir()
+  dst_dir_no_truncated_timestamps <- make_temp_dir()

Review comment:
       Why this change? I wouldn't expect this PR to affect parquet writing options, should it?




----------------------------------------------------------------
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.

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



[GitHub] [arrow] kou commented on pull request #8389: ARROW-8296: [C++][Dataset] Add IpcFileWriteOptions

Posted by GitBox <gi...@apache.org>.
kou commented on pull request #8389:
URL: https://github.com/apache/arrow/pull/8389#issuecomment-705903570


   I'll take a look it and push a fix.


----------------------------------------------------------------
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.

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



[GitHub] [arrow] jorisvandenbossche commented on a change in pull request #8389: ARROW-8296: [C++][Dataset] Add IpcFileWriteOptions

Posted by GitBox <gi...@apache.org>.
jorisvandenbossche commented on a change in pull request #8389:
URL: https://github.com/apache/arrow/pull/8389#discussion_r502468990



##########
File path: cpp/src/arrow/dataset/file_ipc.cc
##########
@@ -185,7 +185,13 @@ Result<std::shared_ptr<FileWriter>> IpcFileFormat::MakeWriter(
 
   auto ipc_options = checked_pointer_cast<IpcFileWriteOptions>(options);
 
-  ARROW_ASSIGN_OR_RAISE(auto writer, ipc::MakeFileWriter(destination, schema));
+  // override use_threads to avoid nested parallelism
+  ipc_options->options->use_threads = false;

Review comment:
       Should this in theory only be done if we have multiple files / fragments? (like the issue we had for parquet)




----------------------------------------------------------------
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.

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



[GitHub] [arrow] nealrichardson closed pull request #8389: ARROW-8296: [C++][Dataset] Add IpcFileWriteOptions

Posted by GitBox <gi...@apache.org>.
nealrichardson closed pull request #8389:
URL: https://github.com/apache/arrow/pull/8389


   


----------------------------------------------------------------
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.

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



[GitHub] [arrow] bkietz commented on a change in pull request #8389: ARROW-8296: [C++][Dataset] Add IpcFileWriteOptions

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #8389:
URL: https://github.com/apache/arrow/pull/8389#discussion_r502661252



##########
File path: r/tests/testthat/test-dataset.R
##########
@@ -943,10 +943,42 @@ test_that("Dataset writing: from RecordBatch", {
   )
 })
 
+test_that("Writing a dataset: Ipc format options & compression", {
+  skip_on_os("windows") # https://issues.apache.org/jira/browse/ARROW-9651
+  ds <- open_dataset(csv_dir, partitioning = "part", format = "csv")
+  dst_dir <- make_temp_dir()
+
+  metadata <- c(hello = "world", eh = "!")
+  codec <- NULL
+  if (codec_is_available("zstd")) {
+    codec <- Codec$create("zstd")
+  }
+
+  write_dataset(ds, dst_dir, format = "feather", metadata = metadata, codec = codec)
+  expect_true(dir.exists(dst_dir))
+
+  file <- ds$filesystem$OpenInputStream(paste(dst_dir, dir(dst_dir)[[1]], sep = "/"))
+  expect_equivalent(metadata, RecordBatchFileReader$create(file)$metadata)

Review comment:
       alright




----------------------------------------------------------------
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.

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



[GitHub] [arrow] bkietz commented on a change in pull request #8389: ARROW-8296: [C++][Dataset] Add IpcFileWriteOptions

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #8389:
URL: https://github.com/apache/arrow/pull/8389#discussion_r502628716



##########
File path: r/R/dataset-format.R
##########
@@ -139,6 +139,13 @@ FileWriteOptions <- R6Class("FileWriteOptions", inherit = ArrowObject,
         dataset___ParquetFileWriteOptions__update(self,
             ParquetWriterProperties$create(...),
             ParquetArrowWriterProperties$create(...))
+      } else if (self$type == "ipc") {
+        args <- list(...)
+        dataset___IpcFileWriteOptions__update(self,
+            get_ipc_use_legacy_format(args$use_legacy_format),
+            args$codec,

Review comment:
       Thus we'd either need to alter `arrow:::shared_ptr()` to support nullptrs or we could implement the reverse mapping and allow R `NULL` -> c++ `shared_ptr<T>(nullptr)`. For now, I'll revert `nil_value` and add overloads of `dataset___IpcFileWriteOptions__update`




----------------------------------------------------------------
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.

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



[GitHub] [arrow] bkietz commented on a change in pull request #8389: ARROW-8296: [C++][Dataset] Add IpcFileWriteOptions

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #8389:
URL: https://github.com/apache/arrow/pull/8389#discussion_r502506462



##########
File path: cpp/src/arrow/dataset/file_ipc.cc
##########
@@ -185,7 +185,13 @@ Result<std::shared_ptr<FileWriter>> IpcFileFormat::MakeWriter(
 
   auto ipc_options = checked_pointer_cast<IpcFileWriteOptions>(options);
 
-  ARROW_ASSIGN_OR_RAISE(auto writer, ipc::MakeFileWriter(destination, schema));
+  // override use_threads to avoid nested parallelism
+  ipc_options->options->use_threads = false;

Review comment:
       That sounds acceptable but I'd like to defer it to a follow up




----------------------------------------------------------------
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.

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



[GitHub] [arrow] kou commented on pull request #8389: ARROW-8296: [C++][Dataset] Add IpcFileWriteOptions

Posted by GitBox <gi...@apache.org>.
kou commented on pull request #8389:
URL: https://github.com/apache/arrow/pull/8389#issuecomment-705943321


   Done.
   
   +1 on the GLib part.


----------------------------------------------------------------
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.

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



[GitHub] [arrow] bkietz commented on a change in pull request #8389: ARROW-8296: [C++][Dataset] Add IpcFileWriteOptions

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #8389:
URL: https://github.com/apache/arrow/pull/8389#discussion_r502559021



##########
File path: r/R/dataset-write.R
##########
@@ -44,7 +44,8 @@
 #' @param filesystem A [FileSystem] where the dataset should be written if it is a
 #' string file path; default is the local file system
 #' @param ... additional format-specific arguments. For available Parquet
-#' options, see [write_parquet()].
+#' options, see [write_parquet()]. For available Feather options, see
+#' [RecordBatchFileWriter$create()].

Review comment:
       Will do




----------------------------------------------------------------
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.

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



[GitHub] [arrow] bkietz commented on a change in pull request #8389: ARROW-8296: [C++][Dataset] Add IpcFileWriteOptions

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #8389:
URL: https://github.com/apache/arrow/pull/8389#discussion_r502558888



##########
File path: r/tests/testthat/test-dataset.R
##########
@@ -943,10 +943,42 @@ test_that("Dataset writing: from RecordBatch", {
   )
 })
 
+test_that("Writing a dataset: Ipc format options & compression", {
+  skip_on_os("windows") # https://issues.apache.org/jira/browse/ARROW-9651
+  ds <- open_dataset(csv_dir, partitioning = "part", format = "csv")
+  dst_dir <- make_temp_dir()
+
+  metadata <- c(hello = "world", eh = "!")
+  codec <- NULL
+  if (codec_is_available("zstd")) {
+    codec <- Codec$create("zstd")
+  }
+
+  write_dataset(ds, dst_dir, format = "feather", metadata = metadata, codec = codec)
+  expect_true(dir.exists(dst_dir))
+
+  file <- ds$filesystem$OpenInputStream(paste(dst_dir, dir(dst_dir)[[1]], sep = "/"))
+  expect_equivalent(metadata, RecordBatchFileReader$create(file)$metadata)
+
+  new_ds <- open_dataset(dst_dir, format = "feather")
+  expect_equivalent(
+    new_ds %>%
+      select(string = chr, integer = int) %>%
+      filter(integer > 6 & integer < 11) %>%
+      collect() %>%
+      summarize(mean = mean(integer)),
+    df1 %>%
+      select(string = chr, integer = int) %>%
+      filter(integer > 6) %>%
+      summarize(mean = mean(integer))
+  )
+})
+
 test_that("Writing a dataset: Parquet format options", {
   skip_on_os("windows") # https://issues.apache.org/jira/browse/ARROW-9651
   ds <- open_dataset(csv_dir, partitioning = "part", format = "csv")
   dst_dir <- make_temp_dir()
+  dst_dir_no_truncated_timestamps <- make_temp_dir()

Review comment:
       My intent was to just make the fact that we have a second destination directory for the no truncated timestamps case clearer by introducing a named variable. I don't think it changes the test's semantics at all




----------------------------------------------------------------
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.

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



[GitHub] [arrow] nealrichardson commented on a change in pull request #8389: ARROW-8296: [C++][Dataset] Add IpcFileWriteOptions

Posted by GitBox <gi...@apache.org>.
nealrichardson commented on a change in pull request #8389:
URL: https://github.com/apache/arrow/pull/8389#discussion_r502590671



##########
File path: r/src/arrow_metadata.h
##########
@@ -0,0 +1,38 @@
+// 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/util/key_value_metadata.h>
+
+inline std::shared_ptr<arrow::KeyValueMetadata> KeyValueMetadata__Make(

Review comment:
       Yes, if these helpers are useful, let's use them everywhere. And yeah I like those names better, thanks.




----------------------------------------------------------------
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.

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



[GitHub] [arrow] bkietz commented on a change in pull request #8389: ARROW-8296: [C++][Dataset] Add IpcFileWriteOptions

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #8389:
URL: https://github.com/apache/arrow/pull/8389#discussion_r502627767



##########
File path: r/R/dataset-format.R
##########
@@ -139,6 +139,13 @@ FileWriteOptions <- R6Class("FileWriteOptions", inherit = ArrowObject,
         dataset___ParquetFileWriteOptions__update(self,
             ParquetWriterProperties$create(...),
             ParquetArrowWriterProperties$create(...))
+      } else if (self$type == "ipc") {
+        args <- list(...)
+        dataset___IpcFileWriteOptions__update(self,
+            get_ipc_use_legacy_format(args$use_legacy_format),
+            args$codec,

Review comment:
       Unfortunately, the `shared_ptr` utility already does the reverse mapping (c++ `shared_ptr<Codec>(nullptr)` -> R `NULL`) https://github.com/apache/arrow/blob/20cd08a8143eed7b48686ac3e696252dc5fdee88/r/R/arrow-package.R#L118-L124 so:
   
   ```r
   uncompressed_type <- arrow:::compression_from_name("uncompressed")
   uncompressed_codec <- arrow:::shared_ptr(Codec, arrow:::util___Codec__Create(uncompressed_type, NA))
   assert_that(is.null(uncompressed_codec))
   ```




----------------------------------------------------------------
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.

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



[GitHub] [arrow] bkietz commented on a change in pull request #8389: ARROW-8296: [C++][Dataset] Add IpcFileWriteOptions

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #8389:
URL: https://github.com/apache/arrow/pull/8389#discussion_r502556438



##########
File path: r/configure
##########
@@ -201,7 +201,10 @@ else
 fi
 
 # Write to Makevars
-sed -e "s|@cflags@|$PKG_CFLAGS|" -e "s|@libs@|$PKG_LIBS|" src/Makevars.in > src/Makevars
+sed -e "s|@cflags@|$PKG_CFLAGS|" \

Review comment:
       will do




----------------------------------------------------------------
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.

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



[GitHub] [arrow] nealrichardson commented on a change in pull request #8389: ARROW-8296: [C++][Dataset] Add IpcFileWriteOptions

Posted by GitBox <gi...@apache.org>.
nealrichardson commented on a change in pull request #8389:
URL: https://github.com/apache/arrow/pull/8389#discussion_r502591443



##########
File path: r/R/dataset-format.R
##########
@@ -139,6 +139,13 @@ FileWriteOptions <- R6Class("FileWriteOptions", inherit = ArrowObject,
         dataset___ParquetFileWriteOptions__update(self,
             ParquetWriterProperties$create(...),
             ParquetArrowWriterProperties$create(...))
+      } else if (self$type == "ipc") {
+        args <- list(...)
+        dataset___IpcFileWriteOptions__update(self,
+            get_ipc_use_legacy_format(args$use_legacy_format),
+            args$codec,

Review comment:
       so "uncompressed" codec is a nullptr? that means you can have `std::shared_ptr<arrow::Codec>` in the `dataset___IpcFileWriteOptions__update` signature, and then check for nullptr (and not worry about R NULL)?




----------------------------------------------------------------
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.

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



[GitHub] [arrow] nealrichardson commented on pull request #8389: ARROW-8296: [C++][Dataset] Add IpcFileWriteOptions

Posted by GitBox <gi...@apache.org>.
nealrichardson commented on pull request #8389:
URL: https://github.com/apache/arrow/pull/8389#issuecomment-706431994


   I'm fixing the R and then will merge


----------------------------------------------------------------
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.

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



[GitHub] [arrow] bkietz commented on a change in pull request #8389: ARROW-8296: [C++][Dataset] Add IpcFileWriteOptions

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #8389:
URL: https://github.com/apache/arrow/pull/8389#discussion_r502561853



##########
File path: r/R/dataset-format.R
##########
@@ -139,6 +139,13 @@ FileWriteOptions <- R6Class("FileWriteOptions", inherit = ArrowObject,
         dataset___ParquetFileWriteOptions__update(self,
             ParquetWriterProperties$create(...),
             ParquetArrowWriterProperties$create(...))
+      } else if (self$type == "ipc") {
+        args <- list(...)
+        dataset___IpcFileWriteOptions__update(self,
+            get_ipc_use_legacy_format(args$use_legacy_format),
+            args$codec,

Review comment:
       I'll add support for `compression, compression_level` but unfortunately `Codec$create("uncompressed")` doesn't work and we'd still need to fall back to either `nil_value` or overloads of `dataset___IpcFileWriteOptions__update`. It would also be my preference to have a non-nullptr codec in all cases but unfortunately the C++ library uses `nullptr` to indicate a lack of compression (there is no "uncompressed" codec), fixing of which is outside scope for this PR.




----------------------------------------------------------------
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.

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



[GitHub] [arrow] github-actions[bot] commented on pull request #8389: ARROW-8296: [C++][Dataset] Add IpcFileWriteOptions

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #8389:
URL: https://github.com/apache/arrow/pull/8389#issuecomment-705269694


   https://issues.apache.org/jira/browse/ARROW-8296


----------------------------------------------------------------
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.

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



[GitHub] [arrow] bkietz commented on pull request #8389: ARROW-8296: [C++][Dataset] Add IpcFileWriteOptions

Posted by GitBox <gi...@apache.org>.
bkietz commented on pull request #8389:
URL: https://github.com/apache/arrow/pull/8389#issuecomment-705901241


   @kou not sure how to idiomatically propagate this change to GLib; it seems there's no way to raise an exception when setting a property?


----------------------------------------------------------------
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.

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



[GitHub] [arrow] bkietz commented on a change in pull request #8389: ARROW-8296: [C++][Dataset] Add IpcFileWriteOptions

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #8389:
URL: https://github.com/apache/arrow/pull/8389#discussion_r502651537



##########
File path: r/tests/testthat/test-dataset.R
##########
@@ -943,10 +943,42 @@ test_that("Dataset writing: from RecordBatch", {
   )
 })
 
+test_that("Writing a dataset: Ipc format options & compression", {
+  skip_on_os("windows") # https://issues.apache.org/jira/browse/ARROW-9651
+  ds <- open_dataset(csv_dir, partitioning = "part", format = "csv")
+  dst_dir <- make_temp_dir()
+
+  metadata <- c(hello = "world", eh = "!")
+  codec <- NULL
+  if (codec_is_available("zstd")) {
+    codec <- Codec$create("zstd")
+  }
+
+  write_dataset(ds, dst_dir, format = "feather", metadata = metadata, codec = codec)
+  expect_true(dir.exists(dst_dir))
+
+  file <- ds$filesystem$OpenInputStream(paste(dst_dir, dir(dst_dir)[[1]], sep = "/"))
+  expect_equivalent(metadata, RecordBatchFileReader$create(file)$metadata)

Review comment:
       Unfortunately this doesn't work; the footer metadata is distinct from the schema metadata (which is what this change would check)




----------------------------------------------------------------
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.

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



[GitHub] [arrow] kou commented on pull request #8389: ARROW-8296: [C++][Dataset] Add IpcFileWriteOptions

Posted by GitBox <gi...@apache.org>.
kou commented on pull request #8389:
URL: https://github.com/apache/arrow/pull/8389#issuecomment-705903570






----------------------------------------------------------------
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.

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



[GitHub] [arrow] bkietz commented on pull request #8389: ARROW-8296: [C++][Dataset] Add IpcFileWriteOptions

Posted by GitBox <gi...@apache.org>.
bkietz commented on pull request #8389:
URL: https://github.com/apache/arrow/pull/8389#issuecomment-706132376


   Thanks @kou!


----------------------------------------------------------------
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.

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



[GitHub] [arrow] nealrichardson commented on a change in pull request #8389: ARROW-8296: [C++][Dataset] Add IpcFileWriteOptions

Posted by GitBox <gi...@apache.org>.
nealrichardson commented on a change in pull request #8389:
URL: https://github.com/apache/arrow/pull/8389#discussion_r502656503



##########
File path: r/R/dataset-format.R
##########
@@ -139,6 +139,13 @@ FileWriteOptions <- R6Class("FileWriteOptions", inherit = ArrowObject,
         dataset___ParquetFileWriteOptions__update(self,
             ParquetWriterProperties$create(...),
             ParquetArrowWriterProperties$create(...))
+      } else if (self$type == "ipc") {
+        args <- list(...)
+        dataset___IpcFileWriteOptions__update(self,
+            get_ipc_use_legacy_format(args$use_legacy_format),
+            args$codec,

Review comment:
       Ok. Given #8256 in flight I don't think we want to mess with `shared_ptr()`, so do what you think is best.




----------------------------------------------------------------
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.

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



[GitHub] [arrow] bkietz commented on a change in pull request #8389: ARROW-8296: [C++][Dataset] Add IpcFileWriteOptions

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #8389:
URL: https://github.com/apache/arrow/pull/8389#discussion_r502557047



##########
File path: r/src/arrow_cpp11.h
##########
@@ -157,19 +157,42 @@ struct ns {
   static SEXP arrow;
 };
 
+// Specialize this struct to define a default value to be used when NULL is given.

Review comment:
       It's a convenience for passing optional parameters; instead of writing multiple overloads of a method with different permutations of optional parameters specified those parameters can be set to `NULL` in R. It's not necessary; should I revert it?




----------------------------------------------------------------
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.

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



[GitHub] [arrow] bkietz commented on pull request #8389: ARROW-8296: [C++][Dataset] Add IpcFileWriteOptions

Posted by GitBox <gi...@apache.org>.
bkietz commented on pull request #8389:
URL: https://github.com/apache/arrow/pull/8389#issuecomment-705901241






----------------------------------------------------------------
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.

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



[GitHub] [arrow] bkietz commented on a change in pull request #8389: ARROW-8296: [C++][Dataset] Add IpcFileWriteOptions

Posted by GitBox <gi...@apache.org>.
bkietz commented on a change in pull request #8389:
URL: https://github.com/apache/arrow/pull/8389#discussion_r502558047



##########
File path: r/src/arrow_metadata.h
##########
@@ -0,0 +1,38 @@
+// 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/util/key_value_metadata.h>
+
+inline std::shared_ptr<arrow::KeyValueMetadata> KeyValueMetadata__Make(

Review comment:
       1. I can replace other instances of KeyValueMetadata wrangling to use these helpers, or I could just leave things inlined as in other files under `arrow/r/`
   2. How about `KeyValueMetadata__from_strings`, `KeyValueMetadata__to_strings`?




----------------------------------------------------------------
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.

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