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/07/27 05:02:20 UTC

[GitHub] [arrow] vibhatha opened a new pull request, #13401: ARROW-16855: [C++] Adding Read Relation ToProto

vibhatha opened a new pull request, #13401:
URL: https://github.com/apache/arrow/pull/13401

   This is the initial PR to set the util functions and structure to include the `ToProto` functionality to relations.
   Here the objective is to create an ACERO relation by interpretting what is included in a Substrait-Relation. 
   In this PR the `read` relation ToProto is added. 


-- 
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] jvanstraten commented on pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
jvanstraten commented on PR #13401:
URL: https://github.com/apache/arrow/pull/13401#issuecomment-1160287804

   > Do you think it’s wise to add a CI to test Substrait related queries using this tool?
   
   IMO every roundtripped plan in every Substrait consumer and/or producer should also be passed through the validator. Otherwise, how would you know for sure that the Substrait plan you've successfully roundtripped through is actually sensible in any way? It does always require a complete plan, though, so you'd need some or other function for each type of thing (expression, relation, etc) that surrounds the thing with a dummy plan. Arrow could hook into it via the C interface (it's not a very pleasant interface because it's intended to be compatible with any language that can call into C, so you might want to wrap it with some C++ stuff; also it will need a Rust compiler to build) or it could just execute the CLI on a generated file (more clunky, but that can just be pulled from PyPI in binary form, so it's probably a bit easier on CI).
   
   I'm sure I'm biased though, since I'm the one who made the validator. It's also starting to considerably lag behind Substrait; it doesn't seem like anyone is sufficiently interested to review/collaborate, so I can't get any PRs through.
   
   Link, just in case: https://github.com/substrait-io/substrait-validator
   
   > One doubtful thing is to check in serialization is whether a projection or filter expression is added or not/ differentiation from default values. For instance filter expression defaults to a boolean literal of value true.
   
   Assuming you mean that in Acero the filter expression is mandatory and is just set to literal true if there is none, IMO you could just do the same thing on the Substrait side, at least for now. Likewise for the projection. Or you could just leave it for a later PR and error out when presented with nontrivial values. I don't know how hard any of these things are; I've never done anything with the Acero representation.


-- 
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] vibhatha commented on a diff in pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
vibhatha commented on code in PR #13401:
URL: https://github.com/apache/arrow/pull/13401#discussion_r956886628


##########
cpp/src/arrow/engine/substrait/relation_internal.cc:
##########
@@ -29,10 +29,20 @@
 #include "arrow/filesystem/localfs.h"
 #include "arrow/filesystem/path_util.h"
 #include "arrow/filesystem/util_internal.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/make_unique.h"
 
 namespace arrow {
+
+using internal::checked_cast;
+using internal::make_unique;
+
 namespace engine {
 
+using SubstraitConverter = std::function<Result<std::unique_ptr<substrait::Rel>>(
+    const std::shared_ptr<Schema>&, const compute::Declaration&, ExtensionSet*,
+    const ConversionOptions&)>;
+

Review Comment:
   This is no longer needed. The refactor removes the generic interface and replaces with specific types.



-- 
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] westonpace commented on a diff in pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
westonpace commented on code in PR #13401:
URL: https://github.com/apache/arrow/pull/13401#discussion_r958606704


##########
cpp/src/arrow/engine/substrait/serde_test.cc:
##########
@@ -42,9 +55,35 @@ using testing::UnorderedElementsAre;
 namespace arrow {
 
 using internal::checked_cast;
-
+using internal::hash_combine;
 namespace engine {
 
+Status WriteParquetData(const std::string& path,
+                        const std::shared_ptr<fs::FileSystem> file_system,
+                        const std::shared_ptr<Table> input) {
+  EXPECT_OK_AND_ASSIGN(auto buffer_writer, file_system->OpenOutputStream(path));
+  PARQUET_THROW_NOT_OK(parquet::arrow::WriteTable(*input, arrow::default_memory_pool(),
+                                                  buffer_writer, /*chunk_size*/ 1));
+  return buffer_writer->Close();
+}

Review Comment:
   In Substrait the IPC format is called `arrow` with `ArrowReadOptions`.



-- 
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] vibhatha commented on a diff in pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
vibhatha commented on code in PR #13401:
URL: https://github.com/apache/arrow/pull/13401#discussion_r907408216


##########
cpp/src/arrow/engine/substrait/relation_internal.cc:
##########
@@ -316,5 +323,97 @@ Result<compute::Declaration> FromProto(const substrait::Rel& rel,
       rel.DebugString());
 }
 
+namespace {
+// TODO: add other types
+enum ArrowRelationType : uint8_t {
+  SCAN,
+  FILTER,
+  PROJECT,
+  JOIN,
+  AGGREGATE,
+};
+
+const std::map<std::string, ArrowRelationType> enum_map{
+    {"scan", ArrowRelationType::SCAN},           {"filter", ArrowRelationType::FILTER},
+    {"project", ArrowRelationType::PROJECT},     {"join", ArrowRelationType::JOIN},
+    {"aggregate", ArrowRelationType::AGGREGATE},
+};
+
+struct ExtractRelation {
+  explicit ExtractRelation(substrait::Rel* rel, ExtensionSet* ext_set)
+      : rel_(rel), ext_set_(ext_set) {}
+
+  Status AddRelation(const compute::Declaration& declaration) {
+    const std::string& rel_name = declaration.factory_name;
+    switch (enum_map.find(rel_name)->second) {
+      case ArrowRelationType::SCAN:
+        return AddReadRelation(declaration);
+      case ArrowRelationType::FILTER:
+        return Status::NotImplemented("Filter operator not supported.");
+      case ArrowRelationType::PROJECT:
+        return Status::NotImplemented("Project operator not supported.");
+      case ArrowRelationType::JOIN:
+        return Status::NotImplemented("Join operator not supported.");
+      case ArrowRelationType::AGGREGATE:
+        return Status::NotImplemented("Aggregate operator not supported.");
+      default:
+        return Status::Invalid("Unsupported factory name :", rel_name);
+    }
+  }
+
+  Status AddReadRelation(const compute::Declaration& declaration) {
+    auto read_rel = internal::make_unique<substrait::ReadRel>();
+    const auto& scan_node_options =
+        internal::checked_cast<const dataset::ScanNodeOptions&>(*declaration.options);
+
+    const auto& fds = internal::checked_cast<const dataset::FileSystemDataset&>(
+        *scan_node_options.dataset);
+
+    // set schema
+    ARROW_ASSIGN_OR_RAISE(auto named_struct, ToProto(*fds.schema(), ext_set_));
+    read_rel->set_allocated_base_schema(named_struct.release());
+
+    // set local files
+    auto read_rel_lfs = internal::make_unique<substrait::ReadRel_LocalFiles>();
+    for (const auto& file : fds.files()) {
+      auto read_rel_lfs_ffs =
+          internal::make_unique<substrait::ReadRel_LocalFiles_FileOrFiles>();
+      read_rel_lfs_ffs->set_uri_path("file://" + file);
+
+      // set file format
+      auto format_type_name = fds.format()->type_name();
+      if (format_type_name == "parquet" || format_type_name == "arrow" ||
+          format_type_name == "feather") {
+        read_rel_lfs_ffs->set_format(
+            substrait::ReadRel::LocalFiles::FileOrFiles::FILE_FORMAT_PARQUET);
+      } else {
+        return Status::Invalid("Unsupported file type : ", format_type_name);
+      }
+      read_rel_lfs->mutable_items()->AddAllocated(read_rel_lfs_ffs.release());
+    }
+    *read_rel->mutable_local_files() = *read_rel_lfs.get();
+
+    rel_->set_allocated_read(read_rel.release());

Review Comment:
   Sure I will add one.



-- 
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] vibhatha commented on a diff in pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
vibhatha commented on code in PR #13401:
URL: https://github.com/apache/arrow/pull/13401#discussion_r956538091


##########
cpp/src/arrow/engine/substrait/relation_internal.cc:
##########
@@ -421,5 +431,171 @@ Result<DeclarationInfo> FromProto(const substrait::Rel& rel, const ExtensionSet&
       rel.DebugString());
 }
 
+Result<std::unique_ptr<substrait::Rel>> ToProto(
+    const compute::Declaration& declr, ExtensionSet* ext_set,
+    const ConversionOptions& conversion_options) {
+  auto rel = make_unique<substrait::Rel>();
+  RETURN_NOT_OK(SerializeAndCombineRelations(declr, ext_set, rel, conversion_options));
+  return std::move(rel);
+}
+
+Status SetRelation(const std::unique_ptr<substrait::Rel>& plan,
+                   const std::unique_ptr<substrait::Rel>& partial_plan,
+                   const std::string& factory_name) {
+  if (factory_name == "scan" && partial_plan->has_read()) {
+    plan->set_allocated_read(partial_plan->release_read());
+  } else if (factory_name == "filter" && partial_plan->has_filter()) {
+    plan->set_allocated_filter(partial_plan->release_filter());
+  } else {
+    return Status::NotImplemented("Substrait converter ", factory_name,
+                                  " not supported.");
+  }
+  return Status::OK();
+}
+
+Result<std::shared_ptr<Schema>> ExtractSchemaToBind(const compute::Declaration& declr) {
+  std::shared_ptr<Schema> bind_schema;
+  if (declr.factory_name == "scan") {
+    const auto& opts = checked_cast<const dataset::ScanNodeOptions&>(*(declr.options));
+    bind_schema = opts.dataset->schema();
+  } else if (declr.factory_name == "filter") {
+    auto input_declr = util::get<compute::Declaration>(declr.inputs[0]);
+    ARROW_ASSIGN_OR_RAISE(bind_schema, ExtractSchemaToBind(input_declr));
+  } else if (declr.factory_name == "sink") {
+    // Note that the sink has no output_schema
+    return bind_schema;
+  } else {
+    return Status::Invalid("Schema extraction failed, unsupported factory ",
+                           declr.factory_name);
+  }
+  return bind_schema;
+}
+
+Status SerializeAndCombineRelations(const compute::Declaration& declaration,
+                                    ExtensionSet* ext_set,
+                                    std::unique_ptr<substrait::Rel>& rel,
+                                    const ConversionOptions& conversion_options) {
+  std::vector<compute::Declaration::Input> inputs = declaration.inputs;
+  for (auto& input : inputs) {
+    auto input_decl = util::get<compute::Declaration>(input);
+    RETURN_NOT_OK(
+        SerializeAndCombineRelations(input_decl, ext_set, rel, conversion_options));
+  }
+  const auto& factory_name = declaration.factory_name;
+  ARROW_ASSIGN_OR_RAISE(auto schema, ExtractSchemaToBind(declaration));
+  // Note that the sink declaration factory doesn't exist for serialization as
+  // Substrait doesn't deal with a sink node definition
+  std::unique_ptr<substrait::Rel> factory_rel;
+  if (factory_name == "scan") {
+    ARROW_ASSIGN_OR_RAISE(factory_rel, ScanRelationConverter(schema, declaration, ext_set,
+                                                             conversion_options));
+  } else if (factory_name == "filter") {
+    ARROW_ASSIGN_OR_RAISE(
+        factory_rel,
+        FilterRelationConverter(schema, declaration, ext_set, conversion_options));
+  } else {
+    return Status::NotImplemented("Factory ", factory_name,
+                                  " not implemented for roundtripping.");
+  }
+
+  if (factory_rel != nullptr) {
+    RETURN_NOT_OK(SetRelation(rel, factory_rel, factory_name));
+  } else {
+    return Status::Invalid("Conversion on factory ", factory_name,
+                           " returned an invalid relation");
+  }

Review Comment:
   Thanks for catching this. 



-- 
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] vibhatha commented on a diff in pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
vibhatha commented on code in PR #13401:
URL: https://github.com/apache/arrow/pull/13401#discussion_r956912961


##########
cpp/src/arrow/engine/substrait/relation_internal.cc:
##########
@@ -421,5 +431,171 @@ Result<DeclarationInfo> FromProto(const substrait::Rel& rel, const ExtensionSet&
       rel.DebugString());
 }
 
+Result<std::unique_ptr<substrait::Rel>> ToProto(
+    const compute::Declaration& declr, ExtensionSet* ext_set,
+    const ConversionOptions& conversion_options) {
+  auto rel = make_unique<substrait::Rel>();
+  RETURN_NOT_OK(SerializeAndCombineRelations(declr, ext_set, rel, conversion_options));
+  return std::move(rel);
+}
+
+Status SetRelation(const std::unique_ptr<substrait::Rel>& plan,
+                   const std::unique_ptr<substrait::Rel>& partial_plan,
+                   const std::string& factory_name) {
+  if (factory_name == "scan" && partial_plan->has_read()) {
+    plan->set_allocated_read(partial_plan->release_read());
+  } else if (factory_name == "filter" && partial_plan->has_filter()) {
+    plan->set_allocated_filter(partial_plan->release_filter());
+  } else {
+    return Status::NotImplemented("Substrait converter ", factory_name,
+                                  " not supported.");
+  }
+  return Status::OK();
+}
+
+Result<std::shared_ptr<Schema>> ExtractSchemaToBind(const compute::Declaration& declr) {
+  std::shared_ptr<Schema> bind_schema;
+  if (declr.factory_name == "scan") {
+    const auto& opts = checked_cast<const dataset::ScanNodeOptions&>(*(declr.options));
+    bind_schema = opts.dataset->schema();
+  } else if (declr.factory_name == "filter") {
+    auto input_declr = util::get<compute::Declaration>(declr.inputs[0]);
+    ARROW_ASSIGN_OR_RAISE(bind_schema, ExtractSchemaToBind(input_declr));
+  } else if (declr.factory_name == "sink") {
+    // Note that the sink has no output_schema
+    return bind_schema;
+  } else {
+    return Status::Invalid("Schema extraction failed, unsupported factory ",
+                           declr.factory_name);
+  }
+  return bind_schema;
+}
+
+Status SerializeAndCombineRelations(const compute::Declaration& declaration,
+                                    ExtensionSet* ext_set,
+                                    std::unique_ptr<substrait::Rel>& rel,
+                                    const ConversionOptions& conversion_options) {
+  std::vector<compute::Declaration::Input> inputs = declaration.inputs;
+  for (auto& input : inputs) {
+    auto input_decl = util::get<compute::Declaration>(input);
+    RETURN_NOT_OK(
+        SerializeAndCombineRelations(input_decl, ext_set, rel, conversion_options));
+  }
+  const auto& factory_name = declaration.factory_name;
+  ARROW_ASSIGN_OR_RAISE(auto schema, ExtractSchemaToBind(declaration));
+  // Note that the sink declaration factory doesn't exist for serialization as
+  // Substrait doesn't deal with a sink node definition
+  std::unique_ptr<substrait::Rel> factory_rel;
+  if (factory_name == "scan") {
+    ARROW_ASSIGN_OR_RAISE(factory_rel, ScanRelationConverter(schema, declaration, ext_set,
+                                                             conversion_options));
+  } else if (factory_name == "filter") {
+    ARROW_ASSIGN_OR_RAISE(
+        factory_rel,
+        FilterRelationConverter(schema, declaration, ext_set, conversion_options));
+  } else {
+    return Status::NotImplemented("Factory ", factory_name,
+                                  " not implemented for roundtripping.");
+  }
+
+  if (factory_rel != nullptr) {
+    RETURN_NOT_OK(SetRelation(rel, factory_rel, factory_name));
+  } else {
+    return Status::Invalid("Conversion on factory ", factory_name,
+                           " returned an invalid relation");
+  }
+  return Status::OK();
+}
+
+Result<std::unique_ptr<substrait::Rel>> GetRelationFromDeclaration(
+    const compute::Declaration& declaration, ExtensionSet* ext_set,
+    const ConversionOptions& conversion_options) {
+  auto declr_input = declaration.inputs[0];
+  // Note that the input is expected in declaration.
+  // ExecNode inputs are not accepted
+  if (util::get_if<compute::ExecNode*>(&declr_input)) {
+    return Status::NotImplemented("Only support Plans written in Declaration format.");
+  }
+  return ToProto(util::get<compute::Declaration>(declr_input), ext_set,
+                 conversion_options);
+}
+
+Result<std::unique_ptr<substrait::Rel>> ScanRelationConverter(
+    const std::shared_ptr<Schema>& schema, const compute::Declaration& declaration,
+    ExtensionSet* ext_set, const ConversionOptions& conversion_options) {
+  auto rel = make_unique<substrait::Rel>();
+  auto read_rel = make_unique<substrait::ReadRel>();
+  const auto& scan_node_options =
+      checked_cast<const dataset::ScanNodeOptions&>(*declaration.options);
+  auto dataset =
+      dynamic_cast<dataset::FileSystemDataset*>(scan_node_options.dataset.get());
+  if (dataset == nullptr) {
+    return Status::Invalid("Can only convert file system datasets to a Substrait plan.");
+  }
+  // set schema
+  ARROW_ASSIGN_OR_RAISE(auto named_struct,
+                        ToProto(*dataset->schema(), ext_set, conversion_options));
+  read_rel->set_allocated_base_schema(named_struct.release());
+
+  // set local files
+  auto read_rel_lfs = make_unique<substrait::ReadRel_LocalFiles>();
+  for (const auto& file : dataset->files()) {
+    auto read_rel_lfs_ffs = make_unique<substrait::ReadRel_LocalFiles_FileOrFiles>();
+    read_rel_lfs_ffs->set_uri_path("file://" + file);
+    // set file format
+    // arrow and feather are temporarily handled via the Parquet format until
+    // upgraded to the latest Substrait version.

Review Comment:
   You're correct. 



-- 
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] vibhatha commented on a diff in pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
vibhatha commented on code in PR #13401:
URL: https://github.com/apache/arrow/pull/13401#discussion_r956538360


##########
cpp/src/arrow/engine/substrait/relation_internal.cc:
##########
@@ -29,10 +29,20 @@
 #include "arrow/filesystem/localfs.h"
 #include "arrow/filesystem/path_util.h"
 #include "arrow/filesystem/util_internal.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/make_unique.h"
 
 namespace arrow {
+
+using internal::checked_cast;
+using internal::make_unique;
+
 namespace engine {
 
+using SubstraitConverter = std::function<Result<std::unique_ptr<substrait::Rel>>(
+    const std::shared_ptr<Schema>&, const compute::Declaration&, ExtensionSet*,
+    const ConversionOptions&)>;
+

Review Comment:
   I added it because the parameter would be too long. Just for the readability and usage. 



-- 
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] vibhatha commented on pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
vibhatha commented on PR #13401:
URL: https://github.com/apache/arrow/pull/13401#issuecomment-1240916263

   @westonpace Thanks a lot for keeping up with the major changes and a few rounds of reviews. 👍 


-- 
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] westonpace commented on a diff in pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
westonpace commented on code in PR #13401:
URL: https://github.com/apache/arrow/pull/13401#discussion_r918433307


##########
cpp/src/arrow/engine/substrait/relation_internal.cc:
##########
@@ -318,5 +324,81 @@ Result<compute::Declaration> FromProto(const substrait::Rel& rel,
       rel.DebugString());
 }
 
+namespace {
+
+Result<std::unique_ptr<substrait::ReadRel>> MakeReadRelation(
+    const compute::Declaration& declaration, ExtensionSet* ext_set) {
+  auto read_rel = make_unique<substrait::ReadRel>();
+  const auto& scan_node_options =
+      checked_cast<const dataset::ScanNodeOptions&>(*declaration.options);
+
+  auto dataset =
+      dynamic_cast<dataset::FileSystemDataset*>(scan_node_options.dataset.get());
+  if (dataset == nullptr) {
+    return Status::Invalid("Can only convert file system datasets to a Substrait plan.");
+  }
+  // set schema
+  ARROW_ASSIGN_OR_RAISE(auto named_struct, ToProto(*dataset->schema(), ext_set));
+  read_rel->set_allocated_base_schema(named_struct.release());
+
+  // set local files
+  auto read_rel_lfs = make_unique<substrait::ReadRel_LocalFiles>();
+  for (const auto& file : dataset->files()) {
+    auto read_rel_lfs_ffs = make_unique<substrait::ReadRel_LocalFiles_FileOrFiles>();
+    read_rel_lfs_ffs->set_uri_path("file://" + file);
+
+    // set file format
+    // arrow and feather are temporarily handled via the Parquet format until
+    // upgraded to the latest Substrait version.
+    auto format_type_name = dataset->format()->type_name();
+    if (format_type_name == "parquet") {
+      auto parquet_fmt =
+          make_unique<substrait::ReadRel_LocalFiles_FileOrFiles_ParquetReadOptions>();
+      read_rel_lfs_ffs->set_allocated_parquet(parquet_fmt.release());
+    } else if (format_type_name == "arrow") {
+      auto arrow_fmt =
+          make_unique<substrait::ReadRel_LocalFiles_FileOrFiles_ArrowReadOptions>();
+      read_rel_lfs_ffs->set_allocated_arrow(arrow_fmt.release());
+    } else if (format_type_name == "orc") {
+      auto orc_fmt =
+          make_unique<substrait::ReadRel_LocalFiles_FileOrFiles_OrcReadOptions>();
+      read_rel_lfs_ffs->set_allocated_orc(orc_fmt.release());
+    } else {
+      return Status::Invalid("Unsupported file type : ", format_type_name);

Review Comment:
   ```suggestion
         return Status::NotImplemented("Unsupported file type: ", format_type_name);
   ```



##########
cpp/src/arrow/engine/substrait/serde_test.cc:
##########
@@ -1383,5 +1400,188 @@ TEST(Substrait, JoinPlanInvalidKeys) {
   }
 }
 
+TEST(Substrait, SerializeRelation) {
+#ifdef _WIN32
+  GTEST_SKIP() << "ARROW-16392: Substrait File URI not supported for Windows";
+#else
+  ExtensionSet ext_set;
+  auto dummy_schema = schema({field("foo", binary())});
+  // creating a dummy dataset using a dummy table
+  auto format = std::make_shared<arrow::dataset::ParquetFileFormat>();
+  auto filesystem = std::make_shared<fs::LocalFileSystem>();
+
+  std::vector<fs::FileInfo> files;
+  const std::vector<std::string> f_paths = {"/tmp/data1.parquet", "/tmp/data2.parquet"};
+
+  for (const auto& f_path : f_paths) {
+    ASSERT_OK_AND_ASSIGN(auto f_file, filesystem->GetFileInfo(f_path));
+    files.push_back(std::move(f_file));
+  }
+
+  ASSERT_OK_AND_ASSIGN(auto ds_factory, dataset::FileSystemDatasetFactory::Make(
+                                            std::move(filesystem), std::move(files),
+                                            std::move(format), {}));
+  ASSERT_OK_AND_ASSIGN(auto dataset, ds_factory->Finish(dummy_schema));
+
+  auto options = std::make_shared<dataset::ScanOptions>();
+  options->projection = compute::project({}, {});
+  auto scan_node_options = dataset::ScanNodeOptions{dataset, options};
+
+  auto scan_declaration = compute::Declaration({"scan", scan_node_options});
+
+  ASSERT_OK_AND_ASSIGN(auto serialized_rel,
+                       SerializeRelation(scan_declaration, &ext_set));
+  ASSERT_OK_AND_ASSIGN(auto deserialized_decl,
+                       DeserializeRelation(*serialized_rel, ext_set));
+
+  auto dataset_comparator = [](std::shared_ptr<dataset::Dataset> ds_lhs,
+                               std::shared_ptr<dataset::Dataset> ds_rhs) -> bool {
+    const auto& fsd_lhs = checked_cast<const dataset::FileSystemDataset&>(*ds_lhs);
+    const auto& fsd_rhs = checked_cast<const dataset::FileSystemDataset&>(*ds_lhs);
+    const auto& files_lhs = fsd_lhs.files();
+    const auto& files_rhs = fsd_rhs.files();
+
+    if (files_lhs.size() != files_rhs.size()) {
+      return false;
+    }
+    uint64_t fidx = 0;
+    for (const auto& l_file : files_lhs) {
+      if (l_file != files_rhs[fidx]) {
+        return false;
+      }
+      fidx++;
+    }
+    bool cmp_file_format = fsd_lhs.format()->Equals(*fsd_lhs.format());
+    bool cmp_file_system = fsd_lhs.filesystem()->Equals(fsd_rhs.filesystem());
+    return cmp_file_format && cmp_file_system;
+  };
+
+  auto scan_option_comparator = [dataset_comparator](
+                                    const dataset::ScanNodeOptions& lhs,
+                                    const dataset::ScanNodeOptions& rhs) -> bool {
+    bool cmp_rso = lhs.require_sequenced_output == rhs.require_sequenced_output;
+    bool cmp_ds = dataset_comparator(lhs.dataset, rhs.dataset);
+    return cmp_rso && cmp_ds;
+  };
+
+  EXPECT_EQ(deserialized_decl.factory_name, scan_declaration.factory_name);
+  const auto& lhs =
+      checked_cast<const dataset::ScanNodeOptions&>(*deserialized_decl.options);
+  const auto& rhs =
+      checked_cast<const dataset::ScanNodeOptions&>(*scan_declaration.options);
+  ASSERT_TRUE(scan_option_comparator(lhs, rhs));
+#endif
+}
+
+TEST(Substrait, SerializeRelationEndToEnd) {
+#ifdef _WIN32
+  GTEST_SKIP() << "ARROW-16392: Substrait File URI not supported for Windows";
+#else
+  ExtensionSet ext_set;
+  compute::ExecContext exec_context;
+
+  auto dummy_schema = schema({field("a", int32()), field("b", int32())});
+  auto table = TableFromJSON(dummy_schema, {R"([
+    [1, 1],
+    [3, 4]
+                        ])",
+                                            R"([
+    [0, 2],
+    [1, 3],
+    [4, 1],
+    [3, 1],
+    [1, 2]
+                        ])",
+                                            R"([
+    [2, 2],
+    [5, 3],
+    [1, 3]
+                        ])"});
+  // Note:: adding `/` to support the currently supported file read format from file
+  // system for Substrait
+  const std::string path = "/testing.parquet";
+
+  EXPECT_OK_AND_ASSIGN(auto filesystem,
+                       fs::internal::MockFileSystem::Make(fs::kNoTime, {}));
+
+  EXPECT_EQ(WriteParquetData(path, filesystem, table), true);
+
+  auto format = std::make_shared<arrow::dataset::ParquetFileFormat>();
+
+  std::vector<fs::FileInfo> files;
+  ASSERT_OK_AND_ASSIGN(auto f_file, filesystem->GetFileInfo(path));
+  files.push_back(std::move(f_file));
+
+  ASSERT_OK_AND_ASSIGN(auto ds_factory, dataset::FileSystemDatasetFactory::Make(
+                                            std::move(filesystem), std::move(files),
+                                            std::move(format), {}));
+
+  ASSERT_OK_AND_ASSIGN(auto dataset, ds_factory->Finish(dummy_schema));
+
+  auto options = std::make_shared<dataset::ScanOptions>();
+  options->projection = compute::project({}, {});
+
+  auto scan_node_options = dataset::ScanNodeOptions{dataset, options};
+
+  arrow::AsyncGenerator<util::optional<compute::ExecBatch> > sink_gen;
+
+  auto sink_node_options = compute::SinkNodeOptions{&sink_gen};
+
+  auto scan_declaration = compute::Declaration({"scan", scan_node_options});
+  auto sink_declaration = compute::Declaration({"sink", sink_node_options});
+
+  auto declarations =
+      compute::Declaration::Sequence({scan_declaration, sink_declaration});
+
+  ASSERT_OK_AND_ASSIGN(auto plan, compute::ExecPlan::Make(&exec_context));
+  ASSERT_OK_AND_ASSIGN(auto decl, declarations.AddToPlan(plan.get()));
+
+  ASSERT_OK(decl->Validate());
+
+  std::shared_ptr<arrow::RecordBatchReader> sink_reader = compute::MakeGeneratorReader(
+      dummy_schema, std::move(sink_gen), exec_context.memory_pool());
+
+  ASSERT_OK(plan->Validate());
+  ASSERT_OK(plan->StartProducing());
+
+  std::shared_ptr<arrow::Table> response_table;
+
+  ASSERT_OK_AND_ASSIGN(response_table,
+                       arrow::Table::FromRecordBatchReader(sink_reader.get()));
+
+  ASSERT_OK_AND_ASSIGN(auto serialized_rel,
+                       SerializeRelation(scan_declaration, &ext_set));
+  ASSERT_OK_AND_ASSIGN(auto deserialized_decl,
+                       DeserializeRelation(*serialized_rel, ext_set));
+
+  arrow::AsyncGenerator<util::optional<compute::ExecBatch> > des_sink_gen;

Review Comment:
   What is the `des` prefix for?  Can you use a whole word?



##########
cpp/src/arrow/engine/substrait/serde_test.cc:
##########
@@ -45,6 +52,15 @@ using internal::checked_cast;
 
 namespace engine {
 
+bool WriteParquetData(const std::string& path,

Review Comment:
   Can you just return the `Status` here instead of a `bool`?



##########
cpp/src/arrow/engine/substrait/relation_internal.cc:
##########
@@ -318,5 +324,81 @@ Result<compute::Declaration> FromProto(const substrait::Rel& rel,
       rel.DebugString());
 }
 
+namespace {
+
+Result<std::unique_ptr<substrait::ReadRel>> MakeReadRelation(
+    const compute::Declaration& declaration, ExtensionSet* ext_set) {
+  auto read_rel = make_unique<substrait::ReadRel>();
+  const auto& scan_node_options =
+      checked_cast<const dataset::ScanNodeOptions&>(*declaration.options);
+
+  auto dataset =
+      dynamic_cast<dataset::FileSystemDataset*>(scan_node_options.dataset.get());
+  if (dataset == nullptr) {
+    return Status::Invalid("Can only convert file system datasets to a Substrait plan.");
+  }
+  // set schema
+  ARROW_ASSIGN_OR_RAISE(auto named_struct, ToProto(*dataset->schema(), ext_set));
+  read_rel->set_allocated_base_schema(named_struct.release());
+
+  // set local files
+  auto read_rel_lfs = make_unique<substrait::ReadRel_LocalFiles>();
+  for (const auto& file : dataset->files()) {
+    auto read_rel_lfs_ffs = make_unique<substrait::ReadRel_LocalFiles_FileOrFiles>();
+    read_rel_lfs_ffs->set_uri_path("file://" + file);
+
+    // set file format
+    // arrow and feather are temporarily handled via the Parquet format until
+    // upgraded to the latest Substrait version.
+    auto format_type_name = dataset->format()->type_name();
+    if (format_type_name == "parquet") {
+      auto parquet_fmt =
+          make_unique<substrait::ReadRel_LocalFiles_FileOrFiles_ParquetReadOptions>();
+      read_rel_lfs_ffs->set_allocated_parquet(parquet_fmt.release());
+    } else if (format_type_name == "arrow") {
+      auto arrow_fmt =
+          make_unique<substrait::ReadRel_LocalFiles_FileOrFiles_ArrowReadOptions>();
+      read_rel_lfs_ffs->set_allocated_arrow(arrow_fmt.release());
+    } else if (format_type_name == "orc") {
+      auto orc_fmt =
+          make_unique<substrait::ReadRel_LocalFiles_FileOrFiles_OrcReadOptions>();
+      read_rel_lfs_ffs->set_allocated_orc(orc_fmt.release());
+    } else {
+      return Status::Invalid("Unsupported file type : ", format_type_name);
+    }
+    read_rel_lfs->mutable_items()->AddAllocated(read_rel_lfs_ffs.release());
+  }
+  *read_rel->mutable_local_files() = *read_rel_lfs.get();

Review Comment:
   ```suggestion
     read_rel->set_allocated_local_files(read_rel_lfs.release());
   ```



##########
cpp/src/arrow/engine/substrait/serde_test.cc:
##########
@@ -1383,5 +1400,188 @@ TEST(Substrait, JoinPlanInvalidKeys) {
   }
 }
 
+TEST(Substrait, SerializeRelation) {
+#ifdef _WIN32
+  GTEST_SKIP() << "ARROW-16392: Substrait File URI not supported for Windows";
+#else
+  ExtensionSet ext_set;
+  auto dummy_schema = schema({field("foo", binary())});
+  // creating a dummy dataset using a dummy table
+  auto format = std::make_shared<arrow::dataset::ParquetFileFormat>();
+  auto filesystem = std::make_shared<fs::LocalFileSystem>();
+
+  std::vector<fs::FileInfo> files;
+  const std::vector<std::string> f_paths = {"/tmp/data1.parquet", "/tmp/data2.parquet"};

Review Comment:
   `/tmp/` will not be portable once we support Windows URIs.  Can you use `arrow::internal::TemporaryDir` from `arrow/util/io_util.h`?



##########
cpp/src/arrow/engine/substrait/serde_test.cc:
##########
@@ -1383,5 +1400,188 @@ TEST(Substrait, JoinPlanInvalidKeys) {
   }
 }
 
+TEST(Substrait, SerializeRelation) {
+#ifdef _WIN32
+  GTEST_SKIP() << "ARROW-16392: Substrait File URI not supported for Windows";
+#else
+  ExtensionSet ext_set;
+  auto dummy_schema = schema({field("foo", binary())});
+  // creating a dummy dataset using a dummy table
+  auto format = std::make_shared<arrow::dataset::ParquetFileFormat>();
+  auto filesystem = std::make_shared<fs::LocalFileSystem>();
+
+  std::vector<fs::FileInfo> files;
+  const std::vector<std::string> f_paths = {"/tmp/data1.parquet", "/tmp/data2.parquet"};
+
+  for (const auto& f_path : f_paths) {
+    ASSERT_OK_AND_ASSIGN(auto f_file, filesystem->GetFileInfo(f_path));
+    files.push_back(std::move(f_file));
+  }
+
+  ASSERT_OK_AND_ASSIGN(auto ds_factory, dataset::FileSystemDatasetFactory::Make(
+                                            std::move(filesystem), std::move(files),
+                                            std::move(format), {}));
+  ASSERT_OK_AND_ASSIGN(auto dataset, ds_factory->Finish(dummy_schema));
+
+  auto options = std::make_shared<dataset::ScanOptions>();
+  options->projection = compute::project({}, {});
+  auto scan_node_options = dataset::ScanNodeOptions{dataset, options};
+
+  auto scan_declaration = compute::Declaration({"scan", scan_node_options});
+
+  ASSERT_OK_AND_ASSIGN(auto serialized_rel,
+                       SerializeRelation(scan_declaration, &ext_set));
+  ASSERT_OK_AND_ASSIGN(auto deserialized_decl,
+                       DeserializeRelation(*serialized_rel, ext_set));
+
+  auto dataset_comparator = [](std::shared_ptr<dataset::Dataset> ds_lhs,
+                               std::shared_ptr<dataset::Dataset> ds_rhs) -> bool {
+    const auto& fsd_lhs = checked_cast<const dataset::FileSystemDataset&>(*ds_lhs);
+    const auto& fsd_rhs = checked_cast<const dataset::FileSystemDataset&>(*ds_lhs);
+    const auto& files_lhs = fsd_lhs.files();
+    const auto& files_rhs = fsd_rhs.files();
+
+    if (files_lhs.size() != files_rhs.size()) {
+      return false;
+    }
+    uint64_t fidx = 0;
+    for (const auto& l_file : files_lhs) {
+      if (l_file != files_rhs[fidx]) {
+        return false;
+      }
+      fidx++;
+    }
+    bool cmp_file_format = fsd_lhs.format()->Equals(*fsd_lhs.format());
+    bool cmp_file_system = fsd_lhs.filesystem()->Equals(fsd_rhs.filesystem());
+    return cmp_file_format && cmp_file_system;
+  };
+
+  auto scan_option_comparator = [dataset_comparator](
+                                    const dataset::ScanNodeOptions& lhs,
+                                    const dataset::ScanNodeOptions& rhs) -> bool {
+    bool cmp_rso = lhs.require_sequenced_output == rhs.require_sequenced_output;
+    bool cmp_ds = dataset_comparator(lhs.dataset, rhs.dataset);
+    return cmp_rso && cmp_ds;
+  };
+
+  EXPECT_EQ(deserialized_decl.factory_name, scan_declaration.factory_name);
+  const auto& lhs =
+      checked_cast<const dataset::ScanNodeOptions&>(*deserialized_decl.options);
+  const auto& rhs =
+      checked_cast<const dataset::ScanNodeOptions&>(*scan_declaration.options);
+  ASSERT_TRUE(scan_option_comparator(lhs, rhs));
+#endif
+}
+
+TEST(Substrait, SerializeRelationEndToEnd) {
+#ifdef _WIN32
+  GTEST_SKIP() << "ARROW-16392: Substrait File URI not supported for Windows";
+#else
+  ExtensionSet ext_set;
+  compute::ExecContext exec_context;
+
+  auto dummy_schema = schema({field("a", int32()), field("b", int32())});
+  auto table = TableFromJSON(dummy_schema, {R"([
+    [1, 1],
+    [3, 4]
+                        ])",
+                                            R"([
+    [0, 2],
+    [1, 3],
+    [4, 1],
+    [3, 1],
+    [1, 2]
+                        ])",
+                                            R"([
+    [2, 2],
+    [5, 3],
+    [1, 3]
+                        ])"});

Review Comment:
   The formatting here seems off



##########
cpp/src/arrow/engine/substrait/serde_test.cc:
##########
@@ -45,6 +52,15 @@ using internal::checked_cast;
 
 namespace engine {
 
+bool WriteParquetData(const std::string& path,
+                      const std::shared_ptr<fs::FileSystem> file_system,
+                      const std::shared_ptr<Table> input, const int64_t chunk_size = 3) {

Review Comment:
   You only call this in one place and you never supply a custom chunk size.  Can you remove this parameter?  Or are you planning a test case that will manipulate it?  It doesn't seem relevant to the issue under test.



##########
cpp/src/arrow/engine/substrait/serde_test.cc:
##########
@@ -1383,5 +1400,188 @@ TEST(Substrait, JoinPlanInvalidKeys) {
   }
 }
 
+TEST(Substrait, SerializeRelation) {
+#ifdef _WIN32
+  GTEST_SKIP() << "ARROW-16392: Substrait File URI not supported for Windows";
+#else
+  ExtensionSet ext_set;
+  auto dummy_schema = schema({field("foo", binary())});
+  // creating a dummy dataset using a dummy table
+  auto format = std::make_shared<arrow::dataset::ParquetFileFormat>();
+  auto filesystem = std::make_shared<fs::LocalFileSystem>();
+
+  std::vector<fs::FileInfo> files;
+  const std::vector<std::string> f_paths = {"/tmp/data1.parquet", "/tmp/data2.parquet"};
+
+  for (const auto& f_path : f_paths) {
+    ASSERT_OK_AND_ASSIGN(auto f_file, filesystem->GetFileInfo(f_path));
+    files.push_back(std::move(f_file));
+  }
+
+  ASSERT_OK_AND_ASSIGN(auto ds_factory, dataset::FileSystemDatasetFactory::Make(
+                                            std::move(filesystem), std::move(files),
+                                            std::move(format), {}));
+  ASSERT_OK_AND_ASSIGN(auto dataset, ds_factory->Finish(dummy_schema));
+
+  auto options = std::make_shared<dataset::ScanOptions>();
+  options->projection = compute::project({}, {});
+  auto scan_node_options = dataset::ScanNodeOptions{dataset, options};
+
+  auto scan_declaration = compute::Declaration({"scan", scan_node_options});
+
+  ASSERT_OK_AND_ASSIGN(auto serialized_rel,
+                       SerializeRelation(scan_declaration, &ext_set));
+  ASSERT_OK_AND_ASSIGN(auto deserialized_decl,
+                       DeserializeRelation(*serialized_rel, ext_set));
+
+  auto dataset_comparator = [](std::shared_ptr<dataset::Dataset> ds_lhs,
+                               std::shared_ptr<dataset::Dataset> ds_rhs) -> bool {
+    const auto& fsd_lhs = checked_cast<const dataset::FileSystemDataset&>(*ds_lhs);
+    const auto& fsd_rhs = checked_cast<const dataset::FileSystemDataset&>(*ds_lhs);
+    const auto& files_lhs = fsd_lhs.files();
+    const auto& files_rhs = fsd_rhs.files();
+
+    if (files_lhs.size() != files_rhs.size()) {
+      return false;
+    }
+    uint64_t fidx = 0;
+    for (const auto& l_file : files_lhs) {
+      if (l_file != files_rhs[fidx]) {
+        return false;
+      }
+      fidx++;
+    }
+    bool cmp_file_format = fsd_lhs.format()->Equals(*fsd_lhs.format());
+    bool cmp_file_system = fsd_lhs.filesystem()->Equals(fsd_rhs.filesystem());
+    return cmp_file_format && cmp_file_system;
+  };
+
+  auto scan_option_comparator = [dataset_comparator](
+                                    const dataset::ScanNodeOptions& lhs,
+                                    const dataset::ScanNodeOptions& rhs) -> bool {
+    bool cmp_rso = lhs.require_sequenced_output == rhs.require_sequenced_output;
+    bool cmp_ds = dataset_comparator(lhs.dataset, rhs.dataset);
+    return cmp_rso && cmp_ds;
+  };
+
+  EXPECT_EQ(deserialized_decl.factory_name, scan_declaration.factory_name);
+  const auto& lhs =
+      checked_cast<const dataset::ScanNodeOptions&>(*deserialized_decl.options);
+  const auto& rhs =
+      checked_cast<const dataset::ScanNodeOptions&>(*scan_declaration.options);
+  ASSERT_TRUE(scan_option_comparator(lhs, rhs));
+#endif
+}
+
+TEST(Substrait, SerializeRelationEndToEnd) {
+#ifdef _WIN32
+  GTEST_SKIP() << "ARROW-16392: Substrait File URI not supported for Windows";
+#else
+  ExtensionSet ext_set;
+  compute::ExecContext exec_context;
+
+  auto dummy_schema = schema({field("a", int32()), field("b", int32())});
+  auto table = TableFromJSON(dummy_schema, {R"([
+    [1, 1],
+    [3, 4]
+                        ])",
+                                            R"([
+    [0, 2],
+    [1, 3],
+    [4, 1],
+    [3, 1],
+    [1, 2]
+                        ])",
+                                            R"([
+    [2, 2],
+    [5, 3],
+    [1, 3]
+                        ])"});
+  // Note:: adding `/` to support the currently supported file read format from file
+  // system for Substrait

Review Comment:
   I'm not sure I understand this note.



-- 
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] vibhatha commented on a diff in pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
vibhatha commented on code in PR #13401:
URL: https://github.com/apache/arrow/pull/13401#discussion_r918451534


##########
cpp/src/arrow/engine/substrait/relation_internal.cc:
##########
@@ -318,5 +324,81 @@ Result<compute::Declaration> FromProto(const substrait::Rel& rel,
       rel.DebugString());
 }
 
+namespace {
+
+Result<std::unique_ptr<substrait::ReadRel>> MakeReadRelation(
+    const compute::Declaration& declaration, ExtensionSet* ext_set) {
+  auto read_rel = make_unique<substrait::ReadRel>();
+  const auto& scan_node_options =
+      checked_cast<const dataset::ScanNodeOptions&>(*declaration.options);
+
+  auto dataset =
+      dynamic_cast<dataset::FileSystemDataset*>(scan_node_options.dataset.get());
+  if (dataset == nullptr) {
+    return Status::Invalid("Can only convert file system datasets to a Substrait plan.");
+  }
+  // set schema
+  ARROW_ASSIGN_OR_RAISE(auto named_struct, ToProto(*dataset->schema(), ext_set));
+  read_rel->set_allocated_base_schema(named_struct.release());
+
+  // set local files
+  auto read_rel_lfs = make_unique<substrait::ReadRel_LocalFiles>();
+  for (const auto& file : dataset->files()) {
+    auto read_rel_lfs_ffs = make_unique<substrait::ReadRel_LocalFiles_FileOrFiles>();
+    read_rel_lfs_ffs->set_uri_path("file://" + file);
+
+    // set file format
+    // arrow and feather are temporarily handled via the Parquet format until
+    // upgraded to the latest Substrait version.
+    auto format_type_name = dataset->format()->type_name();
+    if (format_type_name == "parquet") {
+      auto parquet_fmt =
+          make_unique<substrait::ReadRel_LocalFiles_FileOrFiles_ParquetReadOptions>();
+      read_rel_lfs_ffs->set_allocated_parquet(parquet_fmt.release());
+    } else if (format_type_name == "arrow") {
+      auto arrow_fmt =
+          make_unique<substrait::ReadRel_LocalFiles_FileOrFiles_ArrowReadOptions>();
+      read_rel_lfs_ffs->set_allocated_arrow(arrow_fmt.release());
+    } else if (format_type_name == "orc") {
+      auto orc_fmt =
+          make_unique<substrait::ReadRel_LocalFiles_FileOrFiles_OrcReadOptions>();
+      read_rel_lfs_ffs->set_allocated_orc(orc_fmt.release());
+    } else {
+      return Status::Invalid("Unsupported file type : ", format_type_name);
+    }
+    read_rel_lfs->mutable_items()->AddAllocated(read_rel_lfs_ffs.release());
+  }
+  *read_rel->mutable_local_files() = *read_rel_lfs.get();

Review Comment:
   Good catch.



-- 
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] vibhatha commented on a diff in pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
vibhatha commented on code in PR #13401:
URL: https://github.com/apache/arrow/pull/13401#discussion_r901274897


##########
cpp/src/arrow/engine/substrait/serde_test.cc:
##########
@@ -1173,5 +1175,168 @@ TEST(Substrait, JoinPlanInvalidKeys) {
           &ext_set));
 }
 
+TEST(Substrait, SerializeRelation) {
+#ifdef _WIN32
+  GTEST_SKIP() << "ARROW-16392: Substrait File URI not supported for Windows";
+#else
+  ExtensionSet ext_set;
+  auto dummy_schema = schema({field("f1", int32()), field("f2", int32())});
+  // creating a dummy dataset using a dummy table
+  auto format = std::make_shared<arrow::dataset::ParquetFileFormat>();
+  auto filesystem = std::make_shared<fs::LocalFileSystem>();
+
+  std::vector<fs::FileInfo> files;
+  const std::vector<std::string> f_paths = {"/tmp/data1.parquet", "/tmp/data2.parquet"};
+
+  for (const auto& f_path : f_paths) {
+    ASSERT_OK_AND_ASSIGN(auto f_file, filesystem->GetFileInfo(f_path));
+    files.push_back(std::move(f_file));
+  }
+
+  ASSERT_OK_AND_ASSIGN(auto ds_factory, dataset::FileSystemDatasetFactory::Make(
+                                            std::move(filesystem), std::move(files),
+                                            std::move(format), {}));
+  ASSERT_OK_AND_ASSIGN(auto dataset, ds_factory->Finish(dummy_schema));
+
+  auto options = std::make_shared<dataset::ScanOptions>();
+  options->projection = compute::project({}, {});
+  auto scan_node_options = dataset::ScanNodeOptions{dataset, options};
+
+  auto scan_declaration = compute::Declaration({"scan", scan_node_options});
+
+  ASSERT_OK_AND_ASSIGN(auto serialized_rel,
+                       SerializeRelation(scan_declaration, &ext_set));
+  ASSERT_OK_AND_ASSIGN(auto deserialized_decl,
+                       DeserializeRelation(*serialized_rel, ext_set));
+
+  auto dataset_comparator = [](std::shared_ptr<dataset::Dataset> ds_lhs,

Review Comment:
   cc @westonpace 
   Would it be better if we add a `Equals` method to `FileSystemDataset`?



-- 
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] vibhatha commented on a diff in pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
vibhatha commented on code in PR #13401:
URL: https://github.com/apache/arrow/pull/13401#discussion_r907412170


##########
cpp/src/arrow/engine/substrait/serde_test.cc:
##########
@@ -1173,5 +1175,173 @@ TEST(Substrait, JoinPlanInvalidKeys) {
           &ext_set));
 }
 
+TEST(Substrait, SerializeRelation) {
+#ifdef _WIN32
+  GTEST_SKIP() << "ARROW-16392: Substrait File URI not supported for Windows";
+#else
+  ExtensionSet ext_set;
+  auto dummy_schema = schema({field("foo", binary())});
+  // creating a dummy dataset using a dummy table
+  auto format = std::make_shared<arrow::dataset::ParquetFileFormat>();
+  auto filesystem = std::make_shared<fs::LocalFileSystem>();
+
+  ASSERT_OK_AND_ASSIGN(std::string dir_string,
+                       arrow::internal::GetEnvVar("PARQUET_TEST_DATA"));
+  auto file_name =
+      arrow::internal::PlatformFilename::FromString(dir_string)->Join("binary.parquet");
+
+  std::vector<fs::FileInfo> files;
+  const std::vector<std::string> f_paths = {file_name->ToString()};
+
+  for (const auto& f_path : f_paths) {
+    ASSERT_OK_AND_ASSIGN(auto f_file, filesystem->GetFileInfo(f_path));
+    files.push_back(std::move(f_file));
+  }
+
+  ASSERT_OK_AND_ASSIGN(auto ds_factory, dataset::FileSystemDatasetFactory::Make(
+                                            std::move(filesystem), std::move(files),
+                                            std::move(format), {}));
+  ASSERT_OK_AND_ASSIGN(auto dataset, ds_factory->Finish(dummy_schema));
+
+  auto options = std::make_shared<dataset::ScanOptions>();
+  options->projection = compute::project({}, {});
+  auto scan_node_options = dataset::ScanNodeOptions{dataset, options};
+
+  auto scan_declaration = compute::Declaration({"scan", scan_node_options});
+
+  ASSERT_OK_AND_ASSIGN(auto serialized_rel,
+                       SerializeRelation(scan_declaration, &ext_set));
+  ASSERT_OK_AND_ASSIGN(auto deserialized_decl,
+                       DeserializeRelation(*serialized_rel, ext_set));
+
+  auto dataset_comparator = [](std::shared_ptr<dataset::Dataset> ds_lhs,
+                               std::shared_ptr<dataset::Dataset> ds_rhs) -> bool {
+    const auto& fds_lhs = checked_cast<const dataset::FileSystemDataset&>(*ds_lhs);
+    const auto& fds_rhs = checked_cast<const dataset::FileSystemDataset&>(*ds_lhs);
+    const auto& files_lhs = fds_lhs.files();
+    const auto& files_rhs = fds_rhs.files();
+
+    bool cmp_fsize = files_lhs.size() == files_rhs.size();
+    uint64_t fidx = 0;
+    for (const auto& l_file : files_lhs) {
+      if (l_file != files_rhs[fidx]) {
+        return false;
+      }
+      fidx++;
+    }
+    bool cmp_file_format = fds_lhs.format()->Equals(*fds_lhs.format());
+    bool cmp_file_system = fds_lhs.filesystem()->Equals(fds_rhs.filesystem());
+    return cmp_fsize && cmp_file_format && cmp_file_system;
+  };
+
+  auto scan_option_comparator = [dataset_comparator](
+                                    const dataset::ScanNodeOptions& lhs,
+                                    const dataset::ScanNodeOptions& rhs) -> bool {
+    bool cmp_rso = lhs.require_sequenced_output == rhs.require_sequenced_output;
+    bool cmp_ds = dataset_comparator(lhs.dataset, rhs.dataset);
+    return cmp_rso && cmp_ds;
+  };
+
+  EXPECT_EQ(deserialized_decl.factory_name, scan_declaration.factory_name);
+  const auto& lhs =
+      checked_cast<const dataset::ScanNodeOptions&>(*deserialized_decl.options);
+  const auto& rhs =
+      checked_cast<const dataset::ScanNodeOptions&>(*scan_declaration.options);
+  ASSERT_TRUE(scan_option_comparator(lhs, rhs));
+#endif
+}
+
+TEST(Substrait, SerializeRelationEndToEnd) {

Review Comment:
   Yes indeed, I was thinking the same about these comparators. I cc`ed you somewhere else regarding this. I can create a JIRA to add the equals methods separately and I can work on them. 
   
   And if we used it again definitely we should push this for a util. I will add a note there. 



-- 
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] vibhatha commented on pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
vibhatha commented on PR #13401:
URL: https://github.com/apache/arrow/pull/13401#issuecomment-1160488517

   > > Do you think it’s wise to add a CI to test Substrait related queries using this tool?
   > 
   > IMO every roundtripped plan in every Substrait consumer and/or producer should also be passed through the validator. Otherwise, how would you know for sure that the Substrait plan you've successfully roundtripped through is actually sensible in any way? It does always require a complete plan, though, so you'd need some or other function for each type of thing (expression, relation, etc) that surrounds the thing with a dummy plan. Arrow could hook into it via the C interface (it's not a very pleasant interface because it's intended to be compatible with any language that can call into C, so you might want to wrap it with some C++ stuff; also it will need a Rust compiler to build) or it could just execute the CLI on a generated file (more clunky, but that can just be pulled from PyPI in binary form, so it's probably a bit easier on CI).
   > 
   > I'm sure I'm biased though, since I'm the one who made the validator. It's also starting to considerably lag behind Substrait; it doesn't seem like anyone is sufficiently interested to review/collaborate, so I can't get any PRs through.
   > 
   > Link, just in case: https://github.com/substrait-io/substrait-validator
   
   Intersting thoughts. I will take a look at the tool. It would be better if we can use it to validate things. But I am not sure if it needs to be inside the Arrow source or should it be a plugin for Apache Arrow. cc @westonpace 
   
   > > One doubtful thing is to check in serialization is whether a projection or filter expression is added or not/ differentiation from default values. For instance filter expression defaults to a boolean literal of value true.
   > 
   > Assuming you mean that in Acero the filter expression is mandatory and is just set to literal true if there is none, IMO you could just do the same thing on the Substrait side, at least for now. Likewise for the projection. Or you could just leave it for a later PR and error out when presented with nontrivial values. I don't know how hard any of these things are; I've never done anything with the Acero representation.
   
   Here it is rather, the differentiation between a user passed value vs the default. We could assume the default and do the comparison to see if an explicit value is passed. There is no API calls in Expression to check if it `has_filter` or `has_projection`. May be that kind of a function could be useful. 
   


-- 
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] lidavidm commented on a diff in pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
lidavidm commented on code in PR #13401:
URL: https://github.com/apache/arrow/pull/13401#discussion_r901699154


##########
cpp/src/arrow/engine/substrait/serde_test.cc:
##########
@@ -1173,5 +1175,168 @@ TEST(Substrait, JoinPlanInvalidKeys) {
           &ext_set));
 }
 
+TEST(Substrait, SerializeRelation) {
+#ifdef _WIN32
+  GTEST_SKIP() << "ARROW-16392: Substrait File URI not supported for Windows";
+#else
+  ExtensionSet ext_set;
+  auto dummy_schema = schema({field("f1", int32()), field("f2", int32())});
+  // creating a dummy dataset using a dummy table
+  auto format = std::make_shared<arrow::dataset::ParquetFileFormat>();
+  auto filesystem = std::make_shared<fs::LocalFileSystem>();
+
+  std::vector<fs::FileInfo> files;
+  const std::vector<std::string> f_paths = {"/tmp/data1.parquet", "/tmp/data2.parquet"};
+
+  for (const auto& f_path : f_paths) {
+    ASSERT_OK_AND_ASSIGN(auto f_file, filesystem->GetFileInfo(f_path));
+    files.push_back(std::move(f_file));
+  }
+
+  ASSERT_OK_AND_ASSIGN(auto ds_factory, dataset::FileSystemDatasetFactory::Make(
+                                            std::move(filesystem), std::move(files),
+                                            std::move(format), {}));
+  ASSERT_OK_AND_ASSIGN(auto dataset, ds_factory->Finish(dummy_schema));
+
+  auto options = std::make_shared<dataset::ScanOptions>();
+  options->projection = compute::project({}, {});
+  auto scan_node_options = dataset::ScanNodeOptions{dataset, options};
+
+  auto scan_declaration = compute::Declaration({"scan", scan_node_options});
+
+  ASSERT_OK_AND_ASSIGN(auto serialized_rel,
+                       SerializeRelation(scan_declaration, &ext_set));
+  ASSERT_OK_AND_ASSIGN(auto deserialized_decl,
+                       DeserializeRelation(*serialized_rel, ext_set));
+
+  auto dataset_comparator = [](std::shared_ptr<dataset::Dataset> ds_lhs,
+                               std::shared_ptr<dataset::Dataset> ds_rhs) -> bool {
+    const auto& fds_lhs = checked_cast<const dataset::FileSystemDataset&>(*ds_lhs);
+    const auto& fds_rhs = checked_cast<const dataset::FileSystemDataset&>(*ds_lhs);
+    const auto& files_lhs = fds_lhs.files();
+    const auto& files_rhs = fds_rhs.files();
+
+    bool cmp_fsize = files_lhs.size() == files_rhs.size();
+    uint64_t fidx = 0;
+    for (const auto& l_file : files_lhs) {
+      if (l_file != files_rhs[fidx]) {
+        return false;
+      }
+      fidx++;
+    }
+    bool cmp_file_format = fds_lhs.format()->Equals(*fds_lhs.format());
+    bool cmp_file_system = fds_lhs.filesystem()->Equals(fds_rhs.filesystem());
+    return cmp_fsize && cmp_file_format && cmp_file_system;
+  };
+
+  auto scan_option_comparator = [dataset_comparator](
+                                    const dataset::ScanNodeOptions& lhs,
+                                    const dataset::ScanNodeOptions& rhs) -> bool {
+    bool cmp_rso = lhs.require_sequenced_output == rhs.require_sequenced_output;
+    bool cmp_ds = dataset_comparator(lhs.dataset, rhs.dataset);
+    return cmp_rso && cmp_ds;
+  };

Review Comment:
   We can just do it here IMO



-- 
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] vibhatha commented on pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
vibhatha commented on PR #13401:
URL: https://github.com/apache/arrow/pull/13401#issuecomment-1181187263

   > A few more thoughts. Using parquet is fine. My only concern was the test data directory.
   
   We got rid of that. By the way this could would remain very much same, but ths usage would be different once this is subjected to a registry usage in Substrait `ToProto` methods. 


-- 
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] westonpace commented on a diff in pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
westonpace commented on code in PR #13401:
URL: https://github.com/apache/arrow/pull/13401#discussion_r958604739


##########
cpp/src/arrow/engine/substrait/plan_internal.cc:
##########
@@ -133,5 +136,18 @@ Result<ExtensionSet> GetExtensionSetFromPlan(const substrait::Plan& plan,
                             registry);
 }
 
+Result<std::unique_ptr<substrait::Plan>> PlanToProto(

Review Comment:
   That makes sense.  So the root problem is that a relation and a plan are both represented in Acero by `compute::Declaration` and so there is ambiguity.  I think the name `PlanToProto` is fine.



-- 
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] vibhatha commented on pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
vibhatha commented on PR #13401:
URL: https://github.com/apache/arrow/pull/13401#issuecomment-1229955921

   @westonpace Re: https://github.com/apache/arrow/pull/13401#discussion_r956480272
   
   The URL is like `/var/<some_path>/T//data.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.

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

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


[GitHub] [arrow] vibhatha commented on pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
vibhatha commented on PR #13401:
URL: https://github.com/apache/arrow/pull/13401#issuecomment-1160100835

   > I'm out Monday & Tuesday. Maybe @jvanstraten can take a look? Otherwise I can get to this on Wednesday
   
   Wednesday works for me 👍 


-- 
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] vibhatha commented on a diff in pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
vibhatha commented on code in PR #13401:
URL: https://github.com/apache/arrow/pull/13401#discussion_r901731391


##########
cpp/src/arrow/engine/substrait/serde_test.cc:
##########
@@ -1173,5 +1175,168 @@ TEST(Substrait, JoinPlanInvalidKeys) {
           &ext_set));
 }
 
+TEST(Substrait, SerializeRelation) {
+#ifdef _WIN32
+  GTEST_SKIP() << "ARROW-16392: Substrait File URI not supported for Windows";
+#else
+  ExtensionSet ext_set;
+  auto dummy_schema = schema({field("f1", int32()), field("f2", int32())});
+  // creating a dummy dataset using a dummy table
+  auto format = std::make_shared<arrow::dataset::ParquetFileFormat>();
+  auto filesystem = std::make_shared<fs::LocalFileSystem>();
+
+  std::vector<fs::FileInfo> files;
+  const std::vector<std::string> f_paths = {"/tmp/data1.parquet", "/tmp/data2.parquet"};
+
+  for (const auto& f_path : f_paths) {
+    ASSERT_OK_AND_ASSIGN(auto f_file, filesystem->GetFileInfo(f_path));
+    files.push_back(std::move(f_file));
+  }
+
+  ASSERT_OK_AND_ASSIGN(auto ds_factory, dataset::FileSystemDatasetFactory::Make(
+                                            std::move(filesystem), std::move(files),
+                                            std::move(format), {}));
+  ASSERT_OK_AND_ASSIGN(auto dataset, ds_factory->Finish(dummy_schema));
+
+  auto options = std::make_shared<dataset::ScanOptions>();
+  options->projection = compute::project({}, {});
+  auto scan_node_options = dataset::ScanNodeOptions{dataset, options};
+
+  auto scan_declaration = compute::Declaration({"scan", scan_node_options});
+
+  ASSERT_OK_AND_ASSIGN(auto serialized_rel,
+                       SerializeRelation(scan_declaration, &ext_set));
+  ASSERT_OK_AND_ASSIGN(auto deserialized_decl,
+                       DeserializeRelation(*serialized_rel, ext_set));
+
+  auto dataset_comparator = [](std::shared_ptr<dataset::Dataset> ds_lhs,
+                               std::shared_ptr<dataset::Dataset> ds_rhs) -> bool {
+    const auto& fds_lhs = checked_cast<const dataset::FileSystemDataset&>(*ds_lhs);
+    const auto& fds_rhs = checked_cast<const dataset::FileSystemDataset&>(*ds_lhs);
+    const auto& files_lhs = fds_lhs.files();
+    const auto& files_rhs = fds_rhs.files();
+
+    bool cmp_fsize = files_lhs.size() == files_rhs.size();
+    uint64_t fidx = 0;
+    for (const auto& l_file : files_lhs) {
+      if (l_file != files_rhs[fidx]) {
+        return false;
+      }
+      fidx++;
+    }
+    bool cmp_file_format = fds_lhs.format()->Equals(*fds_lhs.format());
+    bool cmp_file_system = fds_lhs.filesystem()->Equals(fds_rhs.filesystem());
+    return cmp_fsize && cmp_file_format && cmp_file_system;
+  };
+
+  auto scan_option_comparator = [dataset_comparator](
+                                    const dataset::ScanNodeOptions& lhs,
+                                    const dataset::ScanNodeOptions& rhs) -> bool {
+    bool cmp_rso = lhs.require_sequenced_output == rhs.require_sequenced_output;
+    bool cmp_ds = dataset_comparator(lhs.dataset, rhs.dataset);
+    return cmp_rso && cmp_ds;
+  };

Review Comment:
   Sure I will add it here.



-- 
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] vibhatha commented on a diff in pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
vibhatha commented on code in PR #13401:
URL: https://github.com/apache/arrow/pull/13401#discussion_r901275886


##########
cpp/src/arrow/engine/substrait/serde_test.cc:
##########
@@ -1173,5 +1175,168 @@ TEST(Substrait, JoinPlanInvalidKeys) {
           &ext_set));
 }
 
+TEST(Substrait, SerializeRelation) {
+#ifdef _WIN32
+  GTEST_SKIP() << "ARROW-16392: Substrait File URI not supported for Windows";
+#else
+  ExtensionSet ext_set;
+  auto dummy_schema = schema({field("f1", int32()), field("f2", int32())});
+  // creating a dummy dataset using a dummy table
+  auto format = std::make_shared<arrow::dataset::ParquetFileFormat>();
+  auto filesystem = std::make_shared<fs::LocalFileSystem>();
+
+  std::vector<fs::FileInfo> files;
+  const std::vector<std::string> f_paths = {"/tmp/data1.parquet", "/tmp/data2.parquet"};
+
+  for (const auto& f_path : f_paths) {
+    ASSERT_OK_AND_ASSIGN(auto f_file, filesystem->GetFileInfo(f_path));
+    files.push_back(std::move(f_file));
+  }
+
+  ASSERT_OK_AND_ASSIGN(auto ds_factory, dataset::FileSystemDatasetFactory::Make(
+                                            std::move(filesystem), std::move(files),
+                                            std::move(format), {}));
+  ASSERT_OK_AND_ASSIGN(auto dataset, ds_factory->Finish(dummy_schema));
+
+  auto options = std::make_shared<dataset::ScanOptions>();
+  options->projection = compute::project({}, {});
+  auto scan_node_options = dataset::ScanNodeOptions{dataset, options};
+
+  auto scan_declaration = compute::Declaration({"scan", scan_node_options});
+
+  ASSERT_OK_AND_ASSIGN(auto serialized_rel,
+                       SerializeRelation(scan_declaration, &ext_set));
+  ASSERT_OK_AND_ASSIGN(auto deserialized_decl,
+                       DeserializeRelation(*serialized_rel, ext_set));
+
+  auto dataset_comparator = [](std::shared_ptr<dataset::Dataset> ds_lhs,
+                               std::shared_ptr<dataset::Dataset> ds_rhs) -> bool {
+    const auto& fds_lhs = checked_cast<const dataset::FileSystemDataset&>(*ds_lhs);
+    const auto& fds_rhs = checked_cast<const dataset::FileSystemDataset&>(*ds_lhs);
+    const auto& files_lhs = fds_lhs.files();
+    const auto& files_rhs = fds_rhs.files();
+
+    bool cmp_fsize = files_lhs.size() == files_rhs.size();
+    uint64_t fidx = 0;
+    for (const auto& l_file : files_lhs) {
+      if (l_file != files_rhs[fidx]) {
+        return false;
+      }
+      fidx++;
+    }
+    bool cmp_file_format = fds_lhs.format()->Equals(*fds_lhs.format());
+    bool cmp_file_system = fds_lhs.filesystem()->Equals(fds_rhs.filesystem());
+    return cmp_fsize && cmp_file_format && cmp_file_system;
+  };
+
+  auto scan_option_comparator = [dataset_comparator](
+                                    const dataset::ScanNodeOptions& lhs,
+                                    const dataset::ScanNodeOptions& rhs) -> bool {
+    bool cmp_rso = lhs.require_sequenced_output == rhs.require_sequenced_output;
+    bool cmp_ds = dataset_comparator(lhs.dataset, rhs.dataset);
+    return cmp_rso && cmp_ds;
+  };

Review Comment:
   cc @westonpace @lidavidm 
   Would it be better to add a Equals method for `ScanNodeOptions` and may be even for all the other `ExecNodeOptions`?



-- 
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] westonpace commented on pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
westonpace commented on PR #13401:
URL: https://github.com/apache/arrow/pull/13401#issuecomment-1160081211

   I'm out Monday & Tuesday.  Maybe @jvanstraten can take a look?  Otherwise I can get to this on Wednesday 


-- 
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] lidavidm commented on a diff in pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
lidavidm commented on code in PR #13401:
URL: https://github.com/apache/arrow/pull/13401#discussion_r904934325


##########
cpp/src/arrow/engine/substrait/serde_test.cc:
##########
@@ -1173,5 +1175,168 @@ TEST(Substrait, JoinPlanInvalidKeys) {
           &ext_set));
 }
 
+TEST(Substrait, SerializeRelation) {
+#ifdef _WIN32
+  GTEST_SKIP() << "ARROW-16392: Substrait File URI not supported for Windows";
+#else
+  ExtensionSet ext_set;
+  auto dummy_schema = schema({field("f1", int32()), field("f2", int32())});
+  // creating a dummy dataset using a dummy table
+  auto format = std::make_shared<arrow::dataset::ParquetFileFormat>();
+  auto filesystem = std::make_shared<fs::LocalFileSystem>();
+
+  std::vector<fs::FileInfo> files;
+  const std::vector<std::string> f_paths = {"/tmp/data1.parquet", "/tmp/data2.parquet"};
+
+  for (const auto& f_path : f_paths) {
+    ASSERT_OK_AND_ASSIGN(auto f_file, filesystem->GetFileInfo(f_path));
+    files.push_back(std::move(f_file));
+  }
+
+  ASSERT_OK_AND_ASSIGN(auto ds_factory, dataset::FileSystemDatasetFactory::Make(
+                                            std::move(filesystem), std::move(files),
+                                            std::move(format), {}));
+  ASSERT_OK_AND_ASSIGN(auto dataset, ds_factory->Finish(dummy_schema));
+
+  auto options = std::make_shared<dataset::ScanOptions>();
+  options->projection = compute::project({}, {});
+  auto scan_node_options = dataset::ScanNodeOptions{dataset, options};
+
+  auto scan_declaration = compute::Declaration({"scan", scan_node_options});
+
+  ASSERT_OK_AND_ASSIGN(auto serialized_rel,
+                       SerializeRelation(scan_declaration, &ext_set));
+  ASSERT_OK_AND_ASSIGN(auto deserialized_decl,
+                       DeserializeRelation(*serialized_rel, ext_set));
+
+  auto dataset_comparator = [](std::shared_ptr<dataset::Dataset> ds_lhs,
+                               std::shared_ptr<dataset::Dataset> ds_rhs) -> bool {
+    const auto& fds_lhs = checked_cast<const dataset::FileSystemDataset&>(*ds_lhs);
+    const auto& fds_rhs = checked_cast<const dataset::FileSystemDataset&>(*ds_lhs);
+    const auto& files_lhs = fds_lhs.files();
+    const auto& files_rhs = fds_rhs.files();
+
+    bool cmp_fsize = files_lhs.size() == files_rhs.size();
+    uint64_t fidx = 0;
+    for (const auto& l_file : files_lhs) {
+      if (l_file != files_rhs[fidx]) {
+        return false;
+      }
+      fidx++;
+    }
+    bool cmp_file_format = fds_lhs.format()->Equals(*fds_lhs.format());
+    bool cmp_file_system = fds_lhs.filesystem()->Equals(fds_rhs.filesystem());
+    return cmp_fsize && cmp_file_format && cmp_file_system;
+  };
+
+  auto scan_option_comparator = [dataset_comparator](
+                                    const dataset::ScanNodeOptions& lhs,
+                                    const dataset::ScanNodeOptions& rhs) -> bool {
+    bool cmp_rso = lhs.require_sequenced_output == rhs.require_sequenced_output;
+    bool cmp_ds = dataset_comparator(lhs.dataset, rhs.dataset);
+    return cmp_rso && cmp_ds;
+  };

Review Comment:
   It's up to your judgement, though. If you feel like we need to make a new Jira to add those methods and tests, then go ahead.



-- 
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] lidavidm commented on a diff in pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
lidavidm commented on code in PR #13401:
URL: https://github.com/apache/arrow/pull/13401#discussion_r904933888


##########
cpp/src/arrow/engine/substrait/serde_test.cc:
##########
@@ -1173,5 +1175,168 @@ TEST(Substrait, JoinPlanInvalidKeys) {
           &ext_set));
 }
 
+TEST(Substrait, SerializeRelation) {
+#ifdef _WIN32
+  GTEST_SKIP() << "ARROW-16392: Substrait File URI not supported for Windows";
+#else
+  ExtensionSet ext_set;
+  auto dummy_schema = schema({field("f1", int32()), field("f2", int32())});
+  // creating a dummy dataset using a dummy table
+  auto format = std::make_shared<arrow::dataset::ParquetFileFormat>();
+  auto filesystem = std::make_shared<fs::LocalFileSystem>();
+
+  std::vector<fs::FileInfo> files;
+  const std::vector<std::string> f_paths = {"/tmp/data1.parquet", "/tmp/data2.parquet"};
+
+  for (const auto& f_path : f_paths) {
+    ASSERT_OK_AND_ASSIGN(auto f_file, filesystem->GetFileInfo(f_path));
+    files.push_back(std::move(f_file));
+  }
+
+  ASSERT_OK_AND_ASSIGN(auto ds_factory, dataset::FileSystemDatasetFactory::Make(
+                                            std::move(filesystem), std::move(files),
+                                            std::move(format), {}));
+  ASSERT_OK_AND_ASSIGN(auto dataset, ds_factory->Finish(dummy_schema));
+
+  auto options = std::make_shared<dataset::ScanOptions>();
+  options->projection = compute::project({}, {});
+  auto scan_node_options = dataset::ScanNodeOptions{dataset, options};
+
+  auto scan_declaration = compute::Declaration({"scan", scan_node_options});
+
+  ASSERT_OK_AND_ASSIGN(auto serialized_rel,
+                       SerializeRelation(scan_declaration, &ext_set));
+  ASSERT_OK_AND_ASSIGN(auto deserialized_decl,
+                       DeserializeRelation(*serialized_rel, ext_set));
+
+  auto dataset_comparator = [](std::shared_ptr<dataset::Dataset> ds_lhs,
+                               std::shared_ptr<dataset::Dataset> ds_rhs) -> bool {
+    const auto& fds_lhs = checked_cast<const dataset::FileSystemDataset&>(*ds_lhs);
+    const auto& fds_rhs = checked_cast<const dataset::FileSystemDataset&>(*ds_lhs);
+    const auto& files_lhs = fds_lhs.files();
+    const auto& files_rhs = fds_rhs.files();
+
+    bool cmp_fsize = files_lhs.size() == files_rhs.size();
+    uint64_t fidx = 0;
+    for (const auto& l_file : files_lhs) {
+      if (l_file != files_rhs[fidx]) {
+        return false;
+      }
+      fidx++;
+    }
+    bool cmp_file_format = fds_lhs.format()->Equals(*fds_lhs.format());
+    bool cmp_file_system = fds_lhs.filesystem()->Equals(fds_rhs.filesystem());
+    return cmp_fsize && cmp_file_format && cmp_file_system;
+  };
+
+  auto scan_option_comparator = [dataset_comparator](
+                                    const dataset::ScanNodeOptions& lhs,
+                                    const dataset::ScanNodeOptions& rhs) -> bool {
+    bool cmp_rso = lhs.require_sequenced_output == rhs.require_sequenced_output;
+    bool cmp_ds = dataset_comparator(lhs.dataset, rhs.dataset);
+    return cmp_rso && cmp_ds;
+  };

Review Comment:
   I don't see why not



-- 
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] vibhatha commented on a diff in pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
vibhatha commented on code in PR #13401:
URL: https://github.com/apache/arrow/pull/13401#discussion_r956915925


##########
cpp/src/arrow/engine/substrait/relation_internal.cc:
##########
@@ -421,5 +431,171 @@ Result<DeclarationInfo> FromProto(const substrait::Rel& rel, const ExtensionSet&
       rel.DebugString());
 }
 
+Result<std::unique_ptr<substrait::Rel>> ToProto(
+    const compute::Declaration& declr, ExtensionSet* ext_set,
+    const ConversionOptions& conversion_options) {
+  auto rel = make_unique<substrait::Rel>();
+  RETURN_NOT_OK(SerializeAndCombineRelations(declr, ext_set, rel, conversion_options));
+  return std::move(rel);
+}
+
+Status SetRelation(const std::unique_ptr<substrait::Rel>& plan,
+                   const std::unique_ptr<substrait::Rel>& partial_plan,
+                   const std::string& factory_name) {
+  if (factory_name == "scan" && partial_plan->has_read()) {
+    plan->set_allocated_read(partial_plan->release_read());
+  } else if (factory_name == "filter" && partial_plan->has_filter()) {
+    plan->set_allocated_filter(partial_plan->release_filter());
+  } else {
+    return Status::NotImplemented("Substrait converter ", factory_name,
+                                  " not supported.");
+  }
+  return Status::OK();
+}
+
+Result<std::shared_ptr<Schema>> ExtractSchemaToBind(const compute::Declaration& declr) {
+  std::shared_ptr<Schema> bind_schema;
+  if (declr.factory_name == "scan") {
+    const auto& opts = checked_cast<const dataset::ScanNodeOptions&>(*(declr.options));
+    bind_schema = opts.dataset->schema();
+  } else if (declr.factory_name == "filter") {
+    auto input_declr = util::get<compute::Declaration>(declr.inputs[0]);
+    ARROW_ASSIGN_OR_RAISE(bind_schema, ExtractSchemaToBind(input_declr));
+  } else if (declr.factory_name == "sink") {
+    // Note that the sink has no output_schema
+    return bind_schema;
+  } else {
+    return Status::Invalid("Schema extraction failed, unsupported factory ",
+                           declr.factory_name);
+  }
+  return bind_schema;
+}
+
+Status SerializeAndCombineRelations(const compute::Declaration& declaration,
+                                    ExtensionSet* ext_set,
+                                    std::unique_ptr<substrait::Rel>& rel,
+                                    const ConversionOptions& conversion_options) {
+  std::vector<compute::Declaration::Input> inputs = declaration.inputs;
+  for (auto& input : inputs) {
+    auto input_decl = util::get<compute::Declaration>(input);
+    RETURN_NOT_OK(
+        SerializeAndCombineRelations(input_decl, ext_set, rel, conversion_options));
+  }
+  const auto& factory_name = declaration.factory_name;
+  ARROW_ASSIGN_OR_RAISE(auto schema, ExtractSchemaToBind(declaration));
+  // Note that the sink declaration factory doesn't exist for serialization as
+  // Substrait doesn't deal with a sink node definition
+  std::unique_ptr<substrait::Rel> factory_rel;
+  if (factory_name == "scan") {
+    ARROW_ASSIGN_OR_RAISE(factory_rel, ScanRelationConverter(schema, declaration, ext_set,
+                                                             conversion_options));
+  } else if (factory_name == "filter") {
+    ARROW_ASSIGN_OR_RAISE(
+        factory_rel,
+        FilterRelationConverter(schema, declaration, ext_set, conversion_options));

Review Comment:
   This is correct. I removed that usage. 



-- 
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] bkietz commented on pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

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

   Since this API is designed to be a test-only utility I wouldn't expect us to need to deal with custom exec nodes


-- 
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] westonpace commented on a diff in pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
westonpace commented on code in PR #13401:
URL: https://github.com/apache/arrow/pull/13401#discussion_r918476843


##########
cpp/src/arrow/engine/substrait/serde_test.cc:
##########
@@ -1383,5 +1400,188 @@ TEST(Substrait, JoinPlanInvalidKeys) {
   }
 }
 
+TEST(Substrait, SerializeRelation) {
+#ifdef _WIN32
+  GTEST_SKIP() << "ARROW-16392: Substrait File URI not supported for Windows";
+#else
+  ExtensionSet ext_set;
+  auto dummy_schema = schema({field("foo", binary())});
+  // creating a dummy dataset using a dummy table
+  auto format = std::make_shared<arrow::dataset::ParquetFileFormat>();
+  auto filesystem = std::make_shared<fs::LocalFileSystem>();
+
+  std::vector<fs::FileInfo> files;
+  const std::vector<std::string> f_paths = {"/tmp/data1.parquet", "/tmp/data2.parquet"};

Review Comment:
   Yes, but that JIRA will eventually be fixed.  We don't want to make it harder to support Windows.



-- 
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] vibhatha commented on a diff in pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
vibhatha commented on code in PR #13401:
URL: https://github.com/apache/arrow/pull/13401#discussion_r907410211


##########
cpp/src/arrow/engine/substrait/serde_test.cc:
##########
@@ -1173,5 +1175,173 @@ TEST(Substrait, JoinPlanInvalidKeys) {
           &ext_set));
 }
 
+TEST(Substrait, SerializeRelation) {
+#ifdef _WIN32
+  GTEST_SKIP() << "ARROW-16392: Substrait File URI not supported for Windows";
+#else
+  ExtensionSet ext_set;
+  auto dummy_schema = schema({field("foo", binary())});
+  // creating a dummy dataset using a dummy table
+  auto format = std::make_shared<arrow::dataset::ParquetFileFormat>();
+  auto filesystem = std::make_shared<fs::LocalFileSystem>();
+
+  ASSERT_OK_AND_ASSIGN(std::string dir_string,
+                       arrow::internal::GetEnvVar("PARQUET_TEST_DATA"));
+  auto file_name =
+      arrow::internal::PlatformFilename::FromString(dir_string)->Join("binary.parquet");
+
+  std::vector<fs::FileInfo> files;
+  const std::vector<std::string> f_paths = {file_name->ToString()};
+
+  for (const auto& f_path : f_paths) {
+    ASSERT_OK_AND_ASSIGN(auto f_file, filesystem->GetFileInfo(f_path));
+    files.push_back(std::move(f_file));
+  }
+
+  ASSERT_OK_AND_ASSIGN(auto ds_factory, dataset::FileSystemDatasetFactory::Make(
+                                            std::move(filesystem), std::move(files),
+                                            std::move(format), {}));
+  ASSERT_OK_AND_ASSIGN(auto dataset, ds_factory->Finish(dummy_schema));
+
+  auto options = std::make_shared<dataset::ScanOptions>();
+  options->projection = compute::project({}, {});
+  auto scan_node_options = dataset::ScanNodeOptions{dataset, options};
+
+  auto scan_declaration = compute::Declaration({"scan", scan_node_options});
+
+  ASSERT_OK_AND_ASSIGN(auto serialized_rel,
+                       SerializeRelation(scan_declaration, &ext_set));
+  ASSERT_OK_AND_ASSIGN(auto deserialized_decl,
+                       DeserializeRelation(*serialized_rel, ext_set));
+
+  auto dataset_comparator = [](std::shared_ptr<dataset::Dataset> ds_lhs,
+                               std::shared_ptr<dataset::Dataset> ds_rhs) -> bool {
+    const auto& fds_lhs = checked_cast<const dataset::FileSystemDataset&>(*ds_lhs);
+    const auto& fds_rhs = checked_cast<const dataset::FileSystemDataset&>(*ds_lhs);
+    const auto& files_lhs = fds_lhs.files();
+    const auto& files_rhs = fds_rhs.files();
+
+    bool cmp_fsize = files_lhs.size() == files_rhs.size();

Review Comment:
   Good catch.



-- 
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] vibhatha commented on a diff in pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
vibhatha commented on code in PR #13401:
URL: https://github.com/apache/arrow/pull/13401#discussion_r918450190


##########
cpp/src/arrow/engine/substrait/serde_test.cc:
##########
@@ -1383,5 +1400,188 @@ TEST(Substrait, JoinPlanInvalidKeys) {
   }
 }
 
+TEST(Substrait, SerializeRelation) {
+#ifdef _WIN32
+  GTEST_SKIP() << "ARROW-16392: Substrait File URI not supported for Windows";
+#else
+  ExtensionSet ext_set;
+  auto dummy_schema = schema({field("foo", binary())});
+  // creating a dummy dataset using a dummy table
+  auto format = std::make_shared<arrow::dataset::ParquetFileFormat>();
+  auto filesystem = std::make_shared<fs::LocalFileSystem>();
+
+  std::vector<fs::FileInfo> files;
+  const std::vector<std::string> f_paths = {"/tmp/data1.parquet", "/tmp/data2.parquet"};
+
+  for (const auto& f_path : f_paths) {
+    ASSERT_OK_AND_ASSIGN(auto f_file, filesystem->GetFileInfo(f_path));
+    files.push_back(std::move(f_file));
+  }
+
+  ASSERT_OK_AND_ASSIGN(auto ds_factory, dataset::FileSystemDatasetFactory::Make(
+                                            std::move(filesystem), std::move(files),
+                                            std::move(format), {}));
+  ASSERT_OK_AND_ASSIGN(auto dataset, ds_factory->Finish(dummy_schema));
+
+  auto options = std::make_shared<dataset::ScanOptions>();
+  options->projection = compute::project({}, {});
+  auto scan_node_options = dataset::ScanNodeOptions{dataset, options};
+
+  auto scan_declaration = compute::Declaration({"scan", scan_node_options});
+
+  ASSERT_OK_AND_ASSIGN(auto serialized_rel,
+                       SerializeRelation(scan_declaration, &ext_set));
+  ASSERT_OK_AND_ASSIGN(auto deserialized_decl,
+                       DeserializeRelation(*serialized_rel, ext_set));
+
+  auto dataset_comparator = [](std::shared_ptr<dataset::Dataset> ds_lhs,
+                               std::shared_ptr<dataset::Dataset> ds_rhs) -> bool {
+    const auto& fsd_lhs = checked_cast<const dataset::FileSystemDataset&>(*ds_lhs);
+    const auto& fsd_rhs = checked_cast<const dataset::FileSystemDataset&>(*ds_lhs);
+    const auto& files_lhs = fsd_lhs.files();
+    const auto& files_rhs = fsd_rhs.files();
+
+    if (files_lhs.size() != files_rhs.size()) {
+      return false;
+    }
+    uint64_t fidx = 0;
+    for (const auto& l_file : files_lhs) {
+      if (l_file != files_rhs[fidx]) {
+        return false;
+      }
+      fidx++;
+    }
+    bool cmp_file_format = fsd_lhs.format()->Equals(*fsd_lhs.format());
+    bool cmp_file_system = fsd_lhs.filesystem()->Equals(fsd_rhs.filesystem());
+    return cmp_file_format && cmp_file_system;
+  };
+
+  auto scan_option_comparator = [dataset_comparator](
+                                    const dataset::ScanNodeOptions& lhs,
+                                    const dataset::ScanNodeOptions& rhs) -> bool {
+    bool cmp_rso = lhs.require_sequenced_output == rhs.require_sequenced_output;
+    bool cmp_ds = dataset_comparator(lhs.dataset, rhs.dataset);
+    return cmp_rso && cmp_ds;
+  };
+
+  EXPECT_EQ(deserialized_decl.factory_name, scan_declaration.factory_name);
+  const auto& lhs =
+      checked_cast<const dataset::ScanNodeOptions&>(*deserialized_decl.options);
+  const auto& rhs =
+      checked_cast<const dataset::ScanNodeOptions&>(*scan_declaration.options);
+  ASSERT_TRUE(scan_option_comparator(lhs, rhs));
+#endif
+}
+
+TEST(Substrait, SerializeRelationEndToEnd) {
+#ifdef _WIN32
+  GTEST_SKIP() << "ARROW-16392: Substrait File URI not supported for Windows";
+#else
+  ExtensionSet ext_set;
+  compute::ExecContext exec_context;
+
+  auto dummy_schema = schema({field("a", int32()), field("b", int32())});
+  auto table = TableFromJSON(dummy_schema, {R"([
+    [1, 1],
+    [3, 4]
+                        ])",
+                                            R"([
+    [0, 2],
+    [1, 3],
+    [4, 1],
+    [3, 1],
+    [1, 2]
+                        ])",
+                                            R"([
+    [2, 2],
+    [5, 3],
+    [1, 3]
+                        ])"});
+  // Note:: adding `/` to support the currently supported file read format from file
+  // system for Substrait

Review Comment:
   Ah instead of just saying 'test.parquet` we say `/test.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.

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

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


[GitHub] [arrow] vibhatha commented on pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
vibhatha commented on PR #13401:
URL: https://github.com/apache/arrow/pull/13401#issuecomment-1182855025

   ** THIS PR IS UNDER A REFACTOR ** 


-- 
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] vibhatha commented on a diff in pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
vibhatha commented on code in PR #13401:
URL: https://github.com/apache/arrow/pull/13401#discussion_r956926124


##########
cpp/src/arrow/engine/substrait/relation_internal.h:
##########
@@ -40,9 +40,46 @@ struct DeclarationInfo {
   int num_columns;
 };
 
+/// \brief A function to extract Acero Declaration from a Substrait Rel object
 ARROW_ENGINE_EXPORT
 Result<DeclarationInfo> FromProto(const substrait::Rel&, const ExtensionSet&,
                                   const ConversionOptions&);
 
+/// \brief Serializes a Declaration, produce a Substrait Rel and update the global
+/// Substrait plan. A Substrait Rel is passed as a the plan and it is updated with
+/// corresponding Declaration passed for serialization.
+///
+/// Note that this is a rather a helper method useful to fuse a partially serialized
+/// plan with another plan. The reason for having a partially serialized plan is to
+/// avoid unnecessary complication and enable partial plan serialization without
+/// affecting a global plan. Since kept as unique_ptr resources are relased efficiently
+/// upon releasing for the global plan.

Review Comment:
   Probably remove this and only use internally is better as you suggested.



-- 
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] vibhatha commented on a diff in pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
vibhatha commented on code in PR #13401:
URL: https://github.com/apache/arrow/pull/13401#discussion_r965601278


##########
cpp/src/arrow/util/io_util.cc:
##########
@@ -1867,7 +1867,16 @@ Result<std::unique_ptr<TemporaryDir>> TemporaryDir::Make(const std::string& pref
       [&](const NativePathString& base_dir) -> Result<std::unique_ptr<TemporaryDir>> {
     Status st;
     for (int attempt = 0; attempt < 3; ++attempt) {
-      PlatformFilename fn(base_dir + kNativeSep + base_name + kNativeSep);
+      // Note: certain temporary directories of MacOS contains a trailing slash
+      // Handling the base_dir with trailing slash
+      PlatformFilename fn;
+      if (base_dir.back() == kNativeSep) {
+        PlatformFilename fn_base_dir(base_dir);
+        PlatformFilename fn_base_name(base_name + kNativeSep);
+        fn = fn_base_dir.Join(fn_base_name);
+      } else {
+        fn = PlatformFilename(base_dir + kNativeSep + base_name + kNativeSep);
+      }

Review Comment:
   Yes it did. Because the case ignores adding an additional `kNativeSep` when it is already there, in the else it adds if it isn't there. Although, it is pretty simple just do what you suggest. Sorry I didn't catch this. 



-- 
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] vibhatha commented on pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
vibhatha commented on PR #13401:
URL: https://github.com/apache/arrow/pull/13401#issuecomment-1240577493

   @westonpace I updated the PR. Seems like a few CIs are failing. But, it seems like not related to the changes applied here.
   WDYT?


-- 
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] ursabot commented on pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
ursabot commented on PR #13401:
URL: https://github.com/apache/arrow/pull/13401#issuecomment-1241247840

   Benchmark runs are scheduled for baseline = 8fe7e35388a8147527037711e4262981fa81644a and contender = 74756051c4f6a8b13a40057f586817d56198d4ba. 74756051c4f6a8b13a40057f586817d56198d4ba is a master commit associated with this PR. Results will be available as each benchmark for each run completes.
   Conbench compare runs links:
   [Finished :arrow_down:0.0% :arrow_up:0.0%] [ec2-t3-xlarge-us-east-2](https://conbench.ursa.dev/compare/runs/b16e3947daff4cdbb9798edcbe2f53af...cba82d23a47f46c0bcc7032b75677233/)
   [Finished :arrow_down:0.31% :arrow_up:0.2%] [test-mac-arm](https://conbench.ursa.dev/compare/runs/9e184a5e494a48b5a52f0711a1fcc5a8...c4cba8343bf2471cb0069be1fff14414/)
   [Failed :arrow_down:0.0% :arrow_up:0.0%] [ursa-i9-9960x](https://conbench.ursa.dev/compare/runs/b9e767f1c620462a9cd25f7b7c2057a0...0be9118d824841829e1dcbf68fda127c/)
   [Finished :arrow_down:0.46% :arrow_up:0.0%] [ursa-thinkcentre-m75q](https://conbench.ursa.dev/compare/runs/af6bde81f52f4d5cb406dac3974954f3...a1357cf8955542db8212ebfec5c9058f/)
   Buildkite builds:
   [Finished] [`74756051` ec2-t3-xlarge-us-east-2](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ec2-t3-xlarge-us-east-2/builds/1441)
   [Finished] [`74756051` test-mac-arm](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-test-mac-arm/builds/1457)
   [Failed] [`74756051` ursa-i9-9960x](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ursa-i9-9960x/builds/1441)
   [Finished] [`74756051` ursa-thinkcentre-m75q](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ursa-thinkcentre-m75q/builds/1454)
   [Finished] [`8fe7e353` ec2-t3-xlarge-us-east-2](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ec2-t3-xlarge-us-east-2/builds/1440)
   [Finished] [`8fe7e353` test-mac-arm](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-test-mac-arm/builds/1456)
   [Failed] [`8fe7e353` ursa-i9-9960x](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ursa-i9-9960x/builds/1440)
   [Finished] [`8fe7e353` ursa-thinkcentre-m75q](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ursa-thinkcentre-m75q/builds/1453)
   Supported benchmarks:
   ec2-t3-xlarge-us-east-2: Supported benchmark langs: Python, R. Runs only benchmarks with cloud = True
   test-mac-arm: Supported benchmark langs: C++, Python, R
   ursa-i9-9960x: Supported benchmark langs: Python, R, JavaScript
   ursa-thinkcentre-m75q: Supported benchmark langs: C++, Java
   


-- 
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] vibhatha commented on pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
vibhatha commented on PR #13401:
URL: https://github.com/apache/arrow/pull/13401#issuecomment-1159879086

   cc @westonpace added the initial PR to integrate `ToProto` for relations. The detailed task breakdown for `ToProto` is documented in here: https://issues.apache.org/jira/browse/ARROW-16854
   
   The idea is to add part by part in smaller PRs. 


-- 
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] vibhatha commented on a diff in pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
vibhatha commented on code in PR #13401:
URL: https://github.com/apache/arrow/pull/13401#discussion_r953325526


##########
cpp/src/arrow/engine/substrait/registry.h:
##########
@@ -0,0 +1,86 @@
+// 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.
+
+// NOTE: API is EXPERIMENTAL and will change without going through a
+// deprecation cycle
+
+#pragma once
+
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/visibility.h"
+
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/engine/substrait/extension_set.h"
+#include "arrow/engine/substrait/extension_types.h"
+#include "arrow/engine/substrait/options.h"
+#include "arrow/engine/substrait/relation_internal.h"
+#include "arrow/engine/substrait/serde.h"
+#include "arrow/engine/substrait/visibility.h"
+#include "arrow/type_fwd.h"
+
+#include "substrait/algebra.pb.h"  // IWYU pragma: export
+
+namespace arrow {
+
+namespace engine {
+
+/// \brief Acero-Substrait integration contains converters which enables
+/// converting Acero ExecPlan related entities to the corresponding Substrait
+/// entities.
+///
+/// Note that the current registry definition only holds converters to convert
+/// an Acero plan to Substrait plan.
+class ARROW_ENGINE_EXPORT SubstraitConversionRegistry {
+ public:
+  virtual ~SubstraitConversionRegistry() = default;
+
+  /// \brief Alias for Acero-to-Substrait converter
+  using SubstraitConverter = std::function<Result<std::unique_ptr<substrait::Rel>>(
+      const std::shared_ptr<Schema>&, const compute::Declaration&, ExtensionSet*,
+      const ConversionOptions&)>;

Review Comment:
   this is removed now. 



-- 
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] vibhatha commented on a diff in pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
vibhatha commented on code in PR #13401:
URL: https://github.com/apache/arrow/pull/13401#discussion_r915650579


##########
cpp/src/arrow/engine/substrait/serde_test.cc:
##########
@@ -1173,5 +1175,173 @@ TEST(Substrait, JoinPlanInvalidKeys) {
           &ext_set));
 }
 
+TEST(Substrait, SerializeRelation) {
+#ifdef _WIN32
+  GTEST_SKIP() << "ARROW-16392: Substrait File URI not supported for Windows";
+#else
+  ExtensionSet ext_set;
+  auto dummy_schema = schema({field("foo", binary())});
+  // creating a dummy dataset using a dummy table
+  auto format = std::make_shared<arrow::dataset::ParquetFileFormat>();
+  auto filesystem = std::make_shared<fs::LocalFileSystem>();

Review Comment:
   @westonpace I removed the `PARQUET_TEST_DATA` usage from the the current test case, but I had to add `parquet` dependency for the tests. Is it acceptable? Also the older tests are still using `PARQUET_TEST_DATA`, shall we replace it in this PR itself?



-- 
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] vibhatha commented on a diff in pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
vibhatha commented on code in PR #13401:
URL: https://github.com/apache/arrow/pull/13401#discussion_r918450572


##########
cpp/src/arrow/engine/substrait/serde_test.cc:
##########
@@ -1383,5 +1400,188 @@ TEST(Substrait, JoinPlanInvalidKeys) {
   }
 }
 
+TEST(Substrait, SerializeRelation) {
+#ifdef _WIN32
+  GTEST_SKIP() << "ARROW-16392: Substrait File URI not supported for Windows";
+#else
+  ExtensionSet ext_set;
+  auto dummy_schema = schema({field("foo", binary())});
+  // creating a dummy dataset using a dummy table
+  auto format = std::make_shared<arrow::dataset::ParquetFileFormat>();
+  auto filesystem = std::make_shared<fs::LocalFileSystem>();
+
+  std::vector<fs::FileInfo> files;
+  const std::vector<std::string> f_paths = {"/tmp/data1.parquet", "/tmp/data2.parquet"};
+
+  for (const auto& f_path : f_paths) {
+    ASSERT_OK_AND_ASSIGN(auto f_file, filesystem->GetFileInfo(f_path));
+    files.push_back(std::move(f_file));
+  }
+
+  ASSERT_OK_AND_ASSIGN(auto ds_factory, dataset::FileSystemDatasetFactory::Make(
+                                            std::move(filesystem), std::move(files),
+                                            std::move(format), {}));
+  ASSERT_OK_AND_ASSIGN(auto dataset, ds_factory->Finish(dummy_schema));
+
+  auto options = std::make_shared<dataset::ScanOptions>();
+  options->projection = compute::project({}, {});
+  auto scan_node_options = dataset::ScanNodeOptions{dataset, options};
+
+  auto scan_declaration = compute::Declaration({"scan", scan_node_options});
+
+  ASSERT_OK_AND_ASSIGN(auto serialized_rel,
+                       SerializeRelation(scan_declaration, &ext_set));
+  ASSERT_OK_AND_ASSIGN(auto deserialized_decl,
+                       DeserializeRelation(*serialized_rel, ext_set));
+
+  auto dataset_comparator = [](std::shared_ptr<dataset::Dataset> ds_lhs,
+                               std::shared_ptr<dataset::Dataset> ds_rhs) -> bool {
+    const auto& fsd_lhs = checked_cast<const dataset::FileSystemDataset&>(*ds_lhs);
+    const auto& fsd_rhs = checked_cast<const dataset::FileSystemDataset&>(*ds_lhs);
+    const auto& files_lhs = fsd_lhs.files();
+    const auto& files_rhs = fsd_rhs.files();
+
+    if (files_lhs.size() != files_rhs.size()) {
+      return false;
+    }
+    uint64_t fidx = 0;
+    for (const auto& l_file : files_lhs) {
+      if (l_file != files_rhs[fidx]) {
+        return false;
+      }
+      fidx++;
+    }
+    bool cmp_file_format = fsd_lhs.format()->Equals(*fsd_lhs.format());
+    bool cmp_file_system = fsd_lhs.filesystem()->Equals(fsd_rhs.filesystem());
+    return cmp_file_format && cmp_file_system;
+  };
+
+  auto scan_option_comparator = [dataset_comparator](
+                                    const dataset::ScanNodeOptions& lhs,
+                                    const dataset::ScanNodeOptions& rhs) -> bool {
+    bool cmp_rso = lhs.require_sequenced_output == rhs.require_sequenced_output;
+    bool cmp_ds = dataset_comparator(lhs.dataset, rhs.dataset);
+    return cmp_rso && cmp_ds;
+  };
+
+  EXPECT_EQ(deserialized_decl.factory_name, scan_declaration.factory_name);
+  const auto& lhs =
+      checked_cast<const dataset::ScanNodeOptions&>(*deserialized_decl.options);
+  const auto& rhs =
+      checked_cast<const dataset::ScanNodeOptions&>(*scan_declaration.options);
+  ASSERT_TRUE(scan_option_comparator(lhs, rhs));
+#endif
+}
+
+TEST(Substrait, SerializeRelationEndToEnd) {
+#ifdef _WIN32
+  GTEST_SKIP() << "ARROW-16392: Substrait File URI not supported for Windows";
+#else
+  ExtensionSet ext_set;
+  compute::ExecContext exec_context;
+
+  auto dummy_schema = schema({field("a", int32()), field("b", int32())});
+  auto table = TableFromJSON(dummy_schema, {R"([
+    [1, 1],
+    [3, 4]
+                        ])",
+                                            R"([
+    [0, 2],
+    [1, 3],
+    [4, 1],
+    [3, 1],
+    [1, 2]
+                        ])",
+                                            R"([
+    [2, 2],
+    [5, 3],
+    [1, 3]
+                        ])"});
+  // Note:: adding `/` to support the currently supported file read format from file
+  // system for Substrait
+  const std::string path = "/testing.parquet";
+
+  EXPECT_OK_AND_ASSIGN(auto filesystem,
+                       fs::internal::MockFileSystem::Make(fs::kNoTime, {}));
+
+  EXPECT_EQ(WriteParquetData(path, filesystem, table), true);
+
+  auto format = std::make_shared<arrow::dataset::ParquetFileFormat>();
+
+  std::vector<fs::FileInfo> files;
+  ASSERT_OK_AND_ASSIGN(auto f_file, filesystem->GetFileInfo(path));
+  files.push_back(std::move(f_file));
+
+  ASSERT_OK_AND_ASSIGN(auto ds_factory, dataset::FileSystemDatasetFactory::Make(
+                                            std::move(filesystem), std::move(files),
+                                            std::move(format), {}));
+
+  ASSERT_OK_AND_ASSIGN(auto dataset, ds_factory->Finish(dummy_schema));
+
+  auto options = std::make_shared<dataset::ScanOptions>();
+  options->projection = compute::project({}, {});
+
+  auto scan_node_options = dataset::ScanNodeOptions{dataset, options};
+
+  arrow::AsyncGenerator<util::optional<compute::ExecBatch> > sink_gen;
+
+  auto sink_node_options = compute::SinkNodeOptions{&sink_gen};
+
+  auto scan_declaration = compute::Declaration({"scan", scan_node_options});
+  auto sink_declaration = compute::Declaration({"sink", sink_node_options});
+
+  auto declarations =
+      compute::Declaration::Sequence({scan_declaration, sink_declaration});
+
+  ASSERT_OK_AND_ASSIGN(auto plan, compute::ExecPlan::Make(&exec_context));
+  ASSERT_OK_AND_ASSIGN(auto decl, declarations.AddToPlan(plan.get()));
+
+  ASSERT_OK(decl->Validate());
+
+  std::shared_ptr<arrow::RecordBatchReader> sink_reader = compute::MakeGeneratorReader(
+      dummy_schema, std::move(sink_gen), exec_context.memory_pool());
+
+  ASSERT_OK(plan->Validate());
+  ASSERT_OK(plan->StartProducing());
+
+  std::shared_ptr<arrow::Table> response_table;
+
+  ASSERT_OK_AND_ASSIGN(response_table,
+                       arrow::Table::FromRecordBatchReader(sink_reader.get()));
+
+  ASSERT_OK_AND_ASSIGN(auto serialized_rel,
+                       SerializeRelation(scan_declaration, &ext_set));
+  ASSERT_OK_AND_ASSIGN(auto deserialized_decl,
+                       DeserializeRelation(*serialized_rel, ext_set));
+
+  arrow::AsyncGenerator<util::optional<compute::ExecBatch> > des_sink_gen;

Review Comment:
   `des` was meant to represent `deserialized`. I will update it's usage.



-- 
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] vibhatha commented on a diff in pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
vibhatha commented on code in PR #13401:
URL: https://github.com/apache/arrow/pull/13401#discussion_r901698427


##########
cpp/src/arrow/engine/substrait/serde_test.cc:
##########
@@ -1173,5 +1175,168 @@ TEST(Substrait, JoinPlanInvalidKeys) {
           &ext_set));
 }
 
+TEST(Substrait, SerializeRelation) {
+#ifdef _WIN32
+  GTEST_SKIP() << "ARROW-16392: Substrait File URI not supported for Windows";
+#else
+  ExtensionSet ext_set;
+  auto dummy_schema = schema({field("f1", int32()), field("f2", int32())});
+  // creating a dummy dataset using a dummy table
+  auto format = std::make_shared<arrow::dataset::ParquetFileFormat>();
+  auto filesystem = std::make_shared<fs::LocalFileSystem>();
+
+  std::vector<fs::FileInfo> files;
+  const std::vector<std::string> f_paths = {"/tmp/data1.parquet", "/tmp/data2.parquet"};
+
+  for (const auto& f_path : f_paths) {
+    ASSERT_OK_AND_ASSIGN(auto f_file, filesystem->GetFileInfo(f_path));
+    files.push_back(std::move(f_file));
+  }
+
+  ASSERT_OK_AND_ASSIGN(auto ds_factory, dataset::FileSystemDatasetFactory::Make(
+                                            std::move(filesystem), std::move(files),
+                                            std::move(format), {}));
+  ASSERT_OK_AND_ASSIGN(auto dataset, ds_factory->Finish(dummy_schema));
+
+  auto options = std::make_shared<dataset::ScanOptions>();
+  options->projection = compute::project({}, {});
+  auto scan_node_options = dataset::ScanNodeOptions{dataset, options};
+
+  auto scan_declaration = compute::Declaration({"scan", scan_node_options});
+
+  ASSERT_OK_AND_ASSIGN(auto serialized_rel,
+                       SerializeRelation(scan_declaration, &ext_set));
+  ASSERT_OK_AND_ASSIGN(auto deserialized_decl,
+                       DeserializeRelation(*serialized_rel, ext_set));
+
+  auto dataset_comparator = [](std::shared_ptr<dataset::Dataset> ds_lhs,
+                               std::shared_ptr<dataset::Dataset> ds_rhs) -> bool {
+    const auto& fds_lhs = checked_cast<const dataset::FileSystemDataset&>(*ds_lhs);
+    const auto& fds_rhs = checked_cast<const dataset::FileSystemDataset&>(*ds_lhs);
+    const auto& files_lhs = fds_lhs.files();
+    const auto& files_rhs = fds_rhs.files();
+
+    bool cmp_fsize = files_lhs.size() == files_rhs.size();
+    uint64_t fidx = 0;
+    for (const auto& l_file : files_lhs) {
+      if (l_file != files_rhs[fidx]) {
+        return false;
+      }
+      fidx++;
+    }
+    bool cmp_file_format = fds_lhs.format()->Equals(*fds_lhs.format());
+    bool cmp_file_system = fds_lhs.filesystem()->Equals(fds_rhs.filesystem());
+    return cmp_fsize && cmp_file_format && cmp_file_system;
+  };
+
+  auto scan_option_comparator = [dataset_comparator](
+                                    const dataset::ScanNodeOptions& lhs,
+                                    const dataset::ScanNodeOptions& rhs) -> bool {
+    bool cmp_rso = lhs.require_sequenced_output == rhs.require_sequenced_output;
+    bool cmp_ds = dataset_comparator(lhs.dataset, rhs.dataset);
+    return cmp_rso && cmp_ds;
+  };

Review Comment:
   Great, should it be a separate PR or we just include here?



-- 
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] westonpace commented on a diff in pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
westonpace commented on code in PR #13401:
URL: https://github.com/apache/arrow/pull/13401#discussion_r907299931


##########
cpp/src/arrow/engine/substrait/relation_internal.cc:
##########
@@ -316,5 +323,97 @@ Result<compute::Declaration> FromProto(const substrait::Rel& rel,
       rel.DebugString());
 }
 
+namespace {
+// TODO: add other types
+enum ArrowRelationType : uint8_t {
+  SCAN,
+  FILTER,
+  PROJECT,
+  JOIN,
+  AGGREGATE,
+};
+
+const std::map<std::string, ArrowRelationType> enum_map{
+    {"scan", ArrowRelationType::SCAN},           {"filter", ArrowRelationType::FILTER},
+    {"project", ArrowRelationType::PROJECT},     {"join", ArrowRelationType::JOIN},
+    {"aggregate", ArrowRelationType::AGGREGATE},
+};
+
+struct ExtractRelation {
+  explicit ExtractRelation(substrait::Rel* rel, ExtensionSet* ext_set)
+      : rel_(rel), ext_set_(ext_set) {}
+
+  Status AddRelation(const compute::Declaration& declaration) {
+    const std::string& rel_name = declaration.factory_name;
+    switch (enum_map.find(rel_name)->second) {
+      case ArrowRelationType::SCAN:
+        return AddReadRelation(declaration);
+      case ArrowRelationType::FILTER:
+        return Status::NotImplemented("Filter operator not supported.");
+      case ArrowRelationType::PROJECT:
+        return Status::NotImplemented("Project operator not supported.");
+      case ArrowRelationType::JOIN:
+        return Status::NotImplemented("Join operator not supported.");
+      case ArrowRelationType::AGGREGATE:
+        return Status::NotImplemented("Aggregate operator not supported.");
+      default:
+        return Status::Invalid("Unsupported factory name :", rel_name);
+    }
+  }

Review Comment:
   I'm not sure that introducing an enum, just so that we can switch on it, is much cleaner than comparing the strings directly.  For example:
   
   ```
   if (rel_name == "scan") {
     return AddReadRelation(declaration);
   } else if (rel_name == "filter") {
     return Status::NotImplemented("Filter operator not supported.");
   }
   ...
   } else {
     return Status::Invalid("Unsupported factory name :", rel_name);
   }
   ```
   
   Do you intend to use this enum elsewhere?



##########
cpp/src/arrow/engine/substrait/relation_internal.cc:
##########
@@ -316,5 +323,97 @@ Result<compute::Declaration> FromProto(const substrait::Rel& rel,
       rel.DebugString());
 }
 
+namespace {
+// TODO: add other types
+enum ArrowRelationType : uint8_t {
+  SCAN,
+  FILTER,
+  PROJECT,
+  JOIN,
+  AGGREGATE,
+};
+
+const std::map<std::string, ArrowRelationType> enum_map{
+    {"scan", ArrowRelationType::SCAN},           {"filter", ArrowRelationType::FILTER},
+    {"project", ArrowRelationType::PROJECT},     {"join", ArrowRelationType::JOIN},
+    {"aggregate", ArrowRelationType::AGGREGATE},
+};
+
+struct ExtractRelation {
+  explicit ExtractRelation(substrait::Rel* rel, ExtensionSet* ext_set)
+      : rel_(rel), ext_set_(ext_set) {}
+
+  Status AddRelation(const compute::Declaration& declaration) {
+    const std::string& rel_name = declaration.factory_name;
+    switch (enum_map.find(rel_name)->second) {
+      case ArrowRelationType::SCAN:
+        return AddReadRelation(declaration);
+      case ArrowRelationType::FILTER:
+        return Status::NotImplemented("Filter operator not supported.");
+      case ArrowRelationType::PROJECT:
+        return Status::NotImplemented("Project operator not supported.");
+      case ArrowRelationType::JOIN:
+        return Status::NotImplemented("Join operator not supported.");
+      case ArrowRelationType::AGGREGATE:
+        return Status::NotImplemented("Aggregate operator not supported.");
+      default:
+        return Status::Invalid("Unsupported factory name :", rel_name);
+    }
+  }
+
+  Status AddReadRelation(const compute::Declaration& declaration) {
+    auto read_rel = internal::make_unique<substrait::ReadRel>();
+    const auto& scan_node_options =
+        internal::checked_cast<const dataset::ScanNodeOptions&>(*declaration.options);
+
+    const auto& fds = internal::checked_cast<const dataset::FileSystemDataset&>(
+        *scan_node_options.dataset);

Review Comment:
   We don't know for certain that this cast will succeed (unlike the above cast to ScanNodeOptions).  A `checked_cast` will abort if it fails.
   
   Instead we should do a `dynamic_cast` to `dataset::FileSystemDataset*`.  It's slightly slower (`dynamic_cast` is slightly slower) but it will return `nullptr` if the cast fails (in which case we can return an invalid status "Can only convert file system datasets to a Substrait plan")



##########
cpp/src/arrow/engine/substrait/relation_internal.cc:
##########
@@ -316,5 +323,97 @@ Result<compute::Declaration> FromProto(const substrait::Rel& rel,
       rel.DebugString());
 }
 
+namespace {
+// TODO: add other types
+enum ArrowRelationType : uint8_t {
+  SCAN,
+  FILTER,
+  PROJECT,
+  JOIN,
+  AGGREGATE,
+};
+
+const std::map<std::string, ArrowRelationType> enum_map{
+    {"scan", ArrowRelationType::SCAN},           {"filter", ArrowRelationType::FILTER},
+    {"project", ArrowRelationType::PROJECT},     {"join", ArrowRelationType::JOIN},
+    {"aggregate", ArrowRelationType::AGGREGATE},
+};
+
+struct ExtractRelation {
+  explicit ExtractRelation(substrait::Rel* rel, ExtensionSet* ext_set)
+      : rel_(rel), ext_set_(ext_set) {}
+
+  Status AddRelation(const compute::Declaration& declaration) {
+    const std::string& rel_name = declaration.factory_name;
+    switch (enum_map.find(rel_name)->second) {
+      case ArrowRelationType::SCAN:
+        return AddReadRelation(declaration);
+      case ArrowRelationType::FILTER:
+        return Status::NotImplemented("Filter operator not supported.");
+      case ArrowRelationType::PROJECT:
+        return Status::NotImplemented("Project operator not supported.");
+      case ArrowRelationType::JOIN:
+        return Status::NotImplemented("Join operator not supported.");
+      case ArrowRelationType::AGGREGATE:
+        return Status::NotImplemented("Aggregate operator not supported.");
+      default:
+        return Status::Invalid("Unsupported factory name :", rel_name);

Review Comment:
   ```suggestion
           return Status::Invalid("Unsupported exec node factory name :", rel_name);
   ```



##########
cpp/src/arrow/engine/substrait/relation_internal.cc:
##########
@@ -316,5 +323,97 @@ Result<compute::Declaration> FromProto(const substrait::Rel& rel,
       rel.DebugString());
 }
 
+namespace {
+// TODO: add other types
+enum ArrowRelationType : uint8_t {
+  SCAN,
+  FILTER,
+  PROJECT,
+  JOIN,
+  AGGREGATE,
+};
+
+const std::map<std::string, ArrowRelationType> enum_map{
+    {"scan", ArrowRelationType::SCAN},           {"filter", ArrowRelationType::FILTER},
+    {"project", ArrowRelationType::PROJECT},     {"join", ArrowRelationType::JOIN},
+    {"aggregate", ArrowRelationType::AGGREGATE},
+};
+
+struct ExtractRelation {
+  explicit ExtractRelation(substrait::Rel* rel, ExtensionSet* ext_set)
+      : rel_(rel), ext_set_(ext_set) {}
+
+  Status AddRelation(const compute::Declaration& declaration) {
+    const std::string& rel_name = declaration.factory_name;
+    switch (enum_map.find(rel_name)->second) {
+      case ArrowRelationType::SCAN:
+        return AddReadRelation(declaration);
+      case ArrowRelationType::FILTER:
+        return Status::NotImplemented("Filter operator not supported.");
+      case ArrowRelationType::PROJECT:
+        return Status::NotImplemented("Project operator not supported.");
+      case ArrowRelationType::JOIN:
+        return Status::NotImplemented("Join operator not supported.");
+      case ArrowRelationType::AGGREGATE:
+        return Status::NotImplemented("Aggregate operator not supported.");
+      default:
+        return Status::Invalid("Unsupported factory name :", rel_name);
+    }
+  }
+
+  Status AddReadRelation(const compute::Declaration& declaration) {
+    auto read_rel = internal::make_unique<substrait::ReadRel>();
+    const auto& scan_node_options =
+        internal::checked_cast<const dataset::ScanNodeOptions&>(*declaration.options);
+
+    const auto& fds = internal::checked_cast<const dataset::FileSystemDataset&>(
+        *scan_node_options.dataset);
+
+    // set schema
+    ARROW_ASSIGN_OR_RAISE(auto named_struct, ToProto(*fds.schema(), ext_set_));
+    read_rel->set_allocated_base_schema(named_struct.release());
+
+    // set local files
+    auto read_rel_lfs = internal::make_unique<substrait::ReadRel_LocalFiles>();
+    for (const auto& file : fds.files()) {
+      auto read_rel_lfs_ffs =
+          internal::make_unique<substrait::ReadRel_LocalFiles_FileOrFiles>();
+      read_rel_lfs_ffs->set_uri_path("file://" + file);
+
+      // set file format
+      auto format_type_name = fds.format()->type_name();
+      if (format_type_name == "parquet" || format_type_name == "arrow" ||
+          format_type_name == "feather") {

Review Comment:
   We should add a comment here that `arrow` and `feather` are temporarily handled via the Parquet format until we upgrade to the latest Substrait version.  Otherwise this is maybe a little confusing to a future reader.



##########
cpp/src/arrow/engine/substrait/serde_test.cc:
##########
@@ -1173,5 +1175,173 @@ TEST(Substrait, JoinPlanInvalidKeys) {
           &ext_set));
 }
 
+TEST(Substrait, SerializeRelation) {
+#ifdef _WIN32
+  GTEST_SKIP() << "ARROW-16392: Substrait File URI not supported for Windows";
+#else
+  ExtensionSet ext_set;
+  auto dummy_schema = schema({field("foo", binary())});
+  // creating a dummy dataset using a dummy table
+  auto format = std::make_shared<arrow::dataset::ParquetFileFormat>();
+  auto filesystem = std::make_shared<fs::LocalFileSystem>();

Review Comment:
   I think we can use a mock filesystem here instead of relying on PARQUET_TEST_DATA.  We aren't actually reading the file so it doesn't need any contents.



##########
cpp/src/arrow/engine/substrait/relation_internal.cc:
##########
@@ -316,5 +323,97 @@ Result<compute::Declaration> FromProto(const substrait::Rel& rel,
       rel.DebugString());
 }
 
+namespace {
+// TODO: add other types
+enum ArrowRelationType : uint8_t {
+  SCAN,
+  FILTER,
+  PROJECT,
+  JOIN,
+  AGGREGATE,
+};
+
+const std::map<std::string, ArrowRelationType> enum_map{
+    {"scan", ArrowRelationType::SCAN},           {"filter", ArrowRelationType::FILTER},
+    {"project", ArrowRelationType::PROJECT},     {"join", ArrowRelationType::JOIN},
+    {"aggregate", ArrowRelationType::AGGREGATE},
+};
+
+struct ExtractRelation {
+  explicit ExtractRelation(substrait::Rel* rel, ExtensionSet* ext_set)
+      : rel_(rel), ext_set_(ext_set) {}
+
+  Status AddRelation(const compute::Declaration& declaration) {
+    const std::string& rel_name = declaration.factory_name;
+    switch (enum_map.find(rel_name)->second) {
+      case ArrowRelationType::SCAN:
+        return AddReadRelation(declaration);
+      case ArrowRelationType::FILTER:
+        return Status::NotImplemented("Filter operator not supported.");
+      case ArrowRelationType::PROJECT:
+        return Status::NotImplemented("Project operator not supported.");
+      case ArrowRelationType::JOIN:
+        return Status::NotImplemented("Join operator not supported.");
+      case ArrowRelationType::AGGREGATE:
+        return Status::NotImplemented("Aggregate operator not supported.");
+      default:
+        return Status::Invalid("Unsupported factory name :", rel_name);
+    }
+  }
+
+  Status AddReadRelation(const compute::Declaration& declaration) {
+    auto read_rel = internal::make_unique<substrait::ReadRel>();
+    const auto& scan_node_options =
+        internal::checked_cast<const dataset::ScanNodeOptions&>(*declaration.options);
+
+    const auto& fds = internal::checked_cast<const dataset::FileSystemDataset&>(
+        *scan_node_options.dataset);
+
+    // set schema
+    ARROW_ASSIGN_OR_RAISE(auto named_struct, ToProto(*fds.schema(), ext_set_));
+    read_rel->set_allocated_base_schema(named_struct.release());
+
+    // set local files
+    auto read_rel_lfs = internal::make_unique<substrait::ReadRel_LocalFiles>();
+    for (const auto& file : fds.files()) {
+      auto read_rel_lfs_ffs =
+          internal::make_unique<substrait::ReadRel_LocalFiles_FileOrFiles>();
+      read_rel_lfs_ffs->set_uri_path("file://" + file);
+
+      // set file format
+      auto format_type_name = fds.format()->type_name();
+      if (format_type_name == "parquet" || format_type_name == "arrow" ||
+          format_type_name == "feather") {
+        read_rel_lfs_ffs->set_format(
+            substrait::ReadRel::LocalFiles::FileOrFiles::FILE_FORMAT_PARQUET);
+      } else {
+        return Status::Invalid("Unsupported file type : ", format_type_name);
+      }
+      read_rel_lfs->mutable_items()->AddAllocated(read_rel_lfs_ffs.release());
+    }
+    *read_rel->mutable_local_files() = *read_rel_lfs.get();
+
+    rel_->set_allocated_read(read_rel.release());

Review Comment:
   It looks like we aren't handling any kind of pushdown projection or filtering.  Both of those would probably be a bit easier to do if I ever got around to finishing ARROW-16072.  Can you add a follow-up JIRA?



##########
cpp/src/arrow/engine/substrait/serde_test.cc:
##########
@@ -1173,5 +1175,168 @@ TEST(Substrait, JoinPlanInvalidKeys) {
           &ext_set));
 }
 
+TEST(Substrait, SerializeRelation) {
+#ifdef _WIN32
+  GTEST_SKIP() << "ARROW-16392: Substrait File URI not supported for Windows";
+#else
+  ExtensionSet ext_set;
+  auto dummy_schema = schema({field("f1", int32()), field("f2", int32())});
+  // creating a dummy dataset using a dummy table
+  auto format = std::make_shared<arrow::dataset::ParquetFileFormat>();
+  auto filesystem = std::make_shared<fs::LocalFileSystem>();
+
+  std::vector<fs::FileInfo> files;
+  const std::vector<std::string> f_paths = {"/tmp/data1.parquet", "/tmp/data2.parquet"};
+
+  for (const auto& f_path : f_paths) {
+    ASSERT_OK_AND_ASSIGN(auto f_file, filesystem->GetFileInfo(f_path));
+    files.push_back(std::move(f_file));
+  }
+
+  ASSERT_OK_AND_ASSIGN(auto ds_factory, dataset::FileSystemDatasetFactory::Make(
+                                            std::move(filesystem), std::move(files),
+                                            std::move(format), {}));
+  ASSERT_OK_AND_ASSIGN(auto dataset, ds_factory->Finish(dummy_schema));
+
+  auto options = std::make_shared<dataset::ScanOptions>();
+  options->projection = compute::project({}, {});
+  auto scan_node_options = dataset::ScanNodeOptions{dataset, options};
+
+  auto scan_declaration = compute::Declaration({"scan", scan_node_options});
+
+  ASSERT_OK_AND_ASSIGN(auto serialized_rel,
+                       SerializeRelation(scan_declaration, &ext_set));
+  ASSERT_OK_AND_ASSIGN(auto deserialized_decl,
+                       DeserializeRelation(*serialized_rel, ext_set));
+
+  auto dataset_comparator = [](std::shared_ptr<dataset::Dataset> ds_lhs,
+                               std::shared_ptr<dataset::Dataset> ds_rhs) -> bool {
+    const auto& fds_lhs = checked_cast<const dataset::FileSystemDataset&>(*ds_lhs);
+    const auto& fds_rhs = checked_cast<const dataset::FileSystemDataset&>(*ds_lhs);
+    const auto& files_lhs = fds_lhs.files();
+    const auto& files_rhs = fds_rhs.files();
+
+    bool cmp_fsize = files_lhs.size() == files_rhs.size();
+    uint64_t fidx = 0;
+    for (const auto& l_file : files_lhs) {
+      if (l_file != files_rhs[fidx]) {
+        return false;
+      }
+      fidx++;
+    }
+    bool cmp_file_format = fds_lhs.format()->Equals(*fds_lhs.format());
+    bool cmp_file_system = fds_lhs.filesystem()->Equals(fds_rhs.filesystem());
+    return cmp_fsize && cmp_file_format && cmp_file_system;
+  };
+
+  auto scan_option_comparator = [dataset_comparator](
+                                    const dataset::ScanNodeOptions& lhs,
+                                    const dataset::ScanNodeOptions& rhs) -> bool {
+    bool cmp_rso = lhs.require_sequenced_output == rhs.require_sequenced_output;
+    bool cmp_ds = dataset_comparator(lhs.dataset, rhs.dataset);
+    return cmp_rso && cmp_ds;
+  };

Review Comment:
   It makes sense for these things to have an `Equals` method.  It could even be useful to the user.  However, I don't see any reason we need to add it now.  What you have here is fine also.  If we end up doing the same comparison elsewhere we can also turn the lambdas into methods like `AssertDatasetEquals` inside of a `test_util` file.  So let's just keep this as simple lambdas for the moment.



##########
cpp/src/arrow/engine/substrait/serde_test.cc:
##########
@@ -1173,5 +1175,173 @@ TEST(Substrait, JoinPlanInvalidKeys) {
           &ext_set));
 }
 
+TEST(Substrait, SerializeRelation) {
+#ifdef _WIN32
+  GTEST_SKIP() << "ARROW-16392: Substrait File URI not supported for Windows";
+#else
+  ExtensionSet ext_set;
+  auto dummy_schema = schema({field("foo", binary())});
+  // creating a dummy dataset using a dummy table
+  auto format = std::make_shared<arrow::dataset::ParquetFileFormat>();
+  auto filesystem = std::make_shared<fs::LocalFileSystem>();
+
+  ASSERT_OK_AND_ASSIGN(std::string dir_string,
+                       arrow::internal::GetEnvVar("PARQUET_TEST_DATA"));
+  auto file_name =
+      arrow::internal::PlatformFilename::FromString(dir_string)->Join("binary.parquet");
+
+  std::vector<fs::FileInfo> files;
+  const std::vector<std::string> f_paths = {file_name->ToString()};
+
+  for (const auto& f_path : f_paths) {
+    ASSERT_OK_AND_ASSIGN(auto f_file, filesystem->GetFileInfo(f_path));
+    files.push_back(std::move(f_file));
+  }
+
+  ASSERT_OK_AND_ASSIGN(auto ds_factory, dataset::FileSystemDatasetFactory::Make(
+                                            std::move(filesystem), std::move(files),
+                                            std::move(format), {}));
+  ASSERT_OK_AND_ASSIGN(auto dataset, ds_factory->Finish(dummy_schema));
+
+  auto options = std::make_shared<dataset::ScanOptions>();
+  options->projection = compute::project({}, {});
+  auto scan_node_options = dataset::ScanNodeOptions{dataset, options};
+
+  auto scan_declaration = compute::Declaration({"scan", scan_node_options});
+
+  ASSERT_OK_AND_ASSIGN(auto serialized_rel,
+                       SerializeRelation(scan_declaration, &ext_set));
+  ASSERT_OK_AND_ASSIGN(auto deserialized_decl,
+                       DeserializeRelation(*serialized_rel, ext_set));
+
+  auto dataset_comparator = [](std::shared_ptr<dataset::Dataset> ds_lhs,
+                               std::shared_ptr<dataset::Dataset> ds_rhs) -> bool {
+    const auto& fds_lhs = checked_cast<const dataset::FileSystemDataset&>(*ds_lhs);
+    const auto& fds_rhs = checked_cast<const dataset::FileSystemDataset&>(*ds_lhs);
+    const auto& files_lhs = fds_lhs.files();
+    const auto& files_rhs = fds_rhs.files();
+
+    bool cmp_fsize = files_lhs.size() == files_rhs.size();

Review Comment:
   ```suggestion
       if (files_lhs.size() != files_rhs.size()) {
         return false;
       }
   ```
   If `files_lhs.size() > files_rhs.size()` then you could get a segmentation fault below at `files_rhs[fidx]`.  Best to just bail early if the # of files is unequal.



##########
cpp/src/arrow/engine/substrait/serde_test.cc:
##########
@@ -1173,5 +1175,173 @@ TEST(Substrait, JoinPlanInvalidKeys) {
           &ext_set));
 }
 
+TEST(Substrait, SerializeRelation) {
+#ifdef _WIN32
+  GTEST_SKIP() << "ARROW-16392: Substrait File URI not supported for Windows";
+#else
+  ExtensionSet ext_set;
+  auto dummy_schema = schema({field("foo", binary())});
+  // creating a dummy dataset using a dummy table
+  auto format = std::make_shared<arrow::dataset::ParquetFileFormat>();
+  auto filesystem = std::make_shared<fs::LocalFileSystem>();
+
+  ASSERT_OK_AND_ASSIGN(std::string dir_string,
+                       arrow::internal::GetEnvVar("PARQUET_TEST_DATA"));
+  auto file_name =
+      arrow::internal::PlatformFilename::FromString(dir_string)->Join("binary.parquet");
+
+  std::vector<fs::FileInfo> files;
+  const std::vector<std::string> f_paths = {file_name->ToString()};
+
+  for (const auto& f_path : f_paths) {
+    ASSERT_OK_AND_ASSIGN(auto f_file, filesystem->GetFileInfo(f_path));
+    files.push_back(std::move(f_file));
+  }
+
+  ASSERT_OK_AND_ASSIGN(auto ds_factory, dataset::FileSystemDatasetFactory::Make(
+                                            std::move(filesystem), std::move(files),
+                                            std::move(format), {}));
+  ASSERT_OK_AND_ASSIGN(auto dataset, ds_factory->Finish(dummy_schema));
+
+  auto options = std::make_shared<dataset::ScanOptions>();
+  options->projection = compute::project({}, {});
+  auto scan_node_options = dataset::ScanNodeOptions{dataset, options};
+
+  auto scan_declaration = compute::Declaration({"scan", scan_node_options});
+
+  ASSERT_OK_AND_ASSIGN(auto serialized_rel,
+                       SerializeRelation(scan_declaration, &ext_set));
+  ASSERT_OK_AND_ASSIGN(auto deserialized_decl,
+                       DeserializeRelation(*serialized_rel, ext_set));
+
+  auto dataset_comparator = [](std::shared_ptr<dataset::Dataset> ds_lhs,
+                               std::shared_ptr<dataset::Dataset> ds_rhs) -> bool {
+    const auto& fds_lhs = checked_cast<const dataset::FileSystemDataset&>(*ds_lhs);
+    const auto& fds_rhs = checked_cast<const dataset::FileSystemDataset&>(*ds_lhs);
+    const auto& files_lhs = fds_lhs.files();
+    const auto& files_rhs = fds_rhs.files();
+
+    bool cmp_fsize = files_lhs.size() == files_rhs.size();
+    uint64_t fidx = 0;
+    for (const auto& l_file : files_lhs) {
+      if (l_file != files_rhs[fidx]) {
+        return false;
+      }
+      fidx++;
+    }
+    bool cmp_file_format = fds_lhs.format()->Equals(*fds_lhs.format());
+    bool cmp_file_system = fds_lhs.filesystem()->Equals(fds_rhs.filesystem());
+    return cmp_fsize && cmp_file_format && cmp_file_system;
+  };
+
+  auto scan_option_comparator = [dataset_comparator](
+                                    const dataset::ScanNodeOptions& lhs,
+                                    const dataset::ScanNodeOptions& rhs) -> bool {
+    bool cmp_rso = lhs.require_sequenced_output == rhs.require_sequenced_output;
+    bool cmp_ds = dataset_comparator(lhs.dataset, rhs.dataset);
+    return cmp_rso && cmp_ds;
+  };
+
+  EXPECT_EQ(deserialized_decl.factory_name, scan_declaration.factory_name);
+  const auto& lhs =
+      checked_cast<const dataset::ScanNodeOptions&>(*deserialized_decl.options);
+  const auto& rhs =
+      checked_cast<const dataset::ScanNodeOptions&>(*scan_declaration.options);
+  ASSERT_TRUE(scan_option_comparator(lhs, rhs));
+#endif
+}
+
+TEST(Substrait, SerializeRelationEndToEnd) {

Review Comment:
   What is this test adding that the previous test does not already cover?



##########
cpp/src/arrow/engine/substrait/serde_test.cc:
##########
@@ -1173,5 +1175,173 @@ TEST(Substrait, JoinPlanInvalidKeys) {
           &ext_set));
 }
 
+TEST(Substrait, SerializeRelation) {
+#ifdef _WIN32
+  GTEST_SKIP() << "ARROW-16392: Substrait File URI not supported for Windows";
+#else
+  ExtensionSet ext_set;
+  auto dummy_schema = schema({field("foo", binary())});
+  // creating a dummy dataset using a dummy table
+  auto format = std::make_shared<arrow::dataset::ParquetFileFormat>();
+  auto filesystem = std::make_shared<fs::LocalFileSystem>();
+
+  ASSERT_OK_AND_ASSIGN(std::string dir_string,
+                       arrow::internal::GetEnvVar("PARQUET_TEST_DATA"));
+  auto file_name =
+      arrow::internal::PlatformFilename::FromString(dir_string)->Join("binary.parquet");
+
+  std::vector<fs::FileInfo> files;
+  const std::vector<std::string> f_paths = {file_name->ToString()};
+
+  for (const auto& f_path : f_paths) {
+    ASSERT_OK_AND_ASSIGN(auto f_file, filesystem->GetFileInfo(f_path));
+    files.push_back(std::move(f_file));
+  }
+
+  ASSERT_OK_AND_ASSIGN(auto ds_factory, dataset::FileSystemDatasetFactory::Make(
+                                            std::move(filesystem), std::move(files),
+                                            std::move(format), {}));
+  ASSERT_OK_AND_ASSIGN(auto dataset, ds_factory->Finish(dummy_schema));
+
+  auto options = std::make_shared<dataset::ScanOptions>();
+  options->projection = compute::project({}, {});
+  auto scan_node_options = dataset::ScanNodeOptions{dataset, options};
+
+  auto scan_declaration = compute::Declaration({"scan", scan_node_options});
+
+  ASSERT_OK_AND_ASSIGN(auto serialized_rel,
+                       SerializeRelation(scan_declaration, &ext_set));
+  ASSERT_OK_AND_ASSIGN(auto deserialized_decl,
+                       DeserializeRelation(*serialized_rel, ext_set));
+
+  auto dataset_comparator = [](std::shared_ptr<dataset::Dataset> ds_lhs,
+                               std::shared_ptr<dataset::Dataset> ds_rhs) -> bool {
+    const auto& fds_lhs = checked_cast<const dataset::FileSystemDataset&>(*ds_lhs);
+    const auto& fds_rhs = checked_cast<const dataset::FileSystemDataset&>(*ds_lhs);

Review Comment:
   ```suggestion
       const auto& fsd_lhs = checked_cast<const dataset::FileSystemDataset&>(*ds_lhs);
       const auto& fsd_rhs = checked_cast<const dataset::FileSystemDataset&>(*ds_lhs);
   ```
   Minor nit



##########
cpp/src/arrow/engine/substrait/relation_internal.cc:
##########
@@ -316,5 +323,97 @@ Result<compute::Declaration> FromProto(const substrait::Rel& rel,
       rel.DebugString());
 }
 
+namespace {
+// TODO: add other types
+enum ArrowRelationType : uint8_t {
+  SCAN,
+  FILTER,
+  PROJECT,
+  JOIN,
+  AGGREGATE,
+};
+
+const std::map<std::string, ArrowRelationType> enum_map{
+    {"scan", ArrowRelationType::SCAN},           {"filter", ArrowRelationType::FILTER},
+    {"project", ArrowRelationType::PROJECT},     {"join", ArrowRelationType::JOIN},
+    {"aggregate", ArrowRelationType::AGGREGATE},
+};
+
+struct ExtractRelation {
+  explicit ExtractRelation(substrait::Rel* rel, ExtensionSet* ext_set)
+      : rel_(rel), ext_set_(ext_set) {}
+
+  Status AddRelation(const compute::Declaration& declaration) {
+    const std::string& rel_name = declaration.factory_name;
+    switch (enum_map.find(rel_name)->second) {
+      case ArrowRelationType::SCAN:
+        return AddReadRelation(declaration);
+      case ArrowRelationType::FILTER:
+        return Status::NotImplemented("Filter operator not supported.");
+      case ArrowRelationType::PROJECT:
+        return Status::NotImplemented("Project operator not supported.");
+      case ArrowRelationType::JOIN:
+        return Status::NotImplemented("Join operator not supported.");
+      case ArrowRelationType::AGGREGATE:
+        return Status::NotImplemented("Aggregate operator not supported.");
+      default:
+        return Status::Invalid("Unsupported factory name :", rel_name);
+    }
+  }
+
+  Status AddReadRelation(const compute::Declaration& declaration) {
+    auto read_rel = internal::make_unique<substrait::ReadRel>();
+    const auto& scan_node_options =
+        internal::checked_cast<const dataset::ScanNodeOptions&>(*declaration.options);
+
+    const auto& fds = internal::checked_cast<const dataset::FileSystemDataset&>(
+        *scan_node_options.dataset);
+
+    // set schema
+    ARROW_ASSIGN_OR_RAISE(auto named_struct, ToProto(*fds.schema(), ext_set_));
+    read_rel->set_allocated_base_schema(named_struct.release());
+
+    // set local files
+    auto read_rel_lfs = internal::make_unique<substrait::ReadRel_LocalFiles>();
+    for (const auto& file : fds.files()) {
+      auto read_rel_lfs_ffs =
+          internal::make_unique<substrait::ReadRel_LocalFiles_FileOrFiles>();
+      read_rel_lfs_ffs->set_uri_path("file://" + file);
+
+      // set file format
+      auto format_type_name = fds.format()->type_name();
+      if (format_type_name == "parquet" || format_type_name == "arrow" ||
+          format_type_name == "feather") {
+        read_rel_lfs_ffs->set_format(
+            substrait::ReadRel::LocalFiles::FileOrFiles::FILE_FORMAT_PARQUET);
+      } else {
+        return Status::Invalid("Unsupported file type : ", format_type_name);
+      }
+      read_rel_lfs->mutable_items()->AddAllocated(read_rel_lfs_ffs.release());
+    }
+    *read_rel->mutable_local_files() = *read_rel_lfs.get();
+
+    rel_->set_allocated_read(read_rel.release());
+    return Status::OK();
+  }
+
+  Status operator()(const compute::Declaration& declaration) {
+    return AddRelation(declaration);
+  }
+
+ private:
+  substrait::Rel* rel_;
+  ExtensionSet* ext_set_;
+};
+
+}  // namespace
+
+Result<std::unique_ptr<substrait::Rel>> ToProto(const compute::Declaration& declaration,
+                                                ExtensionSet* ext_set) {
+  auto out = internal::make_unique<substrait::Rel>();
+  RETURN_NOT_OK(ExtractRelation(out.get(), ext_set)(declaration));

Review Comment:
   Why do we need a struct here instead something like:
   
   ```
   RETURN_NOT_OK(AddRelation(declaration, ext_set, out.get()));
   ```



##########
cpp/src/arrow/engine/substrait/relation_internal.cc:
##########
@@ -316,5 +323,97 @@ Result<compute::Declaration> FromProto(const substrait::Rel& rel,
       rel.DebugString());
 }
 
+namespace {
+// TODO: add other types
+enum ArrowRelationType : uint8_t {
+  SCAN,
+  FILTER,
+  PROJECT,
+  JOIN,
+  AGGREGATE,
+};
+
+const std::map<std::string, ArrowRelationType> enum_map{
+    {"scan", ArrowRelationType::SCAN},           {"filter", ArrowRelationType::FILTER},
+    {"project", ArrowRelationType::PROJECT},     {"join", ArrowRelationType::JOIN},
+    {"aggregate", ArrowRelationType::AGGREGATE},
+};
+
+struct ExtractRelation {
+  explicit ExtractRelation(substrait::Rel* rel, ExtensionSet* ext_set)
+      : rel_(rel), ext_set_(ext_set) {}
+
+  Status AddRelation(const compute::Declaration& declaration) {
+    const std::string& rel_name = declaration.factory_name;
+    switch (enum_map.find(rel_name)->second) {
+      case ArrowRelationType::SCAN:
+        return AddReadRelation(declaration);
+      case ArrowRelationType::FILTER:
+        return Status::NotImplemented("Filter operator not supported.");
+      case ArrowRelationType::PROJECT:
+        return Status::NotImplemented("Project operator not supported.");
+      case ArrowRelationType::JOIN:
+        return Status::NotImplemented("Join operator not supported.");
+      case ArrowRelationType::AGGREGATE:
+        return Status::NotImplemented("Aggregate operator not supported.");
+      default:
+        return Status::Invalid("Unsupported factory name :", rel_name);
+    }
+  }
+
+  Status AddReadRelation(const compute::Declaration& declaration) {
+    auto read_rel = internal::make_unique<substrait::ReadRel>();
+    const auto& scan_node_options =
+        internal::checked_cast<const dataset::ScanNodeOptions&>(*declaration.options);
+
+    const auto& fds = internal::checked_cast<const dataset::FileSystemDataset&>(

Review Comment:
   Minor nit: `fds` is maybe a little confusing.  Perhaps `dataset`?



-- 
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] vibhatha commented on a diff in pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
vibhatha commented on code in PR #13401:
URL: https://github.com/apache/arrow/pull/13401#discussion_r907424568


##########
cpp/src/arrow/engine/substrait/relation_internal.cc:
##########
@@ -316,5 +323,97 @@ Result<compute::Declaration> FromProto(const substrait::Rel& rel,
       rel.DebugString());
 }
 
+namespace {
+// TODO: add other types
+enum ArrowRelationType : uint8_t {
+  SCAN,
+  FILTER,
+  PROJECT,
+  JOIN,
+  AGGREGATE,
+};
+
+const std::map<std::string, ArrowRelationType> enum_map{
+    {"scan", ArrowRelationType::SCAN},           {"filter", ArrowRelationType::FILTER},
+    {"project", ArrowRelationType::PROJECT},     {"join", ArrowRelationType::JOIN},
+    {"aggregate", ArrowRelationType::AGGREGATE},
+};
+
+struct ExtractRelation {
+  explicit ExtractRelation(substrait::Rel* rel, ExtensionSet* ext_set)
+      : rel_(rel), ext_set_(ext_set) {}
+
+  Status AddRelation(const compute::Declaration& declaration) {
+    const std::string& rel_name = declaration.factory_name;
+    switch (enum_map.find(rel_name)->second) {
+      case ArrowRelationType::SCAN:
+        return AddReadRelation(declaration);
+      case ArrowRelationType::FILTER:
+        return Status::NotImplemented("Filter operator not supported.");
+      case ArrowRelationType::PROJECT:
+        return Status::NotImplemented("Project operator not supported.");
+      case ArrowRelationType::JOIN:
+        return Status::NotImplemented("Join operator not supported.");
+      case ArrowRelationType::AGGREGATE:
+        return Status::NotImplemented("Aggregate operator not supported.");
+      default:
+        return Status::Invalid("Unsupported factory name :", rel_name);
+    }
+  }
+
+  Status AddReadRelation(const compute::Declaration& declaration) {
+    auto read_rel = internal::make_unique<substrait::ReadRel>();
+    const auto& scan_node_options =
+        internal::checked_cast<const dataset::ScanNodeOptions&>(*declaration.options);
+
+    const auto& fds = internal::checked_cast<const dataset::FileSystemDataset&>(
+        *scan_node_options.dataset);
+
+    // set schema
+    ARROW_ASSIGN_OR_RAISE(auto named_struct, ToProto(*fds.schema(), ext_set_));
+    read_rel->set_allocated_base_schema(named_struct.release());
+
+    // set local files
+    auto read_rel_lfs = internal::make_unique<substrait::ReadRel_LocalFiles>();
+    for (const auto& file : fds.files()) {
+      auto read_rel_lfs_ffs =
+          internal::make_unique<substrait::ReadRel_LocalFiles_FileOrFiles>();
+      read_rel_lfs_ffs->set_uri_path("file://" + file);
+
+      // set file format
+      auto format_type_name = fds.format()->type_name();
+      if (format_type_name == "parquet" || format_type_name == "arrow" ||
+          format_type_name == "feather") {
+        read_rel_lfs_ffs->set_format(
+            substrait::ReadRel::LocalFiles::FileOrFiles::FILE_FORMAT_PARQUET);
+      } else {
+        return Status::Invalid("Unsupported file type : ", format_type_name);
+      }
+      read_rel_lfs->mutable_items()->AddAllocated(read_rel_lfs_ffs.release());
+    }
+    *read_rel->mutable_local_files() = *read_rel_lfs.get();
+
+    rel_->set_allocated_read(read_rel.release());
+    return Status::OK();
+  }
+
+  Status operator()(const compute::Declaration& declaration) {
+    return AddRelation(declaration);
+  }
+
+ private:
+  substrait::Rel* rel_;
+  ExtensionSet* ext_set_;
+};
+
+}  // namespace
+
+Result<std::unique_ptr<substrait::Rel>> ToProto(const compute::Declaration& declaration,
+                                                ExtensionSet* ext_set) {
+  auto out = internal::make_unique<substrait::Rel>();
+  RETURN_NOT_OK(ExtractRelation(out.get(), ext_set)(declaration));

Review Comment:
   I agree, on the face of it for the moment, it actually won't need a struct. But I wanted to make it consistent with the other ToProto-related codes in Expression, etc. Plus left room for improvement if required. In other ToProto-related interfaces, it is more complicated than this, it uses Visitor, etc. This one is pretty simple, but I thought it is bettter to abstract things away with a struct. We can definitly loose it and just go for a light weight function.



-- 
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] vibhatha commented on a diff in pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
vibhatha commented on code in PR #13401:
URL: https://github.com/apache/arrow/pull/13401#discussion_r904933667


##########
cpp/src/arrow/engine/substrait/serde_test.cc:
##########
@@ -1173,5 +1175,168 @@ TEST(Substrait, JoinPlanInvalidKeys) {
           &ext_set));
 }
 
+TEST(Substrait, SerializeRelation) {
+#ifdef _WIN32
+  GTEST_SKIP() << "ARROW-16392: Substrait File URI not supported for Windows";
+#else
+  ExtensionSet ext_set;
+  auto dummy_schema = schema({field("f1", int32()), field("f2", int32())});
+  // creating a dummy dataset using a dummy table
+  auto format = std::make_shared<arrow::dataset::ParquetFileFormat>();
+  auto filesystem = std::make_shared<fs::LocalFileSystem>();
+
+  std::vector<fs::FileInfo> files;
+  const std::vector<std::string> f_paths = {"/tmp/data1.parquet", "/tmp/data2.parquet"};
+
+  for (const auto& f_path : f_paths) {
+    ASSERT_OK_AND_ASSIGN(auto f_file, filesystem->GetFileInfo(f_path));
+    files.push_back(std::move(f_file));
+  }
+
+  ASSERT_OK_AND_ASSIGN(auto ds_factory, dataset::FileSystemDatasetFactory::Make(
+                                            std::move(filesystem), std::move(files),
+                                            std::move(format), {}));
+  ASSERT_OK_AND_ASSIGN(auto dataset, ds_factory->Finish(dummy_schema));
+
+  auto options = std::make_shared<dataset::ScanOptions>();
+  options->projection = compute::project({}, {});
+  auto scan_node_options = dataset::ScanNodeOptions{dataset, options};
+
+  auto scan_declaration = compute::Declaration({"scan", scan_node_options});
+
+  ASSERT_OK_AND_ASSIGN(auto serialized_rel,
+                       SerializeRelation(scan_declaration, &ext_set));
+  ASSERT_OK_AND_ASSIGN(auto deserialized_decl,
+                       DeserializeRelation(*serialized_rel, ext_set));
+
+  auto dataset_comparator = [](std::shared_ptr<dataset::Dataset> ds_lhs,
+                               std::shared_ptr<dataset::Dataset> ds_rhs) -> bool {
+    const auto& fds_lhs = checked_cast<const dataset::FileSystemDataset&>(*ds_lhs);
+    const auto& fds_rhs = checked_cast<const dataset::FileSystemDataset&>(*ds_lhs);
+    const auto& files_lhs = fds_lhs.files();
+    const auto& files_rhs = fds_rhs.files();
+
+    bool cmp_fsize = files_lhs.size() == files_rhs.size();
+    uint64_t fidx = 0;
+    for (const auto& l_file : files_lhs) {
+      if (l_file != files_rhs[fidx]) {
+        return false;
+      }
+      fidx++;
+    }
+    bool cmp_file_format = fds_lhs.format()->Equals(*fds_lhs.format());
+    bool cmp_file_system = fds_lhs.filesystem()->Equals(fds_rhs.filesystem());
+    return cmp_fsize && cmp_file_format && cmp_file_system;
+  };
+
+  auto scan_option_comparator = [dataset_comparator](
+                                    const dataset::ScanNodeOptions& lhs,
+                                    const dataset::ScanNodeOptions& rhs) -> bool {
+    bool cmp_rso = lhs.require_sequenced_output == rhs.require_sequenced_output;
+    bool cmp_ds = dataset_comparator(lhs.dataset, rhs.dataset);
+    return cmp_rso && cmp_ds;
+  };

Review Comment:
   Please note that the current `Equals` lambdas doesn't contain the complete functionality. 



-- 
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] bkietz commented on a diff in pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
bkietz commented on code in PR #13401:
URL: https://github.com/apache/arrow/pull/13401#discussion_r937881009


##########
cpp/src/arrow/engine/substrait/serde_test.cc:
##########
@@ -1731,5 +1770,265 @@ TEST(Substrait, AggregateWithFilter) {
   ASSERT_RAISES(NotImplemented, DeserializePlans(*buf, [] { return kNullConsumer; }));
 }
 
+TEST(Substrait, BasicPlanRoundTripping) {
+#ifdef _WIN32
+  GTEST_SKIP() << "ARROW-16392: Substrait File URI not supported for Windows";
+#else
+  compute::ExecContext exec_context;
+  ExtensionSet ext_set;
+  auto dummy_schema = schema(
+      {field("key", int32()), field("shared", int32()), field("distinct", int32())});
+
+  // creating a dummy dataset using a dummy table
+  auto table = TableFromJSON(dummy_schema, {R"([
+      [1, 1, 10],
+      [3, 4, 20]
+    ])",
+                                            R"([
+      [0, 2, 1],
+      [1, 3, 2],
+      [4, 1, 3],
+      [3, 1, 3],
+      [1, 2, 5]
+    ])",
+                                            R"([
+      [2, 2, 12],
+      [5, 3, 12],
+      [1, 3, 12]
+    ])"});
+
+  auto format = std::make_shared<arrow::dataset::ParquetFileFormat>();
+  auto filesystem = std::make_shared<fs::LocalFileSystem>();
+  const std::string file_name = "serde_test.parquet";
+
+  ASSERT_OK_AND_ASSIGN(auto tempdir,
+                       arrow::internal::TemporaryDir::Make("substrait_tempdir"));
+  ASSERT_OK_AND_ASSIGN(auto file_path, tempdir->path().Join(file_name));
+  std::string file_path_str = file_path.ToString();
+
+  // Note: there is an additional forward slash introduced by the tempdir
+  // it must be replaced to properly load into reading files
+  // TODO: (Review: Jira needs to be reported to handle this properly)
+  std::string toReplace("/T//");
+  size_t pos = file_path_str.find(toReplace);
+  file_path_str.replace(pos, toReplace.length(), "/T/");
+
+  ARROW_EXPECT_OK(WriteParquetData(file_path_str, filesystem, table));
+
+  std::vector<fs::FileInfo> files;
+  const std::vector<std::string> f_paths = {file_path_str};
+
+  for (const auto& f_path : f_paths) {
+    ASSERT_OK_AND_ASSIGN(auto f_file, filesystem->GetFileInfo(f_path));
+    files.push_back(std::move(f_file));
+  }
+
+  ASSERT_OK_AND_ASSIGN(auto ds_factory, dataset::FileSystemDatasetFactory::Make(
+                                            filesystem, std::move(files), format, {}));
+  ASSERT_OK_AND_ASSIGN(auto dataset, ds_factory->Finish(dummy_schema));
+
+  auto scan_options = std::make_shared<dataset::ScanOptions>();
+  scan_options->projection = compute::project({}, {});
+  const std::string filter_col = "shared";
+  auto filter = compute::equal(compute::field_ref(filter_col), compute::literal(3));
+
+  arrow::AsyncGenerator<util::optional<compute::ExecBatch>> sink_gen;
+
+  auto scan_node_options = dataset::ScanNodeOptions{dataset, scan_options};
+  auto filter_node_options = compute::FilterNodeOptions{filter};
+  auto sink_node_options = compute::SinkNodeOptions{&sink_gen};
+
+  auto scan_declaration = compute::Declaration({"scan", scan_node_options, "s"});
+  auto filter_declaration = compute::Declaration({"filter", filter_node_options, "f"});
+  auto sink_declaration = compute::Declaration({"sink", sink_node_options, "e"});
+
+  auto declarations = compute::Declaration::Sequence(
+      {scan_declaration, filter_declaration, sink_declaration});
+
+  ASSERT_OK_AND_ASSIGN(auto plan, compute::ExecPlan::Make(&exec_context));
+
+  ASSERT_OK_AND_ASSIGN(auto serialized_plan, SerializePlan(declarations, &ext_set));
+
+  for (auto sp_ext_id_reg :
+       {std::shared_ptr<ExtensionIdRegistry>(), substrait::MakeExtensionIdRegistry()}) {
+    ExtensionIdRegistry* ext_id_reg = sp_ext_id_reg.get();
+    ExtensionSet ext_set(ext_id_reg);
+    ASSERT_OK_AND_ASSIGN(
+        auto sink_decls,
+        DeserializePlans(
+            *serialized_plan, [] { return kNullConsumer; }, ext_id_reg, &ext_set));
+    // filter declaration
+    auto roundtripped_filter = sink_decls[0].inputs[0].get<compute::Declaration>();
+    const auto& filter_opts =
+        checked_cast<const compute::FilterNodeOptions&>(*(roundtripped_filter->options));
+    auto roundtripped_expr = filter_opts.filter_expression;
+
+    if (auto* call = roundtripped_expr.call()) {
+      EXPECT_EQ(call->function_name, "equal");
+      auto args = call->arguments;
+      auto index = args[0].field_ref()->field_path()->indices()[0];
+      EXPECT_EQ(dummy_schema->field_names()[index], filter_col);
+      EXPECT_EQ(args[1], compute::literal(3));
+    }
+    // scan declaration
+    auto roundtripped_scan = roundtripped_filter->inputs[0].get<compute::Declaration>();
+    const auto& dataset_opts =
+        checked_cast<const dataset::ScanNodeOptions&>(*(roundtripped_scan->options));
+    const auto& roundripped_ds = dataset_opts.dataset;
+    EXPECT_TRUE(roundripped_ds->schema()->Equals(*dummy_schema));
+    ASSERT_OK_AND_ASSIGN(auto roundtripped_frgs, roundripped_ds->GetFragments());
+    ASSERT_OK_AND_ASSIGN(auto expected_frgs, dataset->GetFragments());
+
+    auto roundtrip_frg_vec = IteratorToVector(std::move(roundtripped_frgs));
+    auto expected_frg_vec = IteratorToVector(std::move(expected_frgs));
+    EXPECT_EQ(expected_frg_vec.size(), roundtrip_frg_vec.size());
+    int64_t idx = 0;
+    for (auto fragment : expected_frg_vec) {
+      const auto* l_frag = checked_cast<const dataset::FileFragment*>(fragment.get());
+      const auto* r_frag =
+          checked_cast<const dataset::FileFragment*>(roundtrip_frg_vec[idx++].get());
+      EXPECT_TRUE(l_frag->Equals(*r_frag));
+    }
+  }
+#endif
+}
+
+TEST(Substrait, BasicPlanRoundTrippingEndToEnd) {
+#ifdef _WIN32
+  GTEST_SKIP() << "ARROW-16392: Substrait File URI not supported for Windows";
+#else
+  compute::ExecContext exec_context;
+  ExtensionSet ext_set;
+  auto dummy_schema = schema(
+      {field("key", int32()), field("shared", int32()), field("distinct", int32())});
+
+  // creating a dummy dataset using a dummy table
+  auto table = TableFromJSON(dummy_schema, {R"([
+      [1, 1, 10],
+      [3, 4, 20]
+    ])",
+                                            R"([
+      [0, 2, 1],
+      [1, 3, 2],
+      [4, 1, 3],
+      [3, 1, 3],
+      [1, 2, 5]
+    ])",
+                                            R"([
+      [2, 2, 12],
+      [5, 3, 12],
+      [1, 3, 12]
+    ])"});
+
+  auto format = std::make_shared<arrow::dataset::ParquetFileFormat>();
+  auto filesystem = std::make_shared<fs::LocalFileSystem>();
+  const std::string file_name = "serde_test.parquet";
+
+  ASSERT_OK_AND_ASSIGN(auto tempdir,
+                       arrow::internal::TemporaryDir::Make("substrait_tempdir"));
+  ASSERT_OK_AND_ASSIGN(auto file_path, tempdir->path().Join(file_name));
+  std::string file_path_str = file_path.ToNative();
+
+  // Note: there is an additional forward slash introduced by the tempdir
+  // it must be replaced to properly load into reading files
+  // TODO: (Review: Jira needs to be reported to handle this properly)
+  std::string toReplace("/T//");
+  size_t pos = file_path_str.find(toReplace);
+  file_path_str.replace(pos, toReplace.length(), "/T/");
+
+  ARROW_EXPECT_OK(WriteParquetData(file_path_str, filesystem, table));
+
+  std::vector<fs::FileInfo> files;
+  const std::vector<std::string> f_paths = {file_path_str};
+
+  for (const auto& f_path : f_paths) {
+    ASSERT_OK_AND_ASSIGN(auto f_file, filesystem->GetFileInfo(f_path));
+    files.push_back(std::move(f_file));
+  }
+
+  ASSERT_OK_AND_ASSIGN(auto ds_factory, dataset::FileSystemDatasetFactory::Make(
+                                            filesystem, std::move(files), format, {}));
+  ASSERT_OK_AND_ASSIGN(auto dataset, ds_factory->Finish(dummy_schema));
+
+  auto scan_options = std::make_shared<dataset::ScanOptions>();
+  scan_options->projection = compute::project({}, {});
+  const std::string filter_col = "shared";
+  auto filter = compute::equal(compute::field_ref(filter_col), compute::literal(3));
+
+  arrow::AsyncGenerator<util::optional<compute::ExecBatch>> sink_gen;
+
+  auto scan_node_options = dataset::ScanNodeOptions{dataset, scan_options};
+  auto filter_node_options = compute::FilterNodeOptions{filter};
+  auto sink_node_options = compute::SinkNodeOptions{&sink_gen};
+
+  auto scan_declaration = compute::Declaration({"scan", scan_node_options, "s"});
+  auto filter_declaration = compute::Declaration({"filter", filter_node_options, "f"});
+  auto sink_declaration = compute::Declaration({"sink", sink_node_options, "e"});
+
+  auto declarations = compute::Declaration::Sequence(
+      {scan_declaration, filter_declaration, sink_declaration});

Review Comment:
   Nit: instead of naming each of these could we fold them into a single sequence?
   ```suggestion
     auto declarations = compute::Declaration::Sequence({
         {"scan", dataset::ScanNodeOptions{dataset, scan_options}, "s"},
         {"filter", compute::FilterNodeOptions{filter}, "f"},
         {"sink", compute::SinkNodeOptions{&sink_gen}, "e"},
     });
   ```



##########
cpp/src/arrow/engine/substrait/registry.h:
##########
@@ -0,0 +1,86 @@
+// 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.
+
+// NOTE: API is EXPERIMENTAL and will change without going through a
+// deprecation cycle
+
+#pragma once
+
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "arrow/result.h"
+#include "arrow/status.h"
+#include "arrow/util/visibility.h"
+
+#include "arrow/compute/exec/exec_plan.h"
+#include "arrow/engine/substrait/extension_set.h"
+#include "arrow/engine/substrait/extension_types.h"
+#include "arrow/engine/substrait/options.h"
+#include "arrow/engine/substrait/relation_internal.h"
+#include "arrow/engine/substrait/serde.h"
+#include "arrow/engine/substrait/visibility.h"
+#include "arrow/type_fwd.h"
+
+#include "substrait/algebra.pb.h"  // IWYU pragma: export
+
+namespace arrow {
+
+namespace engine {
+
+/// \brief Acero-Substrait integration contains converters which enables
+/// converting Acero ExecPlan related entities to the corresponding Substrait
+/// entities.
+///
+/// Note that the current registry definition only holds converters to convert
+/// an Acero plan to Substrait plan.
+class ARROW_ENGINE_EXPORT SubstraitConversionRegistry {
+ public:
+  virtual ~SubstraitConversionRegistry() = default;
+
+  /// \brief Alias for Acero-to-Substrait converter
+  using SubstraitConverter = std::function<Result<std::unique_ptr<substrait::Rel>>(
+      const std::shared_ptr<Schema>&, const compute::Declaration&, ExtensionSet*,
+      const ConversionOptions&)>;

Review Comment:
   Since this class references the protobuf message classes, it should be confined to `_internal.h` and `.cc` files to ensure it is not installed along with the other arrow headers; the protobuf message classes are currently private/internal



##########
cpp/src/arrow/engine/substrait/serde_test.cc:
##########
@@ -1731,5 +1770,265 @@ TEST(Substrait, AggregateWithFilter) {
   ASSERT_RAISES(NotImplemented, DeserializePlans(*buf, [] { return kNullConsumer; }));
 }
 
+TEST(Substrait, BasicPlanRoundTripping) {
+#ifdef _WIN32
+  GTEST_SKIP() << "ARROW-16392: Substrait File URI not supported for Windows";
+#else
+  compute::ExecContext exec_context;
+  ExtensionSet ext_set;
+  auto dummy_schema = schema(
+      {field("key", int32()), field("shared", int32()), field("distinct", int32())});
+
+  // creating a dummy dataset using a dummy table
+  auto table = TableFromJSON(dummy_schema, {R"([
+      [1, 1, 10],
+      [3, 4, 20]
+    ])",
+                                            R"([
+      [0, 2, 1],
+      [1, 3, 2],
+      [4, 1, 3],
+      [3, 1, 3],
+      [1, 2, 5]
+    ])",
+                                            R"([
+      [2, 2, 12],
+      [5, 3, 12],
+      [1, 3, 12]
+    ])"});
+
+  auto format = std::make_shared<arrow::dataset::ParquetFileFormat>();
+  auto filesystem = std::make_shared<fs::LocalFileSystem>();
+  const std::string file_name = "serde_test.parquet";
+
+  ASSERT_OK_AND_ASSIGN(auto tempdir,
+                       arrow::internal::TemporaryDir::Make("substrait_tempdir"));
+  ASSERT_OK_AND_ASSIGN(auto file_path, tempdir->path().Join(file_name));
+  std::string file_path_str = file_path.ToString();
+
+  // Note: there is an additional forward slash introduced by the tempdir
+  // it must be replaced to properly load into reading files
+  // TODO: (Review: Jira needs to be reported to handle this properly)
+  std::string toReplace("/T//");
+  size_t pos = file_path_str.find(toReplace);
+  file_path_str.replace(pos, toReplace.length(), "/T/");
+
+  ARROW_EXPECT_OK(WriteParquetData(file_path_str, filesystem, table));
+
+  std::vector<fs::FileInfo> files;
+  const std::vector<std::string> f_paths = {file_path_str};
+
+  for (const auto& f_path : f_paths) {
+    ASSERT_OK_AND_ASSIGN(auto f_file, filesystem->GetFileInfo(f_path));
+    files.push_back(std::move(f_file));
+  }
+
+  ASSERT_OK_AND_ASSIGN(auto ds_factory, dataset::FileSystemDatasetFactory::Make(
+                                            filesystem, std::move(files), format, {}));
+  ASSERT_OK_AND_ASSIGN(auto dataset, ds_factory->Finish(dummy_schema));
+
+  auto scan_options = std::make_shared<dataset::ScanOptions>();
+  scan_options->projection = compute::project({}, {});
+  const std::string filter_col = "shared";
+  auto filter = compute::equal(compute::field_ref(filter_col), compute::literal(3));
+
+  arrow::AsyncGenerator<util::optional<compute::ExecBatch>> sink_gen;
+
+  auto scan_node_options = dataset::ScanNodeOptions{dataset, scan_options};
+  auto filter_node_options = compute::FilterNodeOptions{filter};
+  auto sink_node_options = compute::SinkNodeOptions{&sink_gen};
+
+  auto scan_declaration = compute::Declaration({"scan", scan_node_options, "s"});
+  auto filter_declaration = compute::Declaration({"filter", filter_node_options, "f"});
+  auto sink_declaration = compute::Declaration({"sink", sink_node_options, "e"});
+
+  auto declarations = compute::Declaration::Sequence(
+      {scan_declaration, filter_declaration, sink_declaration});
+
+  ASSERT_OK_AND_ASSIGN(auto plan, compute::ExecPlan::Make(&exec_context));
+
+  ASSERT_OK_AND_ASSIGN(auto serialized_plan, SerializePlan(declarations, &ext_set));
+
+  for (auto sp_ext_id_reg :
+       {std::shared_ptr<ExtensionIdRegistry>(), substrait::MakeExtensionIdRegistry()}) {
+    ExtensionIdRegistry* ext_id_reg = sp_ext_id_reg.get();
+    ExtensionSet ext_set(ext_id_reg);
+    ASSERT_OK_AND_ASSIGN(
+        auto sink_decls,
+        DeserializePlans(
+            *serialized_plan, [] { return kNullConsumer; }, ext_id_reg, &ext_set));
+    // filter declaration
+    auto roundtripped_filter = sink_decls[0].inputs[0].get<compute::Declaration>();
+    const auto& filter_opts =
+        checked_cast<const compute::FilterNodeOptions&>(*(roundtripped_filter->options));
+    auto roundtripped_expr = filter_opts.filter_expression;
+
+    if (auto* call = roundtripped_expr.call()) {
+      EXPECT_EQ(call->function_name, "equal");
+      auto args = call->arguments;
+      auto index = args[0].field_ref()->field_path()->indices()[0];
+      EXPECT_EQ(dummy_schema->field_names()[index], filter_col);
+      EXPECT_EQ(args[1], compute::literal(3));
+    }
+    // scan declaration
+    auto roundtripped_scan = roundtripped_filter->inputs[0].get<compute::Declaration>();
+    const auto& dataset_opts =
+        checked_cast<const dataset::ScanNodeOptions&>(*(roundtripped_scan->options));
+    const auto& roundripped_ds = dataset_opts.dataset;
+    EXPECT_TRUE(roundripped_ds->schema()->Equals(*dummy_schema));
+    ASSERT_OK_AND_ASSIGN(auto roundtripped_frgs, roundripped_ds->GetFragments());
+    ASSERT_OK_AND_ASSIGN(auto expected_frgs, dataset->GetFragments());
+
+    auto roundtrip_frg_vec = IteratorToVector(std::move(roundtripped_frgs));
+    auto expected_frg_vec = IteratorToVector(std::move(expected_frgs));
+    EXPECT_EQ(expected_frg_vec.size(), roundtrip_frg_vec.size());
+    int64_t idx = 0;
+    for (auto fragment : expected_frg_vec) {
+      const auto* l_frag = checked_cast<const dataset::FileFragment*>(fragment.get());
+      const auto* r_frag =
+          checked_cast<const dataset::FileFragment*>(roundtrip_frg_vec[idx++].get());
+      EXPECT_TRUE(l_frag->Equals(*r_frag));
+    }
+  }
+#endif
+}
+
+TEST(Substrait, BasicPlanRoundTrippingEndToEnd) {
+#ifdef _WIN32
+  GTEST_SKIP() << "ARROW-16392: Substrait File URI not supported for Windows";
+#else
+  compute::ExecContext exec_context;
+  ExtensionSet ext_set;
+  auto dummy_schema = schema(
+      {field("key", int32()), field("shared", int32()), field("distinct", int32())});
+
+  // creating a dummy dataset using a dummy table
+  auto table = TableFromJSON(dummy_schema, {R"([
+      [1, 1, 10],
+      [3, 4, 20]
+    ])",
+                                            R"([
+      [0, 2, 1],
+      [1, 3, 2],
+      [4, 1, 3],
+      [3, 1, 3],
+      [1, 2, 5]
+    ])",
+                                            R"([
+      [2, 2, 12],
+      [5, 3, 12],
+      [1, 3, 12]
+    ])"});
+
+  auto format = std::make_shared<arrow::dataset::ParquetFileFormat>();
+  auto filesystem = std::make_shared<fs::LocalFileSystem>();
+  const std::string file_name = "serde_test.parquet";
+
+  ASSERT_OK_AND_ASSIGN(auto tempdir,
+                       arrow::internal::TemporaryDir::Make("substrait_tempdir"));
+  ASSERT_OK_AND_ASSIGN(auto file_path, tempdir->path().Join(file_name));
+  std::string file_path_str = file_path.ToNative();
+
+  // Note: there is an additional forward slash introduced by the tempdir
+  // it must be replaced to properly load into reading files
+  // TODO: (Review: Jira needs to be reported to handle this properly)
+  std::string toReplace("/T//");
+  size_t pos = file_path_str.find(toReplace);
+  file_path_str.replace(pos, toReplace.length(), "/T/");
+
+  ARROW_EXPECT_OK(WriteParquetData(file_path_str, filesystem, table));
+
+  std::vector<fs::FileInfo> files;
+  const std::vector<std::string> f_paths = {file_path_str};
+
+  for (const auto& f_path : f_paths) {
+    ASSERT_OK_AND_ASSIGN(auto f_file, filesystem->GetFileInfo(f_path));
+    files.push_back(std::move(f_file));
+  }
+
+  ASSERT_OK_AND_ASSIGN(auto ds_factory, dataset::FileSystemDatasetFactory::Make(
+                                            filesystem, std::move(files), format, {}));
+  ASSERT_OK_AND_ASSIGN(auto dataset, ds_factory->Finish(dummy_schema));
+
+  auto scan_options = std::make_shared<dataset::ScanOptions>();
+  scan_options->projection = compute::project({}, {});
+  const std::string filter_col = "shared";
+  auto filter = compute::equal(compute::field_ref(filter_col), compute::literal(3));
+
+  arrow::AsyncGenerator<util::optional<compute::ExecBatch>> sink_gen;
+
+  auto scan_node_options = dataset::ScanNodeOptions{dataset, scan_options};
+  auto filter_node_options = compute::FilterNodeOptions{filter};
+  auto sink_node_options = compute::SinkNodeOptions{&sink_gen};
+
+  auto scan_declaration = compute::Declaration({"scan", scan_node_options, "s"});
+  auto filter_declaration = compute::Declaration({"filter", filter_node_options, "f"});
+  auto sink_declaration = compute::Declaration({"sink", sink_node_options, "e"});
+
+  auto declarations = compute::Declaration::Sequence(
+      {scan_declaration, filter_declaration, sink_declaration});
+
+  ASSERT_OK_AND_ASSIGN(auto plan, compute::ExecPlan::Make(&exec_context));

Review Comment:
   Instead of declaring here the plan which will be populated with declarations, please move it into GetTableFromPlan and be explicit about finishing the plan's execution there



##########
cpp/src/arrow/engine/substrait/serde_test.cc:
##########
@@ -1731,5 +1770,265 @@ TEST(Substrait, AggregateWithFilter) {
   ASSERT_RAISES(NotImplemented, DeserializePlans(*buf, [] { return kNullConsumer; }));
 }
 
+TEST(Substrait, BasicPlanRoundTripping) {
+#ifdef _WIN32
+  GTEST_SKIP() << "ARROW-16392: Substrait File URI not supported for Windows";
+#else
+  compute::ExecContext exec_context;
+  ExtensionSet ext_set;
+  auto dummy_schema = schema(
+      {field("key", int32()), field("shared", int32()), field("distinct", int32())});
+
+  // creating a dummy dataset using a dummy table
+  auto table = TableFromJSON(dummy_schema, {R"([
+      [1, 1, 10],
+      [3, 4, 20]
+    ])",
+                                            R"([
+      [0, 2, 1],
+      [1, 3, 2],
+      [4, 1, 3],
+      [3, 1, 3],
+      [1, 2, 5]
+    ])",
+                                            R"([
+      [2, 2, 12],
+      [5, 3, 12],
+      [1, 3, 12]
+    ])"});
+
+  auto format = std::make_shared<arrow::dataset::ParquetFileFormat>();
+  auto filesystem = std::make_shared<fs::LocalFileSystem>();
+  const std::string file_name = "serde_test.parquet";
+
+  ASSERT_OK_AND_ASSIGN(auto tempdir,
+                       arrow::internal::TemporaryDir::Make("substrait_tempdir"));
+  ASSERT_OK_AND_ASSIGN(auto file_path, tempdir->path().Join(file_name));
+  std::string file_path_str = file_path.ToString();
+
+  // Note: there is an additional forward slash introduced by the tempdir
+  // it must be replaced to properly load into reading files
+  // TODO: (Review: Jira needs to be reported to handle this properly)
+  std::string toReplace("/T//");
+  size_t pos = file_path_str.find(toReplace);
+  file_path_str.replace(pos, toReplace.length(), "/T/");
+
+  ARROW_EXPECT_OK(WriteParquetData(file_path_str, filesystem, table));
+
+  std::vector<fs::FileInfo> files;
+  const std::vector<std::string> f_paths = {file_path_str};
+
+  for (const auto& f_path : f_paths) {
+    ASSERT_OK_AND_ASSIGN(auto f_file, filesystem->GetFileInfo(f_path));
+    files.push_back(std::move(f_file));
+  }
+
+  ASSERT_OK_AND_ASSIGN(auto ds_factory, dataset::FileSystemDatasetFactory::Make(
+                                            filesystem, std::move(files), format, {}));
+  ASSERT_OK_AND_ASSIGN(auto dataset, ds_factory->Finish(dummy_schema));
+
+  auto scan_options = std::make_shared<dataset::ScanOptions>();
+  scan_options->projection = compute::project({}, {});
+  const std::string filter_col = "shared";
+  auto filter = compute::equal(compute::field_ref(filter_col), compute::literal(3));
+
+  arrow::AsyncGenerator<util::optional<compute::ExecBatch>> sink_gen;
+
+  auto scan_node_options = dataset::ScanNodeOptions{dataset, scan_options};
+  auto filter_node_options = compute::FilterNodeOptions{filter};
+  auto sink_node_options = compute::SinkNodeOptions{&sink_gen};
+
+  auto scan_declaration = compute::Declaration({"scan", scan_node_options, "s"});
+  auto filter_declaration = compute::Declaration({"filter", filter_node_options, "f"});
+  auto sink_declaration = compute::Declaration({"sink", sink_node_options, "e"});
+
+  auto declarations = compute::Declaration::Sequence(
+      {scan_declaration, filter_declaration, sink_declaration});
+
+  ASSERT_OK_AND_ASSIGN(auto plan, compute::ExecPlan::Make(&exec_context));
+
+  ASSERT_OK_AND_ASSIGN(auto serialized_plan, SerializePlan(declarations, &ext_set));
+
+  for (auto sp_ext_id_reg :
+       {std::shared_ptr<ExtensionIdRegistry>(), substrait::MakeExtensionIdRegistry()}) {
+    ExtensionIdRegistry* ext_id_reg = sp_ext_id_reg.get();
+    ExtensionSet ext_set(ext_id_reg);
+    ASSERT_OK_AND_ASSIGN(
+        auto sink_decls,
+        DeserializePlans(
+            *serialized_plan, [] { return kNullConsumer; }, ext_id_reg, &ext_set));
+    // filter declaration
+    auto roundtripped_filter = sink_decls[0].inputs[0].get<compute::Declaration>();
+    const auto& filter_opts =
+        checked_cast<const compute::FilterNodeOptions&>(*(roundtripped_filter->options));
+    auto roundtripped_expr = filter_opts.filter_expression;
+
+    if (auto* call = roundtripped_expr.call()) {
+      EXPECT_EQ(call->function_name, "equal");
+      auto args = call->arguments;
+      auto index = args[0].field_ref()->field_path()->indices()[0];
+      EXPECT_EQ(dummy_schema->field_names()[index], filter_col);
+      EXPECT_EQ(args[1], compute::literal(3));
+    }
+    // scan declaration
+    auto roundtripped_scan = roundtripped_filter->inputs[0].get<compute::Declaration>();
+    const auto& dataset_opts =
+        checked_cast<const dataset::ScanNodeOptions&>(*(roundtripped_scan->options));
+    const auto& roundripped_ds = dataset_opts.dataset;
+    EXPECT_TRUE(roundripped_ds->schema()->Equals(*dummy_schema));
+    ASSERT_OK_AND_ASSIGN(auto roundtripped_frgs, roundripped_ds->GetFragments());
+    ASSERT_OK_AND_ASSIGN(auto expected_frgs, dataset->GetFragments());
+
+    auto roundtrip_frg_vec = IteratorToVector(std::move(roundtripped_frgs));
+    auto expected_frg_vec = IteratorToVector(std::move(expected_frgs));
+    EXPECT_EQ(expected_frg_vec.size(), roundtrip_frg_vec.size());
+    int64_t idx = 0;
+    for (auto fragment : expected_frg_vec) {
+      const auto* l_frag = checked_cast<const dataset::FileFragment*>(fragment.get());
+      const auto* r_frag =
+          checked_cast<const dataset::FileFragment*>(roundtrip_frg_vec[idx++].get());
+      EXPECT_TRUE(l_frag->Equals(*r_frag));
+    }
+  }
+#endif
+}
+
+TEST(Substrait, BasicPlanRoundTrippingEndToEnd) {
+#ifdef _WIN32
+  GTEST_SKIP() << "ARROW-16392: Substrait File URI not supported for Windows";
+#else

Review Comment:
   ```suggestion
   #endif
   ```
   Even if the file uri is not supported on windows, it's still worthwhile to ensure the below code compiles on all platforms



-- 
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] westonpace commented on a diff in pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
westonpace commented on code in PR #13401:
URL: https://github.com/apache/arrow/pull/13401#discussion_r956249655


##########
cpp/src/arrow/engine/substrait/plan_internal.cc:
##########
@@ -133,5 +136,18 @@ Result<ExtensionSet> GetExtensionSetFromPlan(const substrait::Plan& plan,
                             registry);
 }
 
+Result<std::unique_ptr<substrait::Plan>> PlanToProto(

Review Comment:
   I like the name `PlanToProto` better but, for consistency, I think this should be named `ToProto` right?



##########
cpp/src/arrow/engine/substrait/plan_internal.h:
##########
@@ -51,5 +53,17 @@ Result<ExtensionSet> GetExtensionSetFromPlan(
     const substrait::Plan& plan,
     const ExtensionIdRegistry* registry = default_extension_id_registry());
 
+/// \brief Serializes Declaration and produces a substrait::Plan.
+///
+/// Note that, this is a part of roundtripping test API and not
+/// designed to use in production
+/// \param[in] declr the sequence of declarations
+/// \param[in, out] ext_set the extension set to be updated
+/// \param[in] conversion_options the conversion options useful for the serialization

Review Comment:
   ```suggestion
   /// \param[in] conversion_options options to control serialization behavior
   ```



##########
cpp/src/arrow/engine/substrait/plan_internal.h:
##########
@@ -51,5 +53,17 @@ Result<ExtensionSet> GetExtensionSetFromPlan(
     const substrait::Plan& plan,
     const ExtensionIdRegistry* registry = default_extension_id_registry());
 
+/// \brief Serializes Declaration and produces a substrait::Plan.
+///
+/// Note that, this is a part of roundtripping test API and not
+/// designed to use in production
+/// \param[in] declr the sequence of declarations

Review Comment:
   ```suggestion
   /// \param[in] declr the sequence of declarations to be serialized
   ```



##########
cpp/src/arrow/engine/substrait/relation_internal.cc:
##########
@@ -421,5 +431,171 @@ Result<DeclarationInfo> FromProto(const substrait::Rel& rel, const ExtensionSet&
       rel.DebugString());
 }
 
+Result<std::unique_ptr<substrait::Rel>> ToProto(
+    const compute::Declaration& declr, ExtensionSet* ext_set,
+    const ConversionOptions& conversion_options) {
+  auto rel = make_unique<substrait::Rel>();
+  RETURN_NOT_OK(SerializeAndCombineRelations(declr, ext_set, rel, conversion_options));
+  return std::move(rel);
+}
+
+Status SetRelation(const std::unique_ptr<substrait::Rel>& plan,
+                   const std::unique_ptr<substrait::Rel>& partial_plan,
+                   const std::string& factory_name) {
+  if (factory_name == "scan" && partial_plan->has_read()) {
+    plan->set_allocated_read(partial_plan->release_read());
+  } else if (factory_name == "filter" && partial_plan->has_filter()) {
+    plan->set_allocated_filter(partial_plan->release_filter());
+  } else {
+    return Status::NotImplemented("Substrait converter ", factory_name,
+                                  " not supported.");
+  }
+  return Status::OK();
+}
+
+Result<std::shared_ptr<Schema>> ExtractSchemaToBind(const compute::Declaration& declr) {
+  std::shared_ptr<Schema> bind_schema;
+  if (declr.factory_name == "scan") {
+    const auto& opts = checked_cast<const dataset::ScanNodeOptions&>(*(declr.options));
+    bind_schema = opts.dataset->schema();
+  } else if (declr.factory_name == "filter") {
+    auto input_declr = util::get<compute::Declaration>(declr.inputs[0]);
+    ARROW_ASSIGN_OR_RAISE(bind_schema, ExtractSchemaToBind(input_declr));
+  } else if (declr.factory_name == "sink") {
+    // Note that the sink has no output_schema
+    return bind_schema;
+  } else {
+    return Status::Invalid("Schema extraction failed, unsupported factory ",
+                           declr.factory_name);
+  }
+  return bind_schema;
+}
+
+Status SerializeAndCombineRelations(const compute::Declaration& declaration,
+                                    ExtensionSet* ext_set,
+                                    std::unique_ptr<substrait::Rel>& rel,
+                                    const ConversionOptions& conversion_options) {
+  std::vector<compute::Declaration::Input> inputs = declaration.inputs;
+  for (auto& input : inputs) {
+    auto input_decl = util::get<compute::Declaration>(input);
+    RETURN_NOT_OK(
+        SerializeAndCombineRelations(input_decl, ext_set, rel, conversion_options));
+  }
+  const auto& factory_name = declaration.factory_name;
+  ARROW_ASSIGN_OR_RAISE(auto schema, ExtractSchemaToBind(declaration));
+  // Note that the sink declaration factory doesn't exist for serialization as
+  // Substrait doesn't deal with a sink node definition
+  std::unique_ptr<substrait::Rel> factory_rel;
+  if (factory_name == "scan") {
+    ARROW_ASSIGN_OR_RAISE(factory_rel, ScanRelationConverter(schema, declaration, ext_set,
+                                                             conversion_options));
+  } else if (factory_name == "filter") {
+    ARROW_ASSIGN_OR_RAISE(
+        factory_rel,
+        FilterRelationConverter(schema, declaration, ext_set, conversion_options));

Review Comment:
   It seems like it would be slightly clearer to just do:
   
   ```
   rel->set_allocated_filter(factory_rel.release());
   ```
   
   Then you could get rid of `SetRelation` and skip a long if/else loop.



##########
cpp/src/arrow/engine/substrait/relation_internal.cc:
##########
@@ -421,5 +431,171 @@ Result<DeclarationInfo> FromProto(const substrait::Rel& rel, const ExtensionSet&
       rel.DebugString());
 }
 
+Result<std::unique_ptr<substrait::Rel>> ToProto(
+    const compute::Declaration& declr, ExtensionSet* ext_set,
+    const ConversionOptions& conversion_options) {
+  auto rel = make_unique<substrait::Rel>();
+  RETURN_NOT_OK(SerializeAndCombineRelations(declr, ext_set, rel, conversion_options));
+  return std::move(rel);
+}
+
+Status SetRelation(const std::unique_ptr<substrait::Rel>& plan,
+                   const std::unique_ptr<substrait::Rel>& partial_plan,
+                   const std::string& factory_name) {
+  if (factory_name == "scan" && partial_plan->has_read()) {
+    plan->set_allocated_read(partial_plan->release_read());
+  } else if (factory_name == "filter" && partial_plan->has_filter()) {
+    plan->set_allocated_filter(partial_plan->release_filter());
+  } else {
+    return Status::NotImplemented("Substrait converter ", factory_name,
+                                  " not supported.");
+  }
+  return Status::OK();
+}
+
+Result<std::shared_ptr<Schema>> ExtractSchemaToBind(const compute::Declaration& declr) {
+  std::shared_ptr<Schema> bind_schema;
+  if (declr.factory_name == "scan") {
+    const auto& opts = checked_cast<const dataset::ScanNodeOptions&>(*(declr.options));
+    bind_schema = opts.dataset->schema();
+  } else if (declr.factory_name == "filter") {
+    auto input_declr = util::get<compute::Declaration>(declr.inputs[0]);
+    ARROW_ASSIGN_OR_RAISE(bind_schema, ExtractSchemaToBind(input_declr));
+  } else if (declr.factory_name == "sink") {
+    // Note that the sink has no output_schema
+    return bind_schema;
+  } else {
+    return Status::Invalid("Schema extraction failed, unsupported factory ",
+                           declr.factory_name);
+  }
+  return bind_schema;
+}
+
+Status SerializeAndCombineRelations(const compute::Declaration& declaration,
+                                    ExtensionSet* ext_set,
+                                    std::unique_ptr<substrait::Rel>& rel,
+                                    const ConversionOptions& conversion_options) {
+  std::vector<compute::Declaration::Input> inputs = declaration.inputs;
+  for (auto& input : inputs) {
+    auto input_decl = util::get<compute::Declaration>(input);
+    RETURN_NOT_OK(
+        SerializeAndCombineRelations(input_decl, ext_set, rel, conversion_options));

Review Comment:
   I'm not sure this is valid.  Usually you'd want to do something like...
   
   ```
   auto input_rel = internal::make_unique<substrait::Rel>();
   RETURN_NOT_OK(
       SerializeAndCombineRelations(input_decl, ext_set, input_rel, conversion_options));
   rel->set_allocated_input(input_rel.release());
   ```
   
   Sadly, there is no guarantee that `rel` has `input` (e.g. `JoinRel` has `left` and `right`) so I think this input handling will need to be inside the below if/else loop.
   
   Right now I think you are setting the `oneof` inside of `rel` multiple times.



##########
cpp/src/arrow/engine/substrait/serde.h:
##########
@@ -202,6 +207,18 @@ Result<std::shared_ptr<Buffer>> SerializeExpression(
     const compute::Expression& expr, ExtensionSet* ext_set,
     const ConversionOptions& conversion_options = {});
 
+/// \brief Serializes an Arrow compute Declaration to a Substrait Relation message
+///
+/// \param[in] declaration the Arrow compute declaration to serialize
+/// \param[in,out] ext_set the extension mapping to use; may be updated to add

Review Comment:
   I'm not sure what `; may be updated to add` means.



##########
cpp/src/arrow/engine/substrait/relation_internal.h:
##########
@@ -40,9 +40,46 @@ struct DeclarationInfo {
   int num_columns;
 };
 
+/// \brief A function to extract Acero Declaration from a Substrait Rel object
 ARROW_ENGINE_EXPORT
 Result<DeclarationInfo> FromProto(const substrait::Rel&, const ExtensionSet&,
                                   const ConversionOptions&);
 
+/// \brief Serializes a Declaration, produce a Substrait Rel and update the global
+/// Substrait plan. A Substrait Rel is passed as a the plan and it is updated with
+/// corresponding Declaration passed for serialization.
+///
+/// Note that this is a rather a helper method useful to fuse a partially serialized
+/// plan with another plan. The reason for having a partially serialized plan is to
+/// avoid unnecessary complication and enable partial plan serialization without
+/// affecting a global plan. Since kept as unique_ptr resources are relased efficiently
+/// upon releasing for the global plan.
+ARROW_ENGINE_EXPORT Status SerializeAndCombineRelations(const compute::Declaration&,
+                                                        ExtensionSet*,
+                                                        std::unique_ptr<substrait::Rel>&,
+                                                        const ConversionOptions&);
+
+/// \brief Serialize a Declaration and produces a Substrait Rel.
+///
+/// Note that in order to provide a generic interface for ToProto for
+/// declaration it is not specialized for each relation within the Substrait Rel.
+/// Rather a serialized relation is set as a member for the Substrait Rel
+/// (partial Relation) which is later on extracted to update a Substrait Rel
+/// which would be included in the fully serialized Acero Exec Plan.
+/// The ExecNode or ExecPlan is not used in this context as Declaration is preferred
+/// in the Substrait space rather than internal components of Acero execution engine.
+ARROW_ENGINE_EXPORT Result<std::unique_ptr<substrait::Rel>> ToProto(
+    const compute::Declaration&, ExtensionSet*, const ConversionOptions&);
+
+/// \brief Acero to Substrait converter for Acero scan relation.
+ARROW_ENGINE_EXPORT Result<std::unique_ptr<substrait::Rel>> ScanRelationConverter(
+    const std::shared_ptr<Schema>&, const compute::Declaration&, ExtensionSet*,
+    const ConversionOptions&);
+
+/// \brief Acero to Substrait converter for Acero filter relation.
+ARROW_ENGINE_EXPORT Result<std::unique_ptr<substrait::Rel>> FilterRelationConverter(
+    const std::shared_ptr<Schema>&, const compute::Declaration&, ExtensionSet*,
+    const ConversionOptions&);

Review Comment:
   These methods should not be in the `.h` file and should be in an anonymous namespace in the `.cc` file.



##########
cpp/src/arrow/engine/substrait/plan_internal.h:
##########
@@ -51,5 +53,17 @@ Result<ExtensionSet> GetExtensionSetFromPlan(
     const substrait::Plan& plan,
     const ExtensionIdRegistry* registry = default_extension_id_registry());
 
+/// \brief Serializes Declaration and produces a substrait::Plan.

Review Comment:
   ```suggestion
   /// \brief Serialize a declaration and into a substrait::Plan.
   ```



##########
cpp/src/arrow/engine/substrait/serde_test.cc:
##########
@@ -42,9 +55,35 @@ using testing::UnorderedElementsAre;
 namespace arrow {
 
 using internal::checked_cast;
-
+using internal::hash_combine;
 namespace engine {
 
+Status WriteParquetData(const std::string& path,
+                        const std::shared_ptr<fs::FileSystem> file_system,
+                        const std::shared_ptr<Table> input) {
+  EXPECT_OK_AND_ASSIGN(auto buffer_writer, file_system->OpenOutputStream(path));
+  PARQUET_THROW_NOT_OK(parquet::arrow::WriteTable(*input, arrow::default_memory_pool(),
+                                                  buffer_writer, /*chunk_size*/ 1));
+  return buffer_writer->Close();
+}

Review Comment:
   Substrait supports IPC now.  Can we write this test with IPC?



##########
cpp/src/arrow/engine/substrait/plan_internal.h:
##########
@@ -51,5 +53,17 @@ Result<ExtensionSet> GetExtensionSetFromPlan(
     const substrait::Plan& plan,
     const ExtensionIdRegistry* registry = default_extension_id_registry());
 
+/// \brief Serializes Declaration and produces a substrait::Plan.
+///
+/// Note that, this is a part of roundtripping test API and not
+/// designed to use in production
+/// \param[in] declr the sequence of declarations
+/// \param[in, out] ext_set the extension set to be updated
+/// \param[in] conversion_options the conversion options useful for the serialization
+/// \return serialized Acero plan

Review Comment:
   ```suggestion
   /// \return the serialized plan
   ```



##########
cpp/src/arrow/engine/substrait/plan_internal.cc:
##########
@@ -133,5 +136,18 @@ Result<ExtensionSet> GetExtensionSetFromPlan(const substrait::Plan& plan,
                             registry);
 }
 
+Result<std::unique_ptr<substrait::Plan>> PlanToProto(
+    const compute::Declaration& declr, ExtensionSet* ext_set,
+    const ConversionOptions& conversion_options) {
+  auto subs_plan = internal::make_unique<substrait::Plan>();
+  auto plan_rel = internal::make_unique<substrait::PlanRel>();
+  auto rel = internal::make_unique<substrait::Rel>();
+  RETURN_NOT_OK(SerializeAndCombineRelations(declr, ext_set, rel, conversion_options));

Review Comment:
   Why not use the `ToProto` method you just added to `relation_internal.cc`?



##########
cpp/src/arrow/engine/substrait/plan_internal.cc:
##########
@@ -133,5 +136,18 @@ Result<ExtensionSet> GetExtensionSetFromPlan(const substrait::Plan& plan,
                             registry);
 }
 
+Result<std::unique_ptr<substrait::Plan>> PlanToProto(
+    const compute::Declaration& declr, ExtensionSet* ext_set,
+    const ConversionOptions& conversion_options) {
+  auto subs_plan = internal::make_unique<substrait::Plan>();
+  auto plan_rel = internal::make_unique<substrait::PlanRel>();
+  auto rel = internal::make_unique<substrait::Rel>();
+  RETURN_NOT_OK(SerializeAndCombineRelations(declr, ext_set, rel, conversion_options));
+  plan_rel->set_allocated_rel(rel.release());

Review Comment:
   This should be `set_allocated_root` I think.



##########
cpp/src/arrow/engine/substrait/plan_internal.cc:
##########
@@ -17,6 +17,9 @@
 
 #include "arrow/engine/substrait/plan_internal.h"
 
+#include "arrow/dataset/plan.h"
+#include "arrow/dataset/scanner.h"

Review Comment:
   ```suggestion
   ```



##########
cpp/src/arrow/engine/substrait/relation_internal.cc:
##########
@@ -421,5 +431,171 @@ Result<DeclarationInfo> FromProto(const substrait::Rel& rel, const ExtensionSet&
       rel.DebugString());
 }
 
+Result<std::unique_ptr<substrait::Rel>> ToProto(
+    const compute::Declaration& declr, ExtensionSet* ext_set,
+    const ConversionOptions& conversion_options) {
+  auto rel = make_unique<substrait::Rel>();
+  RETURN_NOT_OK(SerializeAndCombineRelations(declr, ext_set, rel, conversion_options));
+  return std::move(rel);
+}
+
+Status SetRelation(const std::unique_ptr<substrait::Rel>& plan,
+                   const std::unique_ptr<substrait::Rel>& partial_plan,
+                   const std::string& factory_name) {
+  if (factory_name == "scan" && partial_plan->has_read()) {
+    plan->set_allocated_read(partial_plan->release_read());
+  } else if (factory_name == "filter" && partial_plan->has_filter()) {
+    plan->set_allocated_filter(partial_plan->release_filter());
+  } else {
+    return Status::NotImplemented("Substrait converter ", factory_name,
+                                  " not supported.");
+  }
+  return Status::OK();
+}
+
+Result<std::shared_ptr<Schema>> ExtractSchemaToBind(const compute::Declaration& declr) {
+  std::shared_ptr<Schema> bind_schema;
+  if (declr.factory_name == "scan") {
+    const auto& opts = checked_cast<const dataset::ScanNodeOptions&>(*(declr.options));
+    bind_schema = opts.dataset->schema();
+  } else if (declr.factory_name == "filter") {
+    auto input_declr = util::get<compute::Declaration>(declr.inputs[0]);
+    ARROW_ASSIGN_OR_RAISE(bind_schema, ExtractSchemaToBind(input_declr));
+  } else if (declr.factory_name == "sink") {
+    // Note that the sink has no output_schema
+    return bind_schema;
+  } else {
+    return Status::Invalid("Schema extraction failed, unsupported factory ",
+                           declr.factory_name);
+  }
+  return bind_schema;
+}
+
+Status SerializeAndCombineRelations(const compute::Declaration& declaration,
+                                    ExtensionSet* ext_set,
+                                    std::unique_ptr<substrait::Rel>& rel,
+                                    const ConversionOptions& conversion_options) {
+  std::vector<compute::Declaration::Input> inputs = declaration.inputs;
+  for (auto& input : inputs) {
+    auto input_decl = util::get<compute::Declaration>(input);
+    RETURN_NOT_OK(
+        SerializeAndCombineRelations(input_decl, ext_set, rel, conversion_options));
+  }
+  const auto& factory_name = declaration.factory_name;
+  ARROW_ASSIGN_OR_RAISE(auto schema, ExtractSchemaToBind(declaration));
+  // Note that the sink declaration factory doesn't exist for serialization as
+  // Substrait doesn't deal with a sink node definition
+  std::unique_ptr<substrait::Rel> factory_rel;
+  if (factory_name == "scan") {
+    ARROW_ASSIGN_OR_RAISE(factory_rel, ScanRelationConverter(schema, declaration, ext_set,
+                                                             conversion_options));
+  } else if (factory_name == "filter") {
+    ARROW_ASSIGN_OR_RAISE(
+        factory_rel,
+        FilterRelationConverter(schema, declaration, ext_set, conversion_options));
+  } else {
+    return Status::NotImplemented("Factory ", factory_name,
+                                  " not implemented for roundtripping.");
+  }
+
+  if (factory_rel != nullptr) {
+    RETURN_NOT_OK(SetRelation(rel, factory_rel, factory_name));
+  } else {
+    return Status::Invalid("Conversion on factory ", factory_name,
+                           " returned an invalid relation");
+  }

Review Comment:
   Why would `SetRelation` return `nullptr` instead of just returning an invalid status?



##########
cpp/src/arrow/engine/substrait/relation_internal.cc:
##########
@@ -421,5 +431,171 @@ Result<DeclarationInfo> FromProto(const substrait::Rel& rel, const ExtensionSet&
       rel.DebugString());
 }
 
+Result<std::unique_ptr<substrait::Rel>> ToProto(
+    const compute::Declaration& declr, ExtensionSet* ext_set,
+    const ConversionOptions& conversion_options) {
+  auto rel = make_unique<substrait::Rel>();
+  RETURN_NOT_OK(SerializeAndCombineRelations(declr, ext_set, rel, conversion_options));
+  return std::move(rel);
+}
+
+Status SetRelation(const std::unique_ptr<substrait::Rel>& plan,
+                   const std::unique_ptr<substrait::Rel>& partial_plan,
+                   const std::string& factory_name) {
+  if (factory_name == "scan" && partial_plan->has_read()) {
+    plan->set_allocated_read(partial_plan->release_read());
+  } else if (factory_name == "filter" && partial_plan->has_filter()) {
+    plan->set_allocated_filter(partial_plan->release_filter());
+  } else {
+    return Status::NotImplemented("Substrait converter ", factory_name,
+                                  " not supported.");
+  }
+  return Status::OK();
+}
+
+Result<std::shared_ptr<Schema>> ExtractSchemaToBind(const compute::Declaration& declr) {
+  std::shared_ptr<Schema> bind_schema;
+  if (declr.factory_name == "scan") {
+    const auto& opts = checked_cast<const dataset::ScanNodeOptions&>(*(declr.options));
+    bind_schema = opts.dataset->schema();
+  } else if (declr.factory_name == "filter") {
+    auto input_declr = util::get<compute::Declaration>(declr.inputs[0]);
+    ARROW_ASSIGN_OR_RAISE(bind_schema, ExtractSchemaToBind(input_declr));
+  } else if (declr.factory_name == "sink") {
+    // Note that the sink has no output_schema
+    return bind_schema;
+  } else {
+    return Status::Invalid("Schema extraction failed, unsupported factory ",
+                           declr.factory_name);
+  }
+  return bind_schema;
+}
+
+Status SerializeAndCombineRelations(const compute::Declaration& declaration,
+                                    ExtensionSet* ext_set,
+                                    std::unique_ptr<substrait::Rel>& rel,
+                                    const ConversionOptions& conversion_options) {
+  std::vector<compute::Declaration::Input> inputs = declaration.inputs;
+  for (auto& input : inputs) {
+    auto input_decl = util::get<compute::Declaration>(input);
+    RETURN_NOT_OK(
+        SerializeAndCombineRelations(input_decl, ext_set, rel, conversion_options));
+  }
+  const auto& factory_name = declaration.factory_name;
+  ARROW_ASSIGN_OR_RAISE(auto schema, ExtractSchemaToBind(declaration));
+  // Note that the sink declaration factory doesn't exist for serialization as
+  // Substrait doesn't deal with a sink node definition
+  std::unique_ptr<substrait::Rel> factory_rel;
+  if (factory_name == "scan") {
+    ARROW_ASSIGN_OR_RAISE(factory_rel, ScanRelationConverter(schema, declaration, ext_set,
+                                                             conversion_options));
+  } else if (factory_name == "filter") {
+    ARROW_ASSIGN_OR_RAISE(
+        factory_rel,
+        FilterRelationConverter(schema, declaration, ext_set, conversion_options));
+  } else {
+    return Status::NotImplemented("Factory ", factory_name,
+                                  " not implemented for roundtripping.");
+  }
+
+  if (factory_rel != nullptr) {
+    RETURN_NOT_OK(SetRelation(rel, factory_rel, factory_name));
+  } else {
+    return Status::Invalid("Conversion on factory ", factory_name,
+                           " returned an invalid relation");
+  }
+  return Status::OK();
+}
+
+Result<std::unique_ptr<substrait::Rel>> GetRelationFromDeclaration(
+    const compute::Declaration& declaration, ExtensionSet* ext_set,
+    const ConversionOptions& conversion_options) {
+  auto declr_input = declaration.inputs[0];
+  // Note that the input is expected in declaration.
+  // ExecNode inputs are not accepted
+  if (util::get_if<compute::ExecNode*>(&declr_input)) {
+    return Status::NotImplemented("Only support Plans written in Declaration format.");
+  }
+  return ToProto(util::get<compute::Declaration>(declr_input), ext_set,
+                 conversion_options);
+}
+
+Result<std::unique_ptr<substrait::Rel>> ScanRelationConverter(
+    const std::shared_ptr<Schema>& schema, const compute::Declaration& declaration,
+    ExtensionSet* ext_set, const ConversionOptions& conversion_options) {
+  auto rel = make_unique<substrait::Rel>();
+  auto read_rel = make_unique<substrait::ReadRel>();
+  const auto& scan_node_options =
+      checked_cast<const dataset::ScanNodeOptions&>(*declaration.options);
+  auto dataset =
+      dynamic_cast<dataset::FileSystemDataset*>(scan_node_options.dataset.get());
+  if (dataset == nullptr) {
+    return Status::Invalid("Can only convert file system datasets to a Substrait plan.");

Review Comment:
   ```suggestion
       return Status::Invalid("Can only convert scan node with FileSystemDataset to a Substrait plan.");
   ```



##########
cpp/src/arrow/engine/substrait/relation_internal.h:
##########
@@ -40,9 +40,46 @@ struct DeclarationInfo {
   int num_columns;
 };
 
+/// \brief A function to extract Acero Declaration from a Substrait Rel object
 ARROW_ENGINE_EXPORT
 Result<DeclarationInfo> FromProto(const substrait::Rel&, const ExtensionSet&,
                                   const ConversionOptions&);
 
+/// \brief Serializes a Declaration, produce a Substrait Rel and update the global
+/// Substrait plan. A Substrait Rel is passed as a the plan and it is updated with
+/// corresponding Declaration passed for serialization.
+///
+/// Note that this is a rather a helper method useful to fuse a partially serialized
+/// plan with another plan. The reason for having a partially serialized plan is to
+/// avoid unnecessary complication and enable partial plan serialization without
+/// affecting a global plan. Since kept as unique_ptr resources are relased efficiently
+/// upon releasing for the global plan.
+ARROW_ENGINE_EXPORT Status SerializeAndCombineRelations(const compute::Declaration&,
+                                                        ExtensionSet*,
+                                                        std::unique_ptr<substrait::Rel>&,
+                                                        const ConversionOptions&);
+
+/// \brief Serialize a Declaration and produces a Substrait Rel.

Review Comment:
   ```suggestion
   /// \brief Convert an Acero Declaration to a Substrait Rel
   ```



##########
cpp/src/arrow/engine/substrait/relation_internal.cc:
##########
@@ -421,5 +431,171 @@ Result<DeclarationInfo> FromProto(const substrait::Rel& rel, const ExtensionSet&
       rel.DebugString());
 }
 
+Result<std::unique_ptr<substrait::Rel>> ToProto(
+    const compute::Declaration& declr, ExtensionSet* ext_set,
+    const ConversionOptions& conversion_options) {
+  auto rel = make_unique<substrait::Rel>();
+  RETURN_NOT_OK(SerializeAndCombineRelations(declr, ext_set, rel, conversion_options));
+  return std::move(rel);
+}
+
+Status SetRelation(const std::unique_ptr<substrait::Rel>& plan,
+                   const std::unique_ptr<substrait::Rel>& partial_plan,
+                   const std::string& factory_name) {
+  if (factory_name == "scan" && partial_plan->has_read()) {
+    plan->set_allocated_read(partial_plan->release_read());
+  } else if (factory_name == "filter" && partial_plan->has_filter()) {
+    plan->set_allocated_filter(partial_plan->release_filter());
+  } else {
+    return Status::NotImplemented("Substrait converter ", factory_name,
+                                  " not supported.");
+  }
+  return Status::OK();
+}
+
+Result<std::shared_ptr<Schema>> ExtractSchemaToBind(const compute::Declaration& declr) {
+  std::shared_ptr<Schema> bind_schema;
+  if (declr.factory_name == "scan") {
+    const auto& opts = checked_cast<const dataset::ScanNodeOptions&>(*(declr.options));
+    bind_schema = opts.dataset->schema();
+  } else if (declr.factory_name == "filter") {
+    auto input_declr = util::get<compute::Declaration>(declr.inputs[0]);
+    ARROW_ASSIGN_OR_RAISE(bind_schema, ExtractSchemaToBind(input_declr));
+  } else if (declr.factory_name == "sink") {
+    // Note that the sink has no output_schema
+    return bind_schema;
+  } else {
+    return Status::Invalid("Schema extraction failed, unsupported factory ",
+                           declr.factory_name);
+  }
+  return bind_schema;
+}
+
+Status SerializeAndCombineRelations(const compute::Declaration& declaration,
+                                    ExtensionSet* ext_set,
+                                    std::unique_ptr<substrait::Rel>& rel,
+                                    const ConversionOptions& conversion_options) {
+  std::vector<compute::Declaration::Input> inputs = declaration.inputs;
+  for (auto& input : inputs) {
+    auto input_decl = util::get<compute::Declaration>(input);
+    RETURN_NOT_OK(
+        SerializeAndCombineRelations(input_decl, ext_set, rel, conversion_options));
+  }
+  const auto& factory_name = declaration.factory_name;
+  ARROW_ASSIGN_OR_RAISE(auto schema, ExtractSchemaToBind(declaration));
+  // Note that the sink declaration factory doesn't exist for serialization as
+  // Substrait doesn't deal with a sink node definition
+  std::unique_ptr<substrait::Rel> factory_rel;
+  if (factory_name == "scan") {
+    ARROW_ASSIGN_OR_RAISE(factory_rel, ScanRelationConverter(schema, declaration, ext_set,
+                                                             conversion_options));
+  } else if (factory_name == "filter") {
+    ARROW_ASSIGN_OR_RAISE(
+        factory_rel,
+        FilterRelationConverter(schema, declaration, ext_set, conversion_options));
+  } else {
+    return Status::NotImplemented("Factory ", factory_name,
+                                  " not implemented for roundtripping.");
+  }
+
+  if (factory_rel != nullptr) {
+    RETURN_NOT_OK(SetRelation(rel, factory_rel, factory_name));
+  } else {
+    return Status::Invalid("Conversion on factory ", factory_name,
+                           " returned an invalid relation");
+  }
+  return Status::OK();
+}
+
+Result<std::unique_ptr<substrait::Rel>> GetRelationFromDeclaration(
+    const compute::Declaration& declaration, ExtensionSet* ext_set,
+    const ConversionOptions& conversion_options) {
+  auto declr_input = declaration.inputs[0];
+  // Note that the input is expected in declaration.
+  // ExecNode inputs are not accepted
+  if (util::get_if<compute::ExecNode*>(&declr_input)) {
+    return Status::NotImplemented("Only support Plans written in Declaration format.");
+  }
+  return ToProto(util::get<compute::Declaration>(declr_input), ext_set,
+                 conversion_options);
+}

Review Comment:
   What is this method doing?  I'm not sure I agree that `declaration.inputs[0]` is always safe.



##########
cpp/src/arrow/engine/substrait/serde.h:
##########
@@ -36,6 +36,11 @@
 namespace arrow {
 namespace engine {
 
+ARROW_ENGINE_EXPORT
+Result<std::shared_ptr<Buffer>> SerializePlan(
+    const compute::Declaration& declr, ExtensionSet* ext_set,
+    const ConversionOptions& conversion_options = {});
+

Review Comment:
   Needs a doc comment



##########
cpp/src/arrow/engine/substrait/relation_internal.h:
##########
@@ -40,9 +40,46 @@ struct DeclarationInfo {
   int num_columns;
 };
 
+/// \brief A function to extract Acero Declaration from a Substrait Rel object
 ARROW_ENGINE_EXPORT
 Result<DeclarationInfo> FromProto(const substrait::Rel&, const ExtensionSet&,
                                   const ConversionOptions&);
 
+/// \brief Serializes a Declaration, produce a Substrait Rel and update the global
+/// Substrait plan. A Substrait Rel is passed as a the plan and it is updated with
+/// corresponding Declaration passed for serialization.
+///
+/// Note that this is a rather a helper method useful to fuse a partially serialized
+/// plan with another plan. The reason for having a partially serialized plan is to
+/// avoid unnecessary complication and enable partial plan serialization without
+/// affecting a global plan. Since kept as unique_ptr resources are relased efficiently
+/// upon releasing for the global plan.

Review Comment:
   ```suggestion
   /// \brief Convert a Declaration (and its inputs) to a Substrait Rel
   ///
   /// A Substrait Rel is passed as a the plan and it is updated with
   /// corresponding Declaration passed for serialization.
   ///
   /// Note that this is a rather a helper method useful to fuse a partially serialized
   /// plan with another plan. The reason for having a partially serialized plan is to
   /// avoid unnecessary complication and enable partial plan serialization without
   /// affecting a global plan. Since kept as unique_ptr resources are relased efficiently
   /// upon releasing for the global plan.
   ```
   
   If this is a helper method does it need to appear in the `.h` file?



##########
cpp/src/arrow/engine/substrait/relation_internal.cc:
##########
@@ -421,5 +431,171 @@ Result<DeclarationInfo> FromProto(const substrait::Rel& rel, const ExtensionSet&
       rel.DebugString());
 }
 
+Result<std::unique_ptr<substrait::Rel>> ToProto(
+    const compute::Declaration& declr, ExtensionSet* ext_set,
+    const ConversionOptions& conversion_options) {
+  auto rel = make_unique<substrait::Rel>();
+  RETURN_NOT_OK(SerializeAndCombineRelations(declr, ext_set, rel, conversion_options));
+  return std::move(rel);
+}
+
+Status SetRelation(const std::unique_ptr<substrait::Rel>& plan,
+                   const std::unique_ptr<substrait::Rel>& partial_plan,
+                   const std::string& factory_name) {
+  if (factory_name == "scan" && partial_plan->has_read()) {
+    plan->set_allocated_read(partial_plan->release_read());
+  } else if (factory_name == "filter" && partial_plan->has_filter()) {
+    plan->set_allocated_filter(partial_plan->release_filter());
+  } else {
+    return Status::NotImplemented("Substrait converter ", factory_name,
+                                  " not supported.");
+  }
+  return Status::OK();
+}
+
+Result<std::shared_ptr<Schema>> ExtractSchemaToBind(const compute::Declaration& declr) {
+  std::shared_ptr<Schema> bind_schema;
+  if (declr.factory_name == "scan") {
+    const auto& opts = checked_cast<const dataset::ScanNodeOptions&>(*(declr.options));
+    bind_schema = opts.dataset->schema();
+  } else if (declr.factory_name == "filter") {
+    auto input_declr = util::get<compute::Declaration>(declr.inputs[0]);
+    ARROW_ASSIGN_OR_RAISE(bind_schema, ExtractSchemaToBind(input_declr));
+  } else if (declr.factory_name == "sink") {
+    // Note that the sink has no output_schema
+    return bind_schema;
+  } else {
+    return Status::Invalid("Schema extraction failed, unsupported factory ",
+                           declr.factory_name);
+  }
+  return bind_schema;
+}
+
+Status SerializeAndCombineRelations(const compute::Declaration& declaration,
+                                    ExtensionSet* ext_set,
+                                    std::unique_ptr<substrait::Rel>& rel,
+                                    const ConversionOptions& conversion_options) {
+  std::vector<compute::Declaration::Input> inputs = declaration.inputs;
+  for (auto& input : inputs) {
+    auto input_decl = util::get<compute::Declaration>(input);
+    RETURN_NOT_OK(
+        SerializeAndCombineRelations(input_decl, ext_set, rel, conversion_options));
+  }
+  const auto& factory_name = declaration.factory_name;
+  ARROW_ASSIGN_OR_RAISE(auto schema, ExtractSchemaToBind(declaration));
+  // Note that the sink declaration factory doesn't exist for serialization as
+  // Substrait doesn't deal with a sink node definition
+  std::unique_ptr<substrait::Rel> factory_rel;
+  if (factory_name == "scan") {
+    ARROW_ASSIGN_OR_RAISE(factory_rel, ScanRelationConverter(schema, declaration, ext_set,
+                                                             conversion_options));
+  } else if (factory_name == "filter") {
+    ARROW_ASSIGN_OR_RAISE(
+        factory_rel,
+        FilterRelationConverter(schema, declaration, ext_set, conversion_options));
+  } else {
+    return Status::NotImplemented("Factory ", factory_name,
+                                  " not implemented for roundtripping.");
+  }
+
+  if (factory_rel != nullptr) {
+    RETURN_NOT_OK(SetRelation(rel, factory_rel, factory_name));
+  } else {
+    return Status::Invalid("Conversion on factory ", factory_name,
+                           " returned an invalid relation");
+  }
+  return Status::OK();
+}
+
+Result<std::unique_ptr<substrait::Rel>> GetRelationFromDeclaration(
+    const compute::Declaration& declaration, ExtensionSet* ext_set,
+    const ConversionOptions& conversion_options) {
+  auto declr_input = declaration.inputs[0];
+  // Note that the input is expected in declaration.
+  // ExecNode inputs are not accepted
+  if (util::get_if<compute::ExecNode*>(&declr_input)) {
+    return Status::NotImplemented("Only support Plans written in Declaration format.");
+  }
+  return ToProto(util::get<compute::Declaration>(declr_input), ext_set,
+                 conversion_options);
+}
+
+Result<std::unique_ptr<substrait::Rel>> ScanRelationConverter(
+    const std::shared_ptr<Schema>& schema, const compute::Declaration& declaration,
+    ExtensionSet* ext_set, const ConversionOptions& conversion_options) {
+  auto rel = make_unique<substrait::Rel>();
+  auto read_rel = make_unique<substrait::ReadRel>();
+  const auto& scan_node_options =
+      checked_cast<const dataset::ScanNodeOptions&>(*declaration.options);
+  auto dataset =
+      dynamic_cast<dataset::FileSystemDataset*>(scan_node_options.dataset.get());
+  if (dataset == nullptr) {
+    return Status::Invalid("Can only convert file system datasets to a Substrait plan.");
+  }
+  // set schema
+  ARROW_ASSIGN_OR_RAISE(auto named_struct,
+                        ToProto(*dataset->schema(), ext_set, conversion_options));
+  read_rel->set_allocated_base_schema(named_struct.release());
+
+  // set local files
+  auto read_rel_lfs = make_unique<substrait::ReadRel_LocalFiles>();
+  for (const auto& file : dataset->files()) {
+    auto read_rel_lfs_ffs = make_unique<substrait::ReadRel_LocalFiles_FileOrFiles>();
+    read_rel_lfs_ffs->set_uri_path("file://" + file);
+    // set file format
+    // arrow and feather are temporarily handled via the Parquet format until
+    // upgraded to the latest Substrait version.

Review Comment:
   We have upgraded to the latest Substrait version. You are calling `set_allocated_arrow` and `set_allocated_orc` below.  I'm not sure this comment applies?



##########
cpp/src/arrow/engine/substrait/relation_internal.h:
##########
@@ -40,9 +40,46 @@ struct DeclarationInfo {
   int num_columns;
 };
 
+/// \brief A function to extract Acero Declaration from a Substrait Rel object
 ARROW_ENGINE_EXPORT
 Result<DeclarationInfo> FromProto(const substrait::Rel&, const ExtensionSet&,
                                   const ConversionOptions&);
 
+/// \brief Serializes a Declaration, produce a Substrait Rel and update the global
+/// Substrait plan. A Substrait Rel is passed as a the plan and it is updated with
+/// corresponding Declaration passed for serialization.
+///
+/// Note that this is a rather a helper method useful to fuse a partially serialized
+/// plan with another plan. The reason for having a partially serialized plan is to
+/// avoid unnecessary complication and enable partial plan serialization without
+/// affecting a global plan. Since kept as unique_ptr resources are relased efficiently
+/// upon releasing for the global plan.
+ARROW_ENGINE_EXPORT Status SerializeAndCombineRelations(const compute::Declaration&,
+                                                        ExtensionSet*,
+                                                        std::unique_ptr<substrait::Rel>&,

Review Comment:
   ```suggestion
                                                           std::unique_ptr<substrait::Rel>*,
   ```
   Arrow's style guide forbids the use of mutable references for out-parameters.  You should use a pointer here.



##########
cpp/src/arrow/engine/substrait/relation_internal.h:
##########
@@ -40,9 +40,46 @@ struct DeclarationInfo {
   int num_columns;
 };
 
+/// \brief A function to extract Acero Declaration from a Substrait Rel object
 ARROW_ENGINE_EXPORT
 Result<DeclarationInfo> FromProto(const substrait::Rel&, const ExtensionSet&,
                                   const ConversionOptions&);
 
+/// \brief Serializes a Declaration, produce a Substrait Rel and update the global
+/// Substrait plan. A Substrait Rel is passed as a the plan and it is updated with
+/// corresponding Declaration passed for serialization.
+///
+/// Note that this is a rather a helper method useful to fuse a partially serialized
+/// plan with another plan. The reason for having a partially serialized plan is to
+/// avoid unnecessary complication and enable partial plan serialization without
+/// affecting a global plan. Since kept as unique_ptr resources are relased efficiently
+/// upon releasing for the global plan.
+ARROW_ENGINE_EXPORT Status SerializeAndCombineRelations(const compute::Declaration&,
+                                                        ExtensionSet*,
+                                                        std::unique_ptr<substrait::Rel>&,
+                                                        const ConversionOptions&);
+
+/// \brief Serialize a Declaration and produces a Substrait Rel.
+///
+/// Note that in order to provide a generic interface for ToProto for
+/// declaration it is not specialized for each relation within the Substrait Rel.
+/// Rather a serialized relation is set as a member for the Substrait Rel
+/// (partial Relation) which is later on extracted to update a Substrait Rel
+/// which would be included in the fully serialized Acero Exec Plan.
+/// The ExecNode or ExecPlan is not used in this context as Declaration is preferred
+/// in the Substrait space rather than internal components of Acero execution engine.

Review Comment:
   I'm not sure I understand this paragraph



##########
cpp/src/arrow/engine/substrait/relation_internal.h:
##########
@@ -40,9 +40,46 @@ struct DeclarationInfo {
   int num_columns;
 };
 
+/// \brief A function to extract Acero Declaration from a Substrait Rel object

Review Comment:
   ```suggestion
   /// \brief Convert a Substrait Rel object to an Acero declaration
   ```



##########
cpp/src/arrow/engine/substrait/relation_internal.cc:
##########
@@ -421,5 +431,171 @@ Result<DeclarationInfo> FromProto(const substrait::Rel& rel, const ExtensionSet&
       rel.DebugString());
 }
 
+Result<std::unique_ptr<substrait::Rel>> ToProto(
+    const compute::Declaration& declr, ExtensionSet* ext_set,
+    const ConversionOptions& conversion_options) {
+  auto rel = make_unique<substrait::Rel>();
+  RETURN_NOT_OK(SerializeAndCombineRelations(declr, ext_set, rel, conversion_options));
+  return std::move(rel);
+}
+
+Status SetRelation(const std::unique_ptr<substrait::Rel>& plan,
+                   const std::unique_ptr<substrait::Rel>& partial_plan,
+                   const std::string& factory_name) {
+  if (factory_name == "scan" && partial_plan->has_read()) {
+    plan->set_allocated_read(partial_plan->release_read());
+  } else if (factory_name == "filter" && partial_plan->has_filter()) {
+    plan->set_allocated_filter(partial_plan->release_filter());
+  } else {
+    return Status::NotImplemented("Substrait converter ", factory_name,
+                                  " not supported.");
+  }
+  return Status::OK();
+}
+
+Result<std::shared_ptr<Schema>> ExtractSchemaToBind(const compute::Declaration& declr) {
+  std::shared_ptr<Schema> bind_schema;
+  if (declr.factory_name == "scan") {
+    const auto& opts = checked_cast<const dataset::ScanNodeOptions&>(*(declr.options));
+    bind_schema = opts.dataset->schema();
+  } else if (declr.factory_name == "filter") {
+    auto input_declr = util::get<compute::Declaration>(declr.inputs[0]);
+    ARROW_ASSIGN_OR_RAISE(bind_schema, ExtractSchemaToBind(input_declr));
+  } else if (declr.factory_name == "sink") {
+    // Note that the sink has no output_schema
+    return bind_schema;
+  } else {
+    return Status::Invalid("Schema extraction failed, unsupported factory ",
+                           declr.factory_name);
+  }
+  return bind_schema;
+}
+
+Status SerializeAndCombineRelations(const compute::Declaration& declaration,
+                                    ExtensionSet* ext_set,
+                                    std::unique_ptr<substrait::Rel>& rel,
+                                    const ConversionOptions& conversion_options) {
+  std::vector<compute::Declaration::Input> inputs = declaration.inputs;
+  for (auto& input : inputs) {
+    auto input_decl = util::get<compute::Declaration>(input);
+    RETURN_NOT_OK(
+        SerializeAndCombineRelations(input_decl, ext_set, rel, conversion_options));
+  }
+  const auto& factory_name = declaration.factory_name;
+  ARROW_ASSIGN_OR_RAISE(auto schema, ExtractSchemaToBind(declaration));
+  // Note that the sink declaration factory doesn't exist for serialization as
+  // Substrait doesn't deal with a sink node definition
+  std::unique_ptr<substrait::Rel> factory_rel;
+  if (factory_name == "scan") {
+    ARROW_ASSIGN_OR_RAISE(factory_rel, ScanRelationConverter(schema, declaration, ext_set,
+                                                             conversion_options));
+  } else if (factory_name == "filter") {
+    ARROW_ASSIGN_OR_RAISE(
+        factory_rel,
+        FilterRelationConverter(schema, declaration, ext_set, conversion_options));
+  } else {
+    return Status::NotImplemented("Factory ", factory_name,
+                                  " not implemented for roundtripping.");
+  }
+
+  if (factory_rel != nullptr) {
+    RETURN_NOT_OK(SetRelation(rel, factory_rel, factory_name));
+  } else {
+    return Status::Invalid("Conversion on factory ", factory_name,
+                           " returned an invalid relation");
+  }
+  return Status::OK();
+}
+
+Result<std::unique_ptr<substrait::Rel>> GetRelationFromDeclaration(
+    const compute::Declaration& declaration, ExtensionSet* ext_set,
+    const ConversionOptions& conversion_options) {
+  auto declr_input = declaration.inputs[0];
+  // Note that the input is expected in declaration.
+  // ExecNode inputs are not accepted
+  if (util::get_if<compute::ExecNode*>(&declr_input)) {
+    return Status::NotImplemented("Only support Plans written in Declaration format.");
+  }
+  return ToProto(util::get<compute::Declaration>(declr_input), ext_set,
+                 conversion_options);
+}
+
+Result<std::unique_ptr<substrait::Rel>> ScanRelationConverter(
+    const std::shared_ptr<Schema>& schema, const compute::Declaration& declaration,
+    ExtensionSet* ext_set, const ConversionOptions& conversion_options) {
+  auto rel = make_unique<substrait::Rel>();
+  auto read_rel = make_unique<substrait::ReadRel>();
+  const auto& scan_node_options =
+      checked_cast<const dataset::ScanNodeOptions&>(*declaration.options);
+  auto dataset =
+      dynamic_cast<dataset::FileSystemDataset*>(scan_node_options.dataset.get());
+  if (dataset == nullptr) {
+    return Status::Invalid("Can only convert file system datasets to a Substrait plan.");
+  }
+  // set schema
+  ARROW_ASSIGN_OR_RAISE(auto named_struct,
+                        ToProto(*dataset->schema(), ext_set, conversion_options));
+  read_rel->set_allocated_base_schema(named_struct.release());
+
+  // set local files
+  auto read_rel_lfs = make_unique<substrait::ReadRel_LocalFiles>();
+  for (const auto& file : dataset->files()) {
+    auto read_rel_lfs_ffs = make_unique<substrait::ReadRel_LocalFiles_FileOrFiles>();
+    read_rel_lfs_ffs->set_uri_path("file://" + file);

Review Comment:
   We should probably add a method in `uri.h` to create a file URI from a path.



##########
cpp/src/arrow/engine/substrait/serde_test.cc:
##########
@@ -1731,5 +1808,265 @@ TEST(Substrait, AggregateWithFilter) {
   ASSERT_RAISES(NotImplemented, DeserializePlans(*buf, [] { return kNullConsumer; }));
 }
 
+TEST(Substrait, BasicPlanRoundTripping) {
+#ifdef _WIN32
+  GTEST_SKIP() << "ARROW-16392: Substrait File URI not supported for Windows";
+#else
+  compute::ExecContext exec_context;
+  ExtensionSet ext_set;
+  auto dummy_schema = schema(
+      {field("key", int32()), field("shared", int32()), field("distinct", int32())});
+
+  // creating a dummy dataset using a dummy table
+  auto table = TableFromJSON(dummy_schema, {R"([
+      [1, 1, 10],
+      [3, 4, 20]
+    ])",
+                                            R"([
+      [0, 2, 1],
+      [1, 3, 2],
+      [4, 1, 3],
+      [3, 1, 3],
+      [1, 2, 5]
+    ])",
+                                            R"([
+      [2, 2, 12],
+      [5, 3, 12],
+      [1, 3, 12]
+    ])"});
+
+  auto format = std::make_shared<arrow::dataset::ParquetFileFormat>();
+  auto filesystem = std::make_shared<fs::LocalFileSystem>();
+  const std::string file_name = "serde_test.parquet";
+
+  ASSERT_OK_AND_ASSIGN(auto tempdir,
+                       arrow::internal::TemporaryDir::Make("substrait_tempdir"));
+  ASSERT_OK_AND_ASSIGN(auto file_path, tempdir->path().Join(file_name));
+  std::string file_path_str = file_path.ToString();
+
+  // Note: there is an additional forward slash introduced by the tempdir
+  // it must be replaced to properly load into reading files
+  // TODO: (Review: Jira needs to be reported to handle this properly)

Review Comment:
   Can you give an example of what the path looks like before you do the replace?  I'm not sure I follow.



##########
cpp/src/arrow/engine/substrait/relation_internal.cc:
##########
@@ -29,10 +29,20 @@
 #include "arrow/filesystem/localfs.h"
 #include "arrow/filesystem/path_util.h"
 #include "arrow/filesystem/util_internal.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/make_unique.h"
 
 namespace arrow {
+
+using internal::checked_cast;
+using internal::make_unique;
+
 namespace engine {
 
+using SubstraitConverter = std::function<Result<std::unique_ptr<substrait::Rel>>(
+    const std::shared_ptr<Schema>&, const compute::Declaration&, ExtensionSet*,
+    const ConversionOptions&)>;
+

Review Comment:
   I think you can get rid of this now that we aren't doing a registry approach?



##########
cpp/src/arrow/engine/substrait/serde.h:
##########
@@ -202,6 +207,18 @@ Result<std::shared_ptr<Buffer>> SerializeExpression(
     const compute::Expression& expr, ExtensionSet* ext_set,
     const ConversionOptions& conversion_options = {});
 
+/// \brief Serializes an Arrow compute Declaration to a Substrait Relation message
+///
+/// \param[in] declaration the Arrow compute declaration to serialize
+/// \param[in,out] ext_set the extension mapping to use; may be updated to add
+/// \param[in] conversion_options options to control how the conversion is done
+/// mappings for the components in the used declaration
+/// \return a buffer containing the protobuf serialization of the corresponding Substrait
+/// relation message

Review Comment:
   ```suggestion
   /// \brief Serialize an Acero Declaration to a binary protobuf Substrait message
   ///
   /// \param[in] declaration the Acero declaration to serialize
   /// \param[in,out] ext_set the extension mapping to use; may be updated to add
   /// \param[in] conversion_options options to control how the conversion is done
   ///
   /// \return a buffer containing the protobuf serialization of the Acero relation
   ```



-- 
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] vibhatha commented on a diff in pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
vibhatha commented on code in PR #13401:
URL: https://github.com/apache/arrow/pull/13401#discussion_r956537882


##########
cpp/src/arrow/engine/substrait/plan_internal.cc:
##########
@@ -133,5 +136,18 @@ Result<ExtensionSet> GetExtensionSetFromPlan(const substrait::Plan& plan,
                             registry);
 }
 
+Result<std::unique_ptr<substrait::Plan>> PlanToProto(

Review Comment:
   The issue is that there is the same function signature in the `relation_internal.h`. What we do there is output the corresponding serialized `substrait::Rel` for a `Declaration`. Since the Substrait relation model entities are final classes which doesn't extend from a generic relation interface, what we do is create a `substrait::Rel` and fill the corresponding part. This is considered as a partial plan. For instance if the passed `Declaration` is a `scan` we populate the `read` in the `substrait::Rel` object. Then in the `plan_internal.cc` we extract that `read` component and bind to `substrait::Rel` which is considered as the full plan. In the `plan_internal.cc` what we do is we pass the `sink` to the `PlanToProto` to get the `substrait::Rel` which is recursively called until the whole plan is serialized. 
   
   So that's the reason for having this function signature to make clear and avoid compiler errors. Wanted to expose both interfaces to the user so that it can be used accordingly. 
   
   



-- 
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] vibhatha commented on a diff in pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
vibhatha commented on code in PR #13401:
URL: https://github.com/apache/arrow/pull/13401#discussion_r956909353


##########
cpp/src/arrow/engine/substrait/relation_internal.h:
##########
@@ -40,9 +40,46 @@ struct DeclarationInfo {
   int num_columns;
 };
 
+/// \brief A function to extract Acero Declaration from a Substrait Rel object
 ARROW_ENGINE_EXPORT
 Result<DeclarationInfo> FromProto(const substrait::Rel&, const ExtensionSet&,
                                   const ConversionOptions&);
 
+/// \brief Serializes a Declaration, produce a Substrait Rel and update the global
+/// Substrait plan. A Substrait Rel is passed as a the plan and it is updated with
+/// corresponding Declaration passed for serialization.
+///
+/// Note that this is a rather a helper method useful to fuse a partially serialized
+/// plan with another plan. The reason for having a partially serialized plan is to
+/// avoid unnecessary complication and enable partial plan serialization without
+/// affecting a global plan. Since kept as unique_ptr resources are relased efficiently
+/// upon releasing for the global plan.

Review Comment:
   Actually not a helper, but this interface is required for both `plan` and `relation` serialization. 



-- 
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] vibhatha commented on a diff in pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
vibhatha commented on code in PR #13401:
URL: https://github.com/apache/arrow/pull/13401#discussion_r956912078


##########
cpp/src/arrow/engine/substrait/relation_internal.h:
##########
@@ -40,9 +40,46 @@ struct DeclarationInfo {
   int num_columns;
 };
 
+/// \brief A function to extract Acero Declaration from a Substrait Rel object
 ARROW_ENGINE_EXPORT
 Result<DeclarationInfo> FromProto(const substrait::Rel&, const ExtensionSet&,
                                   const ConversionOptions&);
 
+/// \brief Serializes a Declaration, produce a Substrait Rel and update the global
+/// Substrait plan. A Substrait Rel is passed as a the plan and it is updated with
+/// corresponding Declaration passed for serialization.
+///
+/// Note that this is a rather a helper method useful to fuse a partially serialized
+/// plan with another plan. The reason for having a partially serialized plan is to
+/// avoid unnecessary complication and enable partial plan serialization without
+/// affecting a global plan. Since kept as unique_ptr resources are relased efficiently
+/// upon releasing for the global plan.

Review Comment:
   Updated the description. 



-- 
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] vibhatha commented on a diff in pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
vibhatha commented on code in PR #13401:
URL: https://github.com/apache/arrow/pull/13401#discussion_r936694253


##########
cpp/src/arrow/engine/substrait/serde_test.cc:
##########
@@ -1383,5 +1400,188 @@ TEST(Substrait, JoinPlanInvalidKeys) {
   }
 }
 
+TEST(Substrait, SerializeRelation) {
+#ifdef _WIN32
+  GTEST_SKIP() << "ARROW-16392: Substrait File URI not supported for Windows";
+#else
+  ExtensionSet ext_set;
+  auto dummy_schema = schema({field("foo", binary())});
+  // creating a dummy dataset using a dummy table
+  auto format = std::make_shared<arrow::dataset::ParquetFileFormat>();
+  auto filesystem = std::make_shared<fs::LocalFileSystem>();
+
+  std::vector<fs::FileInfo> files;
+  const std::vector<std::string> f_paths = {"/tmp/data1.parquet", "/tmp/data2.parquet"};

Review Comment:
   This should be okay now. I updated the test case. 



-- 
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] westonpace commented on pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
westonpace commented on PR #13401:
URL: https://github.com/apache/arrow/pull/13401#issuecomment-1208677326

   > Since this API is designed to be a test-only utility I wouldn't expect us to need to deal with custom exec nodes
   
   @bkietz 
   
   That's a reasonable stance.  I suppose I had hoped we could provide some utility to users with custom relations.  I suppose the internal-ness of Substrait makes that pretty difficult however.  Custom relations in Substrait use the `any` message type in protobuf.  Is it legal to expose `google/protobuf/any.pb.h`?
   
   If yes, then it seems we could still have a registry (or at least a fallback lambda) for custom extensions.
   
   If no, then won't this be a concern for the other direction, e.g. deserializing Substrait plans with custom extensions?


-- 
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] westonpace commented on pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
westonpace commented on PR #13401:
URL: https://github.com/apache/arrow/pull/13401#issuecomment-1175623078

   > Intersting thoughts. I will take a look at the tool. It would be better if we can use it to validate things. But I am not sure if it needs to be inside the Arrow source or should it be a plugin for Apache Arrow. cc @westonpace
   
   I think it would make a lot of sense for unit tests to bring in the validator as a C dependency.


-- 
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] vibhatha commented on a diff in pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
vibhatha commented on code in PR #13401:
URL: https://github.com/apache/arrow/pull/13401#discussion_r956887639


##########
cpp/src/arrow/engine/substrait/relation_internal.cc:
##########
@@ -421,5 +431,171 @@ Result<DeclarationInfo> FromProto(const substrait::Rel& rel, const ExtensionSet&
       rel.DebugString());
 }
 
+Result<std::unique_ptr<substrait::Rel>> ToProto(
+    const compute::Declaration& declr, ExtensionSet* ext_set,
+    const ConversionOptions& conversion_options) {
+  auto rel = make_unique<substrait::Rel>();
+  RETURN_NOT_OK(SerializeAndCombineRelations(declr, ext_set, rel, conversion_options));
+  return std::move(rel);
+}
+
+Status SetRelation(const std::unique_ptr<substrait::Rel>& plan,
+                   const std::unique_ptr<substrait::Rel>& partial_plan,
+                   const std::string& factory_name) {
+  if (factory_name == "scan" && partial_plan->has_read()) {
+    plan->set_allocated_read(partial_plan->release_read());
+  } else if (factory_name == "filter" && partial_plan->has_filter()) {
+    plan->set_allocated_filter(partial_plan->release_filter());
+  } else {
+    return Status::NotImplemented("Substrait converter ", factory_name,
+                                  " not supported.");
+  }
+  return Status::OK();
+}
+
+Result<std::shared_ptr<Schema>> ExtractSchemaToBind(const compute::Declaration& declr) {
+  std::shared_ptr<Schema> bind_schema;
+  if (declr.factory_name == "scan") {
+    const auto& opts = checked_cast<const dataset::ScanNodeOptions&>(*(declr.options));
+    bind_schema = opts.dataset->schema();
+  } else if (declr.factory_name == "filter") {
+    auto input_declr = util::get<compute::Declaration>(declr.inputs[0]);
+    ARROW_ASSIGN_OR_RAISE(bind_schema, ExtractSchemaToBind(input_declr));
+  } else if (declr.factory_name == "sink") {
+    // Note that the sink has no output_schema
+    return bind_schema;
+  } else {
+    return Status::Invalid("Schema extraction failed, unsupported factory ",
+                           declr.factory_name);
+  }
+  return bind_schema;
+}
+
+Status SerializeAndCombineRelations(const compute::Declaration& declaration,
+                                    ExtensionSet* ext_set,
+                                    std::unique_ptr<substrait::Rel>& rel,
+                                    const ConversionOptions& conversion_options) {
+  std::vector<compute::Declaration::Input> inputs = declaration.inputs;
+  for (auto& input : inputs) {
+    auto input_decl = util::get<compute::Declaration>(input);
+    RETURN_NOT_OK(
+        SerializeAndCombineRelations(input_decl, ext_set, rel, conversion_options));
+  }
+  const auto& factory_name = declaration.factory_name;
+  ARROW_ASSIGN_OR_RAISE(auto schema, ExtractSchemaToBind(declaration));
+  // Note that the sink declaration factory doesn't exist for serialization as
+  // Substrait doesn't deal with a sink node definition
+  std::unique_ptr<substrait::Rel> factory_rel;
+  if (factory_name == "scan") {
+    ARROW_ASSIGN_OR_RAISE(factory_rel, ScanRelationConverter(schema, declaration, ext_set,
+                                                             conversion_options));
+  } else if (factory_name == "filter") {
+    ARROW_ASSIGN_OR_RAISE(
+        factory_rel,
+        FilterRelationConverter(schema, declaration, ext_set, conversion_options));
+  } else {
+    return Status::NotImplemented("Factory ", factory_name,
+                                  " not implemented for roundtripping.");
+  }
+
+  if (factory_rel != nullptr) {
+    RETURN_NOT_OK(SetRelation(rel, factory_rel, factory_name));
+  } else {
+    return Status::Invalid("Conversion on factory ", factory_name,
+                           " returned an invalid relation");
+  }
+  return Status::OK();
+}
+
+Result<std::unique_ptr<substrait::Rel>> GetRelationFromDeclaration(
+    const compute::Declaration& declaration, ExtensionSet* ext_set,
+    const ConversionOptions& conversion_options) {
+  auto declr_input = declaration.inputs[0];
+  // Note that the input is expected in declaration.
+  // ExecNode inputs are not accepted
+  if (util::get_if<compute::ExecNode*>(&declr_input)) {
+    return Status::NotImplemented("Only support Plans written in Declaration format.");
+  }
+  return ToProto(util::get<compute::Declaration>(declr_input), ext_set,
+                 conversion_options);
+}
+
+Result<std::unique_ptr<substrait::Rel>> ScanRelationConverter(
+    const std::shared_ptr<Schema>& schema, const compute::Declaration& declaration,
+    ExtensionSet* ext_set, const ConversionOptions& conversion_options) {
+  auto rel = make_unique<substrait::Rel>();
+  auto read_rel = make_unique<substrait::ReadRel>();
+  const auto& scan_node_options =
+      checked_cast<const dataset::ScanNodeOptions&>(*declaration.options);
+  auto dataset =
+      dynamic_cast<dataset::FileSystemDataset*>(scan_node_options.dataset.get());
+  if (dataset == nullptr) {
+    return Status::Invalid("Can only convert file system datasets to a Substrait plan.");
+  }
+  // set schema
+  ARROW_ASSIGN_OR_RAISE(auto named_struct,
+                        ToProto(*dataset->schema(), ext_set, conversion_options));
+  read_rel->set_allocated_base_schema(named_struct.release());
+
+  // set local files
+  auto read_rel_lfs = make_unique<substrait::ReadRel_LocalFiles>();
+  for (const auto& file : dataset->files()) {
+    auto read_rel_lfs_ffs = make_unique<substrait::ReadRel_LocalFiles_FileOrFiles>();
+    read_rel_lfs_ffs->set_uri_path("file://" + file);

Review Comment:
   Probably a follow up PR would be required to clean up all such uses of File URI. For this PR I included the interface and only used it within the `relation_internal.cc` 



-- 
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] vibhatha commented on a diff in pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
vibhatha commented on code in PR #13401:
URL: https://github.com/apache/arrow/pull/13401#discussion_r915724094


##########
cpp/src/arrow/engine/substrait/serde_test.cc:
##########
@@ -1173,5 +1175,173 @@ TEST(Substrait, JoinPlanInvalidKeys) {
           &ext_set));
 }
 
+TEST(Substrait, SerializeRelation) {
+#ifdef _WIN32
+  GTEST_SKIP() << "ARROW-16392: Substrait File URI not supported for Windows";
+#else
+  ExtensionSet ext_set;
+  auto dummy_schema = schema({field("foo", binary())});
+  // creating a dummy dataset using a dummy table
+  auto format = std::make_shared<arrow::dataset::ParquetFileFormat>();
+  auto filesystem = std::make_shared<fs::LocalFileSystem>();

Review Comment:
   On second thoughts. I will handle it later. It may need a little more generic setting. The default one doesn't work well for other tests. For now I am addressing it just for this test case. 



-- 
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] lidavidm commented on a diff in pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
lidavidm commented on code in PR #13401:
URL: https://github.com/apache/arrow/pull/13401#discussion_r904936747


##########
cpp/src/arrow/engine/substrait/serde_test.cc:
##########
@@ -1173,5 +1175,168 @@ TEST(Substrait, JoinPlanInvalidKeys) {
           &ext_set));
 }
 
+TEST(Substrait, SerializeRelation) {
+#ifdef _WIN32
+  GTEST_SKIP() << "ARROW-16392: Substrait File URI not supported for Windows";
+#else
+  ExtensionSet ext_set;
+  auto dummy_schema = schema({field("f1", int32()), field("f2", int32())});
+  // creating a dummy dataset using a dummy table
+  auto format = std::make_shared<arrow::dataset::ParquetFileFormat>();
+  auto filesystem = std::make_shared<fs::LocalFileSystem>();
+
+  std::vector<fs::FileInfo> files;
+  const std::vector<std::string> f_paths = {"/tmp/data1.parquet", "/tmp/data2.parquet"};
+
+  for (const auto& f_path : f_paths) {
+    ASSERT_OK_AND_ASSIGN(auto f_file, filesystem->GetFileInfo(f_path));
+    files.push_back(std::move(f_file));
+  }
+
+  ASSERT_OK_AND_ASSIGN(auto ds_factory, dataset::FileSystemDatasetFactory::Make(
+                                            std::move(filesystem), std::move(files),
+                                            std::move(format), {}));
+  ASSERT_OK_AND_ASSIGN(auto dataset, ds_factory->Finish(dummy_schema));
+
+  auto options = std::make_shared<dataset::ScanOptions>();
+  options->projection = compute::project({}, {});
+  auto scan_node_options = dataset::ScanNodeOptions{dataset, options};
+
+  auto scan_declaration = compute::Declaration({"scan", scan_node_options});
+
+  ASSERT_OK_AND_ASSIGN(auto serialized_rel,
+                       SerializeRelation(scan_declaration, &ext_set));
+  ASSERT_OK_AND_ASSIGN(auto deserialized_decl,
+                       DeserializeRelation(*serialized_rel, ext_set));
+
+  auto dataset_comparator = [](std::shared_ptr<dataset::Dataset> ds_lhs,
+                               std::shared_ptr<dataset::Dataset> ds_rhs) -> bool {
+    const auto& fds_lhs = checked_cast<const dataset::FileSystemDataset&>(*ds_lhs);
+    const auto& fds_rhs = checked_cast<const dataset::FileSystemDataset&>(*ds_lhs);
+    const auto& files_lhs = fds_lhs.files();
+    const auto& files_rhs = fds_rhs.files();
+
+    bool cmp_fsize = files_lhs.size() == files_rhs.size();
+    uint64_t fidx = 0;
+    for (const auto& l_file : files_lhs) {
+      if (l_file != files_rhs[fidx]) {
+        return false;
+      }
+      fidx++;
+    }
+    bool cmp_file_format = fds_lhs.format()->Equals(*fds_lhs.format());
+    bool cmp_file_system = fds_lhs.filesystem()->Equals(fds_rhs.filesystem());
+    return cmp_fsize && cmp_file_format && cmp_file_system;
+  };
+
+  auto scan_option_comparator = [dataset_comparator](
+                                    const dataset::ScanNodeOptions& lhs,
+                                    const dataset::ScanNodeOptions& rhs) -> bool {
+    bool cmp_rso = lhs.require_sequenced_output == rhs.require_sequenced_output;
+    bool cmp_ds = dataset_comparator(lhs.dataset, rhs.dataset);
+    return cmp_rso && cmp_ds;
+  };

Review Comment:
   I'm not currently deep enough in that code to really give you a full answer off the top of my head



-- 
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] vibhatha commented on a diff in pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
vibhatha commented on code in PR #13401:
URL: https://github.com/apache/arrow/pull/13401#discussion_r907421704


##########
cpp/src/arrow/engine/substrait/serde_test.cc:
##########
@@ -1173,5 +1175,173 @@ TEST(Substrait, JoinPlanInvalidKeys) {
           &ext_set));
 }
 
+TEST(Substrait, SerializeRelation) {
+#ifdef _WIN32
+  GTEST_SKIP() << "ARROW-16392: Substrait File URI not supported for Windows";
+#else
+  ExtensionSet ext_set;
+  auto dummy_schema = schema({field("foo", binary())});
+  // creating a dummy dataset using a dummy table
+  auto format = std::make_shared<arrow::dataset::ParquetFileFormat>();
+  auto filesystem = std::make_shared<fs::LocalFileSystem>();
+
+  ASSERT_OK_AND_ASSIGN(std::string dir_string,
+                       arrow::internal::GetEnvVar("PARQUET_TEST_DATA"));
+  auto file_name =
+      arrow::internal::PlatformFilename::FromString(dir_string)->Join("binary.parquet");
+
+  std::vector<fs::FileInfo> files;
+  const std::vector<std::string> f_paths = {file_name->ToString()};
+
+  for (const auto& f_path : f_paths) {
+    ASSERT_OK_AND_ASSIGN(auto f_file, filesystem->GetFileInfo(f_path));
+    files.push_back(std::move(f_file));
+  }
+
+  ASSERT_OK_AND_ASSIGN(auto ds_factory, dataset::FileSystemDatasetFactory::Make(
+                                            std::move(filesystem), std::move(files),
+                                            std::move(format), {}));
+  ASSERT_OK_AND_ASSIGN(auto dataset, ds_factory->Finish(dummy_schema));
+
+  auto options = std::make_shared<dataset::ScanOptions>();
+  options->projection = compute::project({}, {});
+  auto scan_node_options = dataset::ScanNodeOptions{dataset, options};
+
+  auto scan_declaration = compute::Declaration({"scan", scan_node_options});
+
+  ASSERT_OK_AND_ASSIGN(auto serialized_rel,
+                       SerializeRelation(scan_declaration, &ext_set));
+  ASSERT_OK_AND_ASSIGN(auto deserialized_decl,
+                       DeserializeRelation(*serialized_rel, ext_set));
+
+  auto dataset_comparator = [](std::shared_ptr<dataset::Dataset> ds_lhs,
+                               std::shared_ptr<dataset::Dataset> ds_rhs) -> bool {
+    const auto& fds_lhs = checked_cast<const dataset::FileSystemDataset&>(*ds_lhs);
+    const auto& fds_rhs = checked_cast<const dataset::FileSystemDataset&>(*ds_lhs);
+    const auto& files_lhs = fds_lhs.files();
+    const auto& files_rhs = fds_rhs.files();
+
+    bool cmp_fsize = files_lhs.size() == files_rhs.size();
+    uint64_t fidx = 0;
+    for (const auto& l_file : files_lhs) {
+      if (l_file != files_rhs[fidx]) {
+        return false;
+      }
+      fidx++;
+    }
+    bool cmp_file_format = fds_lhs.format()->Equals(*fds_lhs.format());
+    bool cmp_file_system = fds_lhs.filesystem()->Equals(fds_rhs.filesystem());
+    return cmp_fsize && cmp_file_format && cmp_file_system;
+  };
+
+  auto scan_option_comparator = [dataset_comparator](
+                                    const dataset::ScanNodeOptions& lhs,
+                                    const dataset::ScanNodeOptions& rhs) -> bool {
+    bool cmp_rso = lhs.require_sequenced_output == rhs.require_sequenced_output;
+    bool cmp_ds = dataset_comparator(lhs.dataset, rhs.dataset);
+    return cmp_rso && cmp_ds;
+  };
+
+  EXPECT_EQ(deserialized_decl.factory_name, scan_declaration.factory_name);
+  const auto& lhs =
+      checked_cast<const dataset::ScanNodeOptions&>(*deserialized_decl.options);
+  const auto& rhs =
+      checked_cast<const dataset::ScanNodeOptions&>(*scan_declaration.options);
+  ASSERT_TRUE(scan_option_comparator(lhs, rhs));
+#endif
+}
+
+TEST(Substrait, SerializeRelationEndToEnd) {

Review Comment:
   Ah its here: https://github.com/apache/arrow/pull/13401#discussion_r904965152



-- 
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] vibhatha commented on a diff in pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
vibhatha commented on code in PR #13401:
URL: https://github.com/apache/arrow/pull/13401#discussion_r907391525


##########
cpp/src/arrow/engine/substrait/relation_internal.cc:
##########
@@ -316,5 +323,97 @@ Result<compute::Declaration> FromProto(const substrait::Rel& rel,
       rel.DebugString());
 }
 
+namespace {
+// TODO: add other types
+enum ArrowRelationType : uint8_t {
+  SCAN,
+  FILTER,
+  PROJECT,
+  JOIN,
+  AGGREGATE,
+};
+
+const std::map<std::string, ArrowRelationType> enum_map{
+    {"scan", ArrowRelationType::SCAN},           {"filter", ArrowRelationType::FILTER},
+    {"project", ArrowRelationType::PROJECT},     {"join", ArrowRelationType::JOIN},
+    {"aggregate", ArrowRelationType::AGGREGATE},
+};
+
+struct ExtractRelation {
+  explicit ExtractRelation(substrait::Rel* rel, ExtensionSet* ext_set)
+      : rel_(rel), ext_set_(ext_set) {}
+
+  Status AddRelation(const compute::Declaration& declaration) {
+    const std::string& rel_name = declaration.factory_name;
+    switch (enum_map.find(rel_name)->second) {
+      case ArrowRelationType::SCAN:
+        return AddReadRelation(declaration);
+      case ArrowRelationType::FILTER:
+        return Status::NotImplemented("Filter operator not supported.");
+      case ArrowRelationType::PROJECT:
+        return Status::NotImplemented("Project operator not supported.");
+      case ArrowRelationType::JOIN:
+        return Status::NotImplemented("Join operator not supported.");
+      case ArrowRelationType::AGGREGATE:
+        return Status::NotImplemented("Aggregate operator not supported.");
+      default:
+        return Status::Invalid("Unsupported factory name :", rel_name);
+    }
+  }

Review Comment:
   I thought about this as well. I want suggest to use an `enum` for `ExecNode` representation. Plus I wanted to push it because it could be faster than `if-else`. If we have a complex query, the ToProto would called many times. I was thinking about this point. WDYT?



-- 
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] vibhatha commented on a diff in pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
vibhatha commented on code in PR #13401:
URL: https://github.com/apache/arrow/pull/13401#discussion_r907407468


##########
cpp/src/arrow/engine/substrait/serde_test.cc:
##########
@@ -1173,5 +1175,173 @@ TEST(Substrait, JoinPlanInvalidKeys) {
           &ext_set));
 }
 
+TEST(Substrait, SerializeRelation) {
+#ifdef _WIN32
+  GTEST_SKIP() << "ARROW-16392: Substrait File URI not supported for Windows";
+#else
+  ExtensionSet ext_set;
+  auto dummy_schema = schema({field("foo", binary())});
+  // creating a dummy dataset using a dummy table
+  auto format = std::make_shared<arrow::dataset::ParquetFileFormat>();
+  auto filesystem = std::make_shared<fs::LocalFileSystem>();

Review Comment:
   I think there was an issue using Mock filesystem as I can recall with Substrait related things. In a previous PR for Substrait we also used as we have done here. I cannot excatly recall why we didn't use Mock filesystem, but I can try.



-- 
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] vibhatha commented on pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
vibhatha commented on PR #13401:
URL: https://github.com/apache/arrow/pull/13401#issuecomment-1160235282

   > I don't feel qualified to comment on those design questions, but FWIW, I ran the serialized output of the test case through the validator and it's okay-ish (the validator doesn't like the lack of a `NULLABILITY_REQUIRED` in the struct that represents the schema, but that's pretty pedantic I guess), and the code looks fine to me.
   
   Thanks a lot for the quick check on this. It’s very interesting how you validated things using the tool. Do you think it’s wise to add a CI to test Substrait related queries using this tool? 
   
   Please feel free to add suggestions. @jvanstraten 
   
   
   One doubtful thing is to check in serialization is whether a projection or filter expression is added or not/ differentiation from default values. For instance filter expression defaults to a boolean literal of value true.
   
   
   cc @westonpace For future reference in the review. 
   


-- 
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] vibhatha commented on a diff in pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
vibhatha commented on code in PR #13401:
URL: https://github.com/apache/arrow/pull/13401#discussion_r914360146


##########
cpp/src/arrow/engine/substrait/relation_internal.cc:
##########
@@ -316,5 +323,97 @@ Result<compute::Declaration> FromProto(const substrait::Rel& rel,
       rel.DebugString());
 }
 
+namespace {
+// TODO: add other types
+enum ArrowRelationType : uint8_t {
+  SCAN,
+  FILTER,
+  PROJECT,
+  JOIN,
+  AGGREGATE,
+};
+
+const std::map<std::string, ArrowRelationType> enum_map{
+    {"scan", ArrowRelationType::SCAN},           {"filter", ArrowRelationType::FILTER},
+    {"project", ArrowRelationType::PROJECT},     {"join", ArrowRelationType::JOIN},
+    {"aggregate", ArrowRelationType::AGGREGATE},
+};
+
+struct ExtractRelation {
+  explicit ExtractRelation(substrait::Rel* rel, ExtensionSet* ext_set)
+      : rel_(rel), ext_set_(ext_set) {}
+
+  Status AddRelation(const compute::Declaration& declaration) {
+    const std::string& rel_name = declaration.factory_name;
+    switch (enum_map.find(rel_name)->second) {
+      case ArrowRelationType::SCAN:
+        return AddReadRelation(declaration);
+      case ArrowRelationType::FILTER:
+        return Status::NotImplemented("Filter operator not supported.");
+      case ArrowRelationType::PROJECT:
+        return Status::NotImplemented("Project operator not supported.");
+      case ArrowRelationType::JOIN:
+        return Status::NotImplemented("Join operator not supported.");
+      case ArrowRelationType::AGGREGATE:
+        return Status::NotImplemented("Aggregate operator not supported.");
+      default:
+        return Status::Invalid("Unsupported factory name :", rel_name);
+    }
+  }
+
+  Status AddReadRelation(const compute::Declaration& declaration) {
+    auto read_rel = internal::make_unique<substrait::ReadRel>();
+    const auto& scan_node_options =
+        internal::checked_cast<const dataset::ScanNodeOptions&>(*declaration.options);
+
+    const auto& fds = internal::checked_cast<const dataset::FileSystemDataset&>(
+        *scan_node_options.dataset);
+
+    // set schema
+    ARROW_ASSIGN_OR_RAISE(auto named_struct, ToProto(*fds.schema(), ext_set_));
+    read_rel->set_allocated_base_schema(named_struct.release());
+
+    // set local files
+    auto read_rel_lfs = internal::make_unique<substrait::ReadRel_LocalFiles>();
+    for (const auto& file : fds.files()) {
+      auto read_rel_lfs_ffs =
+          internal::make_unique<substrait::ReadRel_LocalFiles_FileOrFiles>();
+      read_rel_lfs_ffs->set_uri_path("file://" + file);
+
+      // set file format
+      auto format_type_name = fds.format()->type_name();
+      if (format_type_name == "parquet" || format_type_name == "arrow" ||
+          format_type_name == "feather") {
+        read_rel_lfs_ffs->set_format(
+            substrait::ReadRel::LocalFiles::FileOrFiles::FILE_FORMAT_PARQUET);
+      } else {
+        return Status::Invalid("Unsupported file type : ", format_type_name);
+      }
+      read_rel_lfs->mutable_items()->AddAllocated(read_rel_lfs_ffs.release());
+    }
+    *read_rel->mutable_local_files() = *read_rel_lfs.get();
+
+    rel_->set_allocated_read(read_rel.release());
+    return Status::OK();
+  }
+
+  Status operator()(const compute::Declaration& declaration) {
+    return AddRelation(declaration);
+  }
+
+ private:
+  substrait::Rel* rel_;
+  ExtensionSet* ext_set_;
+};
+
+}  // namespace
+
+Result<std::unique_ptr<substrait::Rel>> ToProto(const compute::Declaration& declaration,
+                                                ExtensionSet* ext_set) {
+  auto out = internal::make_unique<substrait::Rel>();
+  RETURN_NOT_OK(ExtractRelation(out.get(), ext_set)(declaration));

Review Comment:
   Sure. I will refactor



-- 
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] westonpace commented on a diff in pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
westonpace commented on code in PR #13401:
URL: https://github.com/apache/arrow/pull/13401#discussion_r914300910


##########
cpp/src/arrow/engine/substrait/relation_internal.cc:
##########
@@ -316,5 +323,97 @@ Result<compute::Declaration> FromProto(const substrait::Rel& rel,
       rel.DebugString());
 }
 
+namespace {
+// TODO: add other types
+enum ArrowRelationType : uint8_t {
+  SCAN,
+  FILTER,
+  PROJECT,
+  JOIN,
+  AGGREGATE,
+};
+
+const std::map<std::string, ArrowRelationType> enum_map{
+    {"scan", ArrowRelationType::SCAN},           {"filter", ArrowRelationType::FILTER},
+    {"project", ArrowRelationType::PROJECT},     {"join", ArrowRelationType::JOIN},
+    {"aggregate", ArrowRelationType::AGGREGATE},
+};
+
+struct ExtractRelation {
+  explicit ExtractRelation(substrait::Rel* rel, ExtensionSet* ext_set)
+      : rel_(rel), ext_set_(ext_set) {}
+
+  Status AddRelation(const compute::Declaration& declaration) {
+    const std::string& rel_name = declaration.factory_name;
+    switch (enum_map.find(rel_name)->second) {
+      case ArrowRelationType::SCAN:
+        return AddReadRelation(declaration);
+      case ArrowRelationType::FILTER:
+        return Status::NotImplemented("Filter operator not supported.");
+      case ArrowRelationType::PROJECT:
+        return Status::NotImplemented("Project operator not supported.");
+      case ArrowRelationType::JOIN:
+        return Status::NotImplemented("Join operator not supported.");
+      case ArrowRelationType::AGGREGATE:
+        return Status::NotImplemented("Aggregate operator not supported.");
+      default:
+        return Status::Invalid("Unsupported factory name :", rel_name);
+    }
+  }

Review Comment:
   It would increase readability / robustness but I don't think it's possible because we need to be open to users adding new node types and node factories beyond what Arrow provides.
   
   An enum would be slightly faster but converting a plan from Substrait to Acero is also something that happens once per query so it is ok if it takes a little bit of time.  If it were happening per-batch (or especially per-row) then it might be more of a consideration.



-- 
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] westonpace commented on a diff in pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
westonpace commented on code in PR #13401:
URL: https://github.com/apache/arrow/pull/13401#discussion_r914301585


##########
cpp/src/arrow/engine/substrait/serde_test.cc:
##########
@@ -1173,5 +1175,173 @@ TEST(Substrait, JoinPlanInvalidKeys) {
           &ext_set));
 }
 
+TEST(Substrait, SerializeRelation) {
+#ifdef _WIN32
+  GTEST_SKIP() << "ARROW-16392: Substrait File URI not supported for Windows";
+#else
+  ExtensionSet ext_set;
+  auto dummy_schema = schema({field("foo", binary())});
+  // creating a dummy dataset using a dummy table
+  auto format = std::make_shared<arrow::dataset::ParquetFileFormat>();
+  auto filesystem = std::make_shared<fs::LocalFileSystem>();

Review Comment:
   Please do try.  I would like to reduce our dependence on PARQUET_TEST_DATA as much as possible



-- 
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] vibhatha closed pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
vibhatha closed pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto
URL: https://github.com/apache/arrow/pull/13401


-- 
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] vibhatha commented on a diff in pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
vibhatha commented on code in PR #13401:
URL: https://github.com/apache/arrow/pull/13401#discussion_r956536925


##########
cpp/src/arrow/engine/substrait/serde_test.cc:
##########
@@ -1383,5 +1400,188 @@ TEST(Substrait, JoinPlanInvalidKeys) {
   }
 }
 
+TEST(Substrait, SerializeRelation) {
+#ifdef _WIN32
+  GTEST_SKIP() << "ARROW-16392: Substrait File URI not supported for Windows";
+#else
+  ExtensionSet ext_set;
+  auto dummy_schema = schema({field("foo", binary())});
+  // creating a dummy dataset using a dummy table
+  auto format = std::make_shared<arrow::dataset::ParquetFileFormat>();
+  auto filesystem = std::make_shared<fs::LocalFileSystem>();
+
+  std::vector<fs::FileInfo> files;
+  const std::vector<std::string> f_paths = {"/tmp/data1.parquet", "/tmp/data2.parquet"};
+
+  for (const auto& f_path : f_paths) {
+    ASSERT_OK_AND_ASSIGN(auto f_file, filesystem->GetFileInfo(f_path));
+    files.push_back(std::move(f_file));
+  }
+
+  ASSERT_OK_AND_ASSIGN(auto ds_factory, dataset::FileSystemDatasetFactory::Make(
+                                            std::move(filesystem), std::move(files),
+                                            std::move(format), {}));
+  ASSERT_OK_AND_ASSIGN(auto dataset, ds_factory->Finish(dummy_schema));
+
+  auto options = std::make_shared<dataset::ScanOptions>();
+  options->projection = compute::project({}, {});
+  auto scan_node_options = dataset::ScanNodeOptions{dataset, options};
+
+  auto scan_declaration = compute::Declaration({"scan", scan_node_options});
+
+  ASSERT_OK_AND_ASSIGN(auto serialized_rel,
+                       SerializeRelation(scan_declaration, &ext_set));
+  ASSERT_OK_AND_ASSIGN(auto deserialized_decl,
+                       DeserializeRelation(*serialized_rel, ext_set));
+
+  auto dataset_comparator = [](std::shared_ptr<dataset::Dataset> ds_lhs,
+                               std::shared_ptr<dataset::Dataset> ds_rhs) -> bool {
+    const auto& fsd_lhs = checked_cast<const dataset::FileSystemDataset&>(*ds_lhs);
+    const auto& fsd_rhs = checked_cast<const dataset::FileSystemDataset&>(*ds_lhs);
+    const auto& files_lhs = fsd_lhs.files();
+    const auto& files_rhs = fsd_rhs.files();
+
+    if (files_lhs.size() != files_rhs.size()) {
+      return false;
+    }
+    uint64_t fidx = 0;
+    for (const auto& l_file : files_lhs) {
+      if (l_file != files_rhs[fidx]) {
+        return false;
+      }
+      fidx++;
+    }
+    bool cmp_file_format = fsd_lhs.format()->Equals(*fsd_lhs.format());
+    bool cmp_file_system = fsd_lhs.filesystem()->Equals(fsd_rhs.filesystem());
+    return cmp_file_format && cmp_file_system;
+  };
+
+  auto scan_option_comparator = [dataset_comparator](
+                                    const dataset::ScanNodeOptions& lhs,
+                                    const dataset::ScanNodeOptions& rhs) -> bool {
+    bool cmp_rso = lhs.require_sequenced_output == rhs.require_sequenced_output;
+    bool cmp_ds = dataset_comparator(lhs.dataset, rhs.dataset);
+    return cmp_rso && cmp_ds;
+  };
+
+  EXPECT_EQ(deserialized_decl.factory_name, scan_declaration.factory_name);
+  const auto& lhs =
+      checked_cast<const dataset::ScanNodeOptions&>(*deserialized_decl.options);
+  const auto& rhs =
+      checked_cast<const dataset::ScanNodeOptions&>(*scan_declaration.options);
+  ASSERT_TRUE(scan_option_comparator(lhs, rhs));
+#endif
+}
+
+TEST(Substrait, SerializeRelationEndToEnd) {
+#ifdef _WIN32
+  GTEST_SKIP() << "ARROW-16392: Substrait File URI not supported for Windows";
+#else
+  ExtensionSet ext_set;
+  compute::ExecContext exec_context;
+
+  auto dummy_schema = schema({field("a", int32()), field("b", int32())});
+  auto table = TableFromJSON(dummy_schema, {R"([
+    [1, 1],
+    [3, 4]
+                        ])",
+                                            R"([
+    [0, 2],
+    [1, 3],
+    [4, 1],
+    [3, 1],
+    [1, 2]
+                        ])",
+                                            R"([
+    [2, 2],
+    [5, 3],
+    [1, 3]
+                        ])"});

Review Comment:
   ping: when I reformatted it comes to this shape. I tried a few times. Not sure what's wrong.



-- 
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] vibhatha commented on a diff in pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
vibhatha commented on code in PR #13401:
URL: https://github.com/apache/arrow/pull/13401#discussion_r956538375


##########
cpp/src/arrow/engine/substrait/relation_internal.cc:
##########
@@ -29,10 +29,20 @@
 #include "arrow/filesystem/localfs.h"
 #include "arrow/filesystem/path_util.h"
 #include "arrow/filesystem/util_internal.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/make_unique.h"
 
 namespace arrow {
+
+using internal::checked_cast;
+using internal::make_unique;
+
 namespace engine {
 
+using SubstraitConverter = std::function<Result<std::unique_ptr<substrait::Rel>>(
+    const std::shared_ptr<Schema>&, const compute::Declaration&, ExtensionSet*,
+    const ConversionOptions&)>;
+

Review Comment:
   Yes we used it before with the `registry` but kept it for the above mentioned reason. 



-- 
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] vibhatha commented on a diff in pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
vibhatha commented on code in PR #13401:
URL: https://github.com/apache/arrow/pull/13401#discussion_r956538389


##########
cpp/src/arrow/engine/substrait/serde_test.cc:
##########
@@ -42,9 +55,35 @@ using testing::UnorderedElementsAre;
 namespace arrow {
 
 using internal::checked_cast;
-
+using internal::hash_combine;
 namespace engine {
 
+Status WriteParquetData(const std::string& path,
+                        const std::shared_ptr<fs::FileSystem> file_system,
+                        const std::shared_ptr<Table> input) {
+  EXPECT_OK_AND_ASSIGN(auto buffer_writer, file_system->OpenOutputStream(path));
+  PARQUET_THROW_NOT_OK(parquet::arrow::WriteTable(*input, arrow::default_memory_pool(),
+                                                  buffer_writer, /*chunk_size*/ 1));
+  return buffer_writer->Close();
+}

Review Comment:
   Let me take a look. 



-- 
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] vibhatha commented on pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
vibhatha commented on PR #13401:
URL: https://github.com/apache/arrow/pull/13401#issuecomment-1239515059

   @westonpace I added a fix for the path issue on Mac. I think now it is more generalized. 
   
   Any other suggestions?


-- 
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] westonpace commented on pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
westonpace commented on PR #13401:
URL: https://github.com/apache/arrow/pull/13401#issuecomment-1205886633

   I think the goal for a registry was to support conversion of custom exec nodes to Substrait.  I agree that we don't want to expose Substrait internal classes though.  @bkietz any ideas?  This probably isn't the biggest priority at the moment if we can't find a way to support 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.

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

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


[GitHub] [arrow] vibhatha commented on pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
vibhatha commented on PR #13401:
URL: https://github.com/apache/arrow/pull/13401#issuecomment-1232342350

   > @westonpace Re: [#13401 (comment)](https://github.com/apache/arrow/pull/13401#discussion_r956480272)
   > 
   > The URL is like `/var/<some_path>/T//data.parquet`
   
   @westonpace tracking down the issue
   
   ```bash
   Tempdir path : /var/folders/vq/2v5vdv2n1t9b_fp4lp17h3d40000gn/T//3mpj4aud/
   File Path : /var/folders/vq/2v5vdv2n1t9b_fp4lp17h3d40000gn/T//3mpj4aud/serde_test.parquet
   /Users/vibhatha/github/fork/arrow/cpp/src/arrow/filesystem/path_util.cc:104:  Check failed: !stem.empty() 
   0   libarrow.1000.0.0.dylib             0x0000000111b06aa0 _ZN5arrow4util7CerrLog14PrintBackTraceEv + 52
   1   libarrow.1000.0.0.dylib             0x0000000111b06a54 _ZN5arrow4util7CerrLogD2Ev + 96
   2   libarrow.1000.0.0.dylib             0x0000000111b069b4 _ZN5arrow4util7CerrLogD1Ev + 28
   3   libarrow.1000.0.0.dylib             0x0000000111b069e0 _ZN5arrow4util7CerrLogD0Ev + 28
   4   libarrow.1000.0.0.dylib             0x0000000111b068b0 _ZN5arrow4util8ArrowLogD2Ev + 84
   5   libarrow.1000.0.0.dylib             0x0000000111b068ec _ZN5arrow4util8ArrowLogD1Ev + 28
   ```
   
   When the tempdir is created, that path contains this `/T//` component in the URI. 
   And this could be a MAC thing as I am developing on Mac M1. It is better to file a JIRA for this IMHO. 
   
   Should we fix it first and then fix this PR? Or put a macro with the ticket id to resolve 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.

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

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


[GitHub] [arrow] vibhatha commented on a diff in pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
vibhatha commented on code in PR #13401:
URL: https://github.com/apache/arrow/pull/13401#discussion_r904933144


##########
cpp/src/arrow/engine/substrait/serde_test.cc:
##########
@@ -1173,5 +1175,168 @@ TEST(Substrait, JoinPlanInvalidKeys) {
           &ext_set));
 }
 
+TEST(Substrait, SerializeRelation) {
+#ifdef _WIN32
+  GTEST_SKIP() << "ARROW-16392: Substrait File URI not supported for Windows";
+#else
+  ExtensionSet ext_set;
+  auto dummy_schema = schema({field("f1", int32()), field("f2", int32())});
+  // creating a dummy dataset using a dummy table
+  auto format = std::make_shared<arrow::dataset::ParquetFileFormat>();
+  auto filesystem = std::make_shared<fs::LocalFileSystem>();
+
+  std::vector<fs::FileInfo> files;
+  const std::vector<std::string> f_paths = {"/tmp/data1.parquet", "/tmp/data2.parquet"};
+
+  for (const auto& f_path : f_paths) {
+    ASSERT_OK_AND_ASSIGN(auto f_file, filesystem->GetFileInfo(f_path));
+    files.push_back(std::move(f_file));
+  }
+
+  ASSERT_OK_AND_ASSIGN(auto ds_factory, dataset::FileSystemDatasetFactory::Make(
+                                            std::move(filesystem), std::move(files),
+                                            std::move(format), {}));
+  ASSERT_OK_AND_ASSIGN(auto dataset, ds_factory->Finish(dummy_schema));
+
+  auto options = std::make_shared<dataset::ScanOptions>();
+  options->projection = compute::project({}, {});
+  auto scan_node_options = dataset::ScanNodeOptions{dataset, options};
+
+  auto scan_declaration = compute::Declaration({"scan", scan_node_options});
+
+  ASSERT_OK_AND_ASSIGN(auto serialized_rel,
+                       SerializeRelation(scan_declaration, &ext_set));
+  ASSERT_OK_AND_ASSIGN(auto deserialized_decl,
+                       DeserializeRelation(*serialized_rel, ext_set));
+
+  auto dataset_comparator = [](std::shared_ptr<dataset::Dataset> ds_lhs,
+                               std::shared_ptr<dataset::Dataset> ds_rhs) -> bool {
+    const auto& fds_lhs = checked_cast<const dataset::FileSystemDataset&>(*ds_lhs);
+    const auto& fds_rhs = checked_cast<const dataset::FileSystemDataset&>(*ds_lhs);
+    const auto& files_lhs = fds_lhs.files();
+    const auto& files_rhs = fds_rhs.files();
+
+    bool cmp_fsize = files_lhs.size() == files_rhs.size();
+    uint64_t fidx = 0;
+    for (const auto& l_file : files_lhs) {
+      if (l_file != files_rhs[fidx]) {
+        return false;
+      }
+      fidx++;
+    }
+    bool cmp_file_format = fds_lhs.format()->Equals(*fds_lhs.format());
+    bool cmp_file_system = fds_lhs.filesystem()->Equals(fds_rhs.filesystem());
+    return cmp_fsize && cmp_file_format && cmp_file_system;
+  };
+
+  auto scan_option_comparator = [dataset_comparator](
+                                    const dataset::ScanNodeOptions& lhs,
+                                    const dataset::ScanNodeOptions& rhs) -> bool {
+    bool cmp_rso = lhs.require_sequenced_output == rhs.require_sequenced_output;
+    bool cmp_ds = dataset_comparator(lhs.dataset, rhs.dataset);
+    return cmp_rso && cmp_ds;
+  };

Review Comment:
   @lidavidm 
   Regarding the `Equals` methods required, 
   
   For `FileSystemDataset` we need to consider the, `FileFormat` (`Equals` available), `FileSystem` (`Equals` available), `FileFragment` (`Equals` TODO add method straightforward), `Partitioning` (`Equals` TODO needs to Add method for Base class and for child classes) and `FragmentSubtree` (`Equals` TODO add method - straightforward)
   
   Is it still okay to add these change in the same 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.

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

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


[GitHub] [arrow] vibhatha commented on a diff in pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
vibhatha commented on code in PR #13401:
URL: https://github.com/apache/arrow/pull/13401#discussion_r904934765


##########
cpp/src/arrow/engine/substrait/serde_test.cc:
##########
@@ -1173,5 +1175,168 @@ TEST(Substrait, JoinPlanInvalidKeys) {
           &ext_set));
 }
 
+TEST(Substrait, SerializeRelation) {
+#ifdef _WIN32
+  GTEST_SKIP() << "ARROW-16392: Substrait File URI not supported for Windows";
+#else
+  ExtensionSet ext_set;
+  auto dummy_schema = schema({field("f1", int32()), field("f2", int32())});
+  // creating a dummy dataset using a dummy table
+  auto format = std::make_shared<arrow::dataset::ParquetFileFormat>();
+  auto filesystem = std::make_shared<fs::LocalFileSystem>();
+
+  std::vector<fs::FileInfo> files;
+  const std::vector<std::string> f_paths = {"/tmp/data1.parquet", "/tmp/data2.parquet"};
+
+  for (const auto& f_path : f_paths) {
+    ASSERT_OK_AND_ASSIGN(auto f_file, filesystem->GetFileInfo(f_path));
+    files.push_back(std::move(f_file));
+  }
+
+  ASSERT_OK_AND_ASSIGN(auto ds_factory, dataset::FileSystemDatasetFactory::Make(
+                                            std::move(filesystem), std::move(files),
+                                            std::move(format), {}));
+  ASSERT_OK_AND_ASSIGN(auto dataset, ds_factory->Finish(dummy_schema));
+
+  auto options = std::make_shared<dataset::ScanOptions>();
+  options->projection = compute::project({}, {});
+  auto scan_node_options = dataset::ScanNodeOptions{dataset, options};
+
+  auto scan_declaration = compute::Declaration({"scan", scan_node_options});
+
+  ASSERT_OK_AND_ASSIGN(auto serialized_rel,
+                       SerializeRelation(scan_declaration, &ext_set));
+  ASSERT_OK_AND_ASSIGN(auto deserialized_decl,
+                       DeserializeRelation(*serialized_rel, ext_set));
+
+  auto dataset_comparator = [](std::shared_ptr<dataset::Dataset> ds_lhs,
+                               std::shared_ptr<dataset::Dataset> ds_rhs) -> bool {
+    const auto& fds_lhs = checked_cast<const dataset::FileSystemDataset&>(*ds_lhs);
+    const auto& fds_rhs = checked_cast<const dataset::FileSystemDataset&>(*ds_lhs);
+    const auto& files_lhs = fds_lhs.files();
+    const auto& files_rhs = fds_rhs.files();
+
+    bool cmp_fsize = files_lhs.size() == files_rhs.size();
+    uint64_t fidx = 0;
+    for (const auto& l_file : files_lhs) {
+      if (l_file != files_rhs[fidx]) {
+        return false;
+      }
+      fidx++;
+    }
+    bool cmp_file_format = fds_lhs.format()->Equals(*fds_lhs.format());
+    bool cmp_file_system = fds_lhs.filesystem()->Equals(fds_rhs.filesystem());
+    return cmp_fsize && cmp_file_format && cmp_file_system;
+  };
+
+  auto scan_option_comparator = [dataset_comparator](
+                                    const dataset::ScanNodeOptions& lhs,
+                                    const dataset::ScanNodeOptions& rhs) -> bool {
+    bool cmp_rso = lhs.require_sequenced_output == rhs.require_sequenced_output;
+    bool cmp_ds = dataset_comparator(lhs.dataset, rhs.dataset);
+    return cmp_rso && cmp_ds;
+  };

Review Comment:
   Or is it satisfactory to just write them as test utils. I assume if these `Equals` are not going to be widely used, we can add it to test utils. 



-- 
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] vibhatha commented on a diff in pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
vibhatha commented on code in PR #13401:
URL: https://github.com/apache/arrow/pull/13401#discussion_r904939792


##########
cpp/src/arrow/engine/substrait/serde_test.cc:
##########
@@ -1173,5 +1175,168 @@ TEST(Substrait, JoinPlanInvalidKeys) {
           &ext_set));
 }
 
+TEST(Substrait, SerializeRelation) {
+#ifdef _WIN32
+  GTEST_SKIP() << "ARROW-16392: Substrait File URI not supported for Windows";
+#else
+  ExtensionSet ext_set;
+  auto dummy_schema = schema({field("f1", int32()), field("f2", int32())});
+  // creating a dummy dataset using a dummy table
+  auto format = std::make_shared<arrow::dataset::ParquetFileFormat>();
+  auto filesystem = std::make_shared<fs::LocalFileSystem>();
+
+  std::vector<fs::FileInfo> files;
+  const std::vector<std::string> f_paths = {"/tmp/data1.parquet", "/tmp/data2.parquet"};
+
+  for (const auto& f_path : f_paths) {
+    ASSERT_OK_AND_ASSIGN(auto f_file, filesystem->GetFileInfo(f_path));
+    files.push_back(std::move(f_file));
+  }
+
+  ASSERT_OK_AND_ASSIGN(auto ds_factory, dataset::FileSystemDatasetFactory::Make(
+                                            std::move(filesystem), std::move(files),
+                                            std::move(format), {}));
+  ASSERT_OK_AND_ASSIGN(auto dataset, ds_factory->Finish(dummy_schema));
+
+  auto options = std::make_shared<dataset::ScanOptions>();
+  options->projection = compute::project({}, {});
+  auto scan_node_options = dataset::ScanNodeOptions{dataset, options};
+
+  auto scan_declaration = compute::Declaration({"scan", scan_node_options});
+
+  ASSERT_OK_AND_ASSIGN(auto serialized_rel,
+                       SerializeRelation(scan_declaration, &ext_set));
+  ASSERT_OK_AND_ASSIGN(auto deserialized_decl,
+                       DeserializeRelation(*serialized_rel, ext_set));
+
+  auto dataset_comparator = [](std::shared_ptr<dataset::Dataset> ds_lhs,
+                               std::shared_ptr<dataset::Dataset> ds_rhs) -> bool {
+    const auto& fds_lhs = checked_cast<const dataset::FileSystemDataset&>(*ds_lhs);
+    const auto& fds_rhs = checked_cast<const dataset::FileSystemDataset&>(*ds_lhs);
+    const auto& files_lhs = fds_lhs.files();
+    const auto& files_rhs = fds_rhs.files();
+
+    bool cmp_fsize = files_lhs.size() == files_rhs.size();
+    uint64_t fidx = 0;
+    for (const auto& l_file : files_lhs) {
+      if (l_file != files_rhs[fidx]) {
+        return false;
+      }
+      fidx++;
+    }
+    bool cmp_file_format = fds_lhs.format()->Equals(*fds_lhs.format());
+    bool cmp_file_system = fds_lhs.filesystem()->Equals(fds_rhs.filesystem());
+    return cmp_fsize && cmp_file_format && cmp_file_system;
+  };
+
+  auto scan_option_comparator = [dataset_comparator](
+                                    const dataset::ScanNodeOptions& lhs,
+                                    const dataset::ScanNodeOptions& rhs) -> bool {
+    bool cmp_rso = lhs.require_sequenced_output == rhs.require_sequenced_output;
+    bool cmp_ds = dataset_comparator(lhs.dataset, rhs.dataset);
+    return cmp_rso && cmp_ds;
+  };

Review Comment:
   Let's add them in the same PR then. Otherwise it would complicate improving this PR simultaneously. It is not very complicated, but for the organisation,  these could have been written as supportive functions separately. 



-- 
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] lidavidm commented on a diff in pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
lidavidm commented on code in PR #13401:
URL: https://github.com/apache/arrow/pull/13401#discussion_r904936486


##########
cpp/src/arrow/engine/substrait/serde_test.cc:
##########
@@ -1173,5 +1175,168 @@ TEST(Substrait, JoinPlanInvalidKeys) {
           &ext_set));
 }
 
+TEST(Substrait, SerializeRelation) {
+#ifdef _WIN32
+  GTEST_SKIP() << "ARROW-16392: Substrait File URI not supported for Windows";
+#else
+  ExtensionSet ext_set;
+  auto dummy_schema = schema({field("f1", int32()), field("f2", int32())});
+  // creating a dummy dataset using a dummy table
+  auto format = std::make_shared<arrow::dataset::ParquetFileFormat>();
+  auto filesystem = std::make_shared<fs::LocalFileSystem>();
+
+  std::vector<fs::FileInfo> files;
+  const std::vector<std::string> f_paths = {"/tmp/data1.parquet", "/tmp/data2.parquet"};
+
+  for (const auto& f_path : f_paths) {
+    ASSERT_OK_AND_ASSIGN(auto f_file, filesystem->GetFileInfo(f_path));
+    files.push_back(std::move(f_file));
+  }
+
+  ASSERT_OK_AND_ASSIGN(auto ds_factory, dataset::FileSystemDatasetFactory::Make(
+                                            std::move(filesystem), std::move(files),
+                                            std::move(format), {}));
+  ASSERT_OK_AND_ASSIGN(auto dataset, ds_factory->Finish(dummy_schema));
+
+  auto options = std::make_shared<dataset::ScanOptions>();
+  options->projection = compute::project({}, {});
+  auto scan_node_options = dataset::ScanNodeOptions{dataset, options};
+
+  auto scan_declaration = compute::Declaration({"scan", scan_node_options});
+
+  ASSERT_OK_AND_ASSIGN(auto serialized_rel,
+                       SerializeRelation(scan_declaration, &ext_set));
+  ASSERT_OK_AND_ASSIGN(auto deserialized_decl,
+                       DeserializeRelation(*serialized_rel, ext_set));
+
+  auto dataset_comparator = [](std::shared_ptr<dataset::Dataset> ds_lhs,
+                               std::shared_ptr<dataset::Dataset> ds_rhs) -> bool {
+    const auto& fds_lhs = checked_cast<const dataset::FileSystemDataset&>(*ds_lhs);
+    const auto& fds_rhs = checked_cast<const dataset::FileSystemDataset&>(*ds_lhs);
+    const auto& files_lhs = fds_lhs.files();
+    const auto& files_rhs = fds_rhs.files();
+
+    bool cmp_fsize = files_lhs.size() == files_rhs.size();
+    uint64_t fidx = 0;
+    for (const auto& l_file : files_lhs) {
+      if (l_file != files_rhs[fidx]) {
+        return false;
+      }
+      fidx++;
+    }
+    bool cmp_file_format = fds_lhs.format()->Equals(*fds_lhs.format());
+    bool cmp_file_system = fds_lhs.filesystem()->Equals(fds_rhs.filesystem());
+    return cmp_fsize && cmp_file_format && cmp_file_system;
+  };
+
+  auto scan_option_comparator = [dataset_comparator](
+                                    const dataset::ScanNodeOptions& lhs,
+                                    const dataset::ScanNodeOptions& rhs) -> bool {
+    bool cmp_rso = lhs.require_sequenced_output == rhs.require_sequenced_output;
+    bool cmp_ds = dataset_comparator(lhs.dataset, rhs.dataset);
+    return cmp_rso && cmp_ds;
+  };

Review Comment:
   Again, it's up to your best judgement. I would have thought these are mostly straightforward implementations, but if not and  you feel like it would complicate things too much, then just leave them as is.



-- 
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] vibhatha commented on a diff in pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
vibhatha commented on code in PR #13401:
URL: https://github.com/apache/arrow/pull/13401#discussion_r907457069


##########
cpp/src/arrow/engine/substrait/serde_test.cc:
##########
@@ -1173,5 +1175,173 @@ TEST(Substrait, JoinPlanInvalidKeys) {
           &ext_set));
 }
 
+TEST(Substrait, SerializeRelation) {
+#ifdef _WIN32
+  GTEST_SKIP() << "ARROW-16392: Substrait File URI not supported for Windows";
+#else
+  ExtensionSet ext_set;
+  auto dummy_schema = schema({field("foo", binary())});
+  // creating a dummy dataset using a dummy table
+  auto format = std::make_shared<arrow::dataset::ParquetFileFormat>();
+  auto filesystem = std::make_shared<fs::LocalFileSystem>();
+
+  ASSERT_OK_AND_ASSIGN(std::string dir_string,
+                       arrow::internal::GetEnvVar("PARQUET_TEST_DATA"));
+  auto file_name =
+      arrow::internal::PlatformFilename::FromString(dir_string)->Join("binary.parquet");
+
+  std::vector<fs::FileInfo> files;
+  const std::vector<std::string> f_paths = {file_name->ToString()};
+
+  for (const auto& f_path : f_paths) {
+    ASSERT_OK_AND_ASSIGN(auto f_file, filesystem->GetFileInfo(f_path));
+    files.push_back(std::move(f_file));
+  }
+
+  ASSERT_OK_AND_ASSIGN(auto ds_factory, dataset::FileSystemDatasetFactory::Make(
+                                            std::move(filesystem), std::move(files),
+                                            std::move(format), {}));
+  ASSERT_OK_AND_ASSIGN(auto dataset, ds_factory->Finish(dummy_schema));
+
+  auto options = std::make_shared<dataset::ScanOptions>();
+  options->projection = compute::project({}, {});
+  auto scan_node_options = dataset::ScanNodeOptions{dataset, options};
+
+  auto scan_declaration = compute::Declaration({"scan", scan_node_options});
+
+  ASSERT_OK_AND_ASSIGN(auto serialized_rel,
+                       SerializeRelation(scan_declaration, &ext_set));
+  ASSERT_OK_AND_ASSIGN(auto deserialized_decl,
+                       DeserializeRelation(*serialized_rel, ext_set));
+
+  auto dataset_comparator = [](std::shared_ptr<dataset::Dataset> ds_lhs,
+                               std::shared_ptr<dataset::Dataset> ds_rhs) -> bool {
+    const auto& fds_lhs = checked_cast<const dataset::FileSystemDataset&>(*ds_lhs);
+    const auto& fds_rhs = checked_cast<const dataset::FileSystemDataset&>(*ds_lhs);
+    const auto& files_lhs = fds_lhs.files();
+    const auto& files_rhs = fds_rhs.files();
+
+    bool cmp_fsize = files_lhs.size() == files_rhs.size();
+    uint64_t fidx = 0;
+    for (const auto& l_file : files_lhs) {
+      if (l_file != files_rhs[fidx]) {
+        return false;
+      }
+      fidx++;
+    }
+    bool cmp_file_format = fds_lhs.format()->Equals(*fds_lhs.format());
+    bool cmp_file_system = fds_lhs.filesystem()->Equals(fds_rhs.filesystem());
+    return cmp_fsize && cmp_file_format && cmp_file_system;
+  };
+
+  auto scan_option_comparator = [dataset_comparator](
+                                    const dataset::ScanNodeOptions& lhs,
+                                    const dataset::ScanNodeOptions& rhs) -> bool {
+    bool cmp_rso = lhs.require_sequenced_output == rhs.require_sequenced_output;
+    bool cmp_ds = dataset_comparator(lhs.dataset, rhs.dataset);
+    return cmp_rso && cmp_ds;
+  };
+
+  EXPECT_EQ(deserialized_decl.factory_name, scan_declaration.factory_name);
+  const auto& lhs =
+      checked_cast<const dataset::ScanNodeOptions&>(*deserialized_decl.options);
+  const auto& rhs =
+      checked_cast<const dataset::ScanNodeOptions&>(*scan_declaration.options);
+  ASSERT_TRUE(scan_option_comparator(lhs, rhs));
+#endif
+}
+
+TEST(Substrait, SerializeRelationEndToEnd) {

Review Comment:
   The previous test just compares the configs, this test is an end-to-end test which considers the output data from a given exec plan and ser/de exec plans output. It is just a validation on keeping the functionality intact.



-- 
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] vibhatha commented on a diff in pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
vibhatha commented on code in PR #13401:
URL: https://github.com/apache/arrow/pull/13401#discussion_r956537083


##########
cpp/src/arrow/engine/substrait/serde_test.cc:
##########
@@ -1731,5 +1808,265 @@ TEST(Substrait, AggregateWithFilter) {
   ASSERT_RAISES(NotImplemented, DeserializePlans(*buf, [] { return kNullConsumer; }));
 }
 
+TEST(Substrait, BasicPlanRoundTripping) {
+#ifdef _WIN32
+  GTEST_SKIP() << "ARROW-16392: Substrait File URI not supported for Windows";
+#else
+  compute::ExecContext exec_context;
+  ExtensionSet ext_set;
+  auto dummy_schema = schema(
+      {field("key", int32()), field("shared", int32()), field("distinct", int32())});
+
+  // creating a dummy dataset using a dummy table
+  auto table = TableFromJSON(dummy_schema, {R"([
+      [1, 1, 10],
+      [3, 4, 20]
+    ])",
+                                            R"([
+      [0, 2, 1],
+      [1, 3, 2],
+      [4, 1, 3],
+      [3, 1, 3],
+      [1, 2, 5]
+    ])",
+                                            R"([
+      [2, 2, 12],
+      [5, 3, 12],
+      [1, 3, 12]
+    ])"});
+
+  auto format = std::make_shared<arrow::dataset::ParquetFileFormat>();
+  auto filesystem = std::make_shared<fs::LocalFileSystem>();
+  const std::string file_name = "serde_test.parquet";
+
+  ASSERT_OK_AND_ASSIGN(auto tempdir,
+                       arrow::internal::TemporaryDir::Make("substrait_tempdir"));
+  ASSERT_OK_AND_ASSIGN(auto file_path, tempdir->path().Join(file_name));
+  std::string file_path_str = file_path.ToString();
+
+  // Note: there is an additional forward slash introduced by the tempdir
+  // it must be replaced to properly load into reading files
+  // TODO: (Review: Jira needs to be reported to handle this properly)

Review Comment:
   Yes, an example is 
   
   `/var/files/T//substrait_test_data.parquet`.
   
    Here with `/T//` it gives an error when files are discovered. 



-- 
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] vibhatha commented on a diff in pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
vibhatha commented on code in PR #13401:
URL: https://github.com/apache/arrow/pull/13401#discussion_r956915382


##########
cpp/src/arrow/engine/substrait/relation_internal.cc:
##########
@@ -421,5 +431,171 @@ Result<DeclarationInfo> FromProto(const substrait::Rel& rel, const ExtensionSet&
       rel.DebugString());
 }
 
+Result<std::unique_ptr<substrait::Rel>> ToProto(
+    const compute::Declaration& declr, ExtensionSet* ext_set,
+    const ConversionOptions& conversion_options) {
+  auto rel = make_unique<substrait::Rel>();
+  RETURN_NOT_OK(SerializeAndCombineRelations(declr, ext_set, rel, conversion_options));
+  return std::move(rel);
+}
+
+Status SetRelation(const std::unique_ptr<substrait::Rel>& plan,
+                   const std::unique_ptr<substrait::Rel>& partial_plan,
+                   const std::string& factory_name) {
+  if (factory_name == "scan" && partial_plan->has_read()) {
+    plan->set_allocated_read(partial_plan->release_read());
+  } else if (factory_name == "filter" && partial_plan->has_filter()) {
+    plan->set_allocated_filter(partial_plan->release_filter());
+  } else {
+    return Status::NotImplemented("Substrait converter ", factory_name,
+                                  " not supported.");
+  }
+  return Status::OK();
+}
+
+Result<std::shared_ptr<Schema>> ExtractSchemaToBind(const compute::Declaration& declr) {
+  std::shared_ptr<Schema> bind_schema;
+  if (declr.factory_name == "scan") {
+    const auto& opts = checked_cast<const dataset::ScanNodeOptions&>(*(declr.options));
+    bind_schema = opts.dataset->schema();
+  } else if (declr.factory_name == "filter") {
+    auto input_declr = util::get<compute::Declaration>(declr.inputs[0]);
+    ARROW_ASSIGN_OR_RAISE(bind_schema, ExtractSchemaToBind(input_declr));
+  } else if (declr.factory_name == "sink") {
+    // Note that the sink has no output_schema
+    return bind_schema;
+  } else {
+    return Status::Invalid("Schema extraction failed, unsupported factory ",
+                           declr.factory_name);
+  }
+  return bind_schema;
+}
+
+Status SerializeAndCombineRelations(const compute::Declaration& declaration,
+                                    ExtensionSet* ext_set,
+                                    std::unique_ptr<substrait::Rel>& rel,
+                                    const ConversionOptions& conversion_options) {
+  std::vector<compute::Declaration::Input> inputs = declaration.inputs;
+  for (auto& input : inputs) {
+    auto input_decl = util::get<compute::Declaration>(input);
+    RETURN_NOT_OK(
+        SerializeAndCombineRelations(input_decl, ext_set, rel, conversion_options));
+  }
+  const auto& factory_name = declaration.factory_name;
+  ARROW_ASSIGN_OR_RAISE(auto schema, ExtractSchemaToBind(declaration));
+  // Note that the sink declaration factory doesn't exist for serialization as
+  // Substrait doesn't deal with a sink node definition
+  std::unique_ptr<substrait::Rel> factory_rel;
+  if (factory_name == "scan") {
+    ARROW_ASSIGN_OR_RAISE(factory_rel, ScanRelationConverter(schema, declaration, ext_set,
+                                                             conversion_options));
+  } else if (factory_name == "filter") {
+    ARROW_ASSIGN_OR_RAISE(
+        factory_rel,
+        FilterRelationConverter(schema, declaration, ext_set, conversion_options));
+  } else {
+    return Status::NotImplemented("Factory ", factory_name,
+                                  " not implemented for roundtripping.");
+  }
+
+  if (factory_rel != nullptr) {
+    RETURN_NOT_OK(SetRelation(rel, factory_rel, factory_name));
+  } else {
+    return Status::Invalid("Conversion on factory ", factory_name,
+                           " returned an invalid relation");
+  }
+  return Status::OK();
+}
+
+Result<std::unique_ptr<substrait::Rel>> GetRelationFromDeclaration(
+    const compute::Declaration& declaration, ExtensionSet* ext_set,
+    const ConversionOptions& conversion_options) {
+  auto declr_input = declaration.inputs[0];
+  // Note that the input is expected in declaration.
+  // ExecNode inputs are not accepted
+  if (util::get_if<compute::ExecNode*>(&declr_input)) {
+    return Status::NotImplemented("Only support Plans written in Declaration format.");
+  }
+  return ToProto(util::get<compute::Declaration>(declr_input), ext_set,
+                 conversion_options);
+}

Review Comment:
   What if the input of the `Declaration` consists of both `ExecNode` and `Declaration`? This was the doubt. 
   Since there is a `util::Variant` in the `Declaration` to accept the input, added this method to make sure the input is a `Declaration`. 
   
   I also have the doubt with taking the `declaration.inputs[0]`. May be iterate this and cover all inputs. But I saw in many nodes, this was hardcoded to be 0. But do you think my argument is valid regarding the possibility of `Input` being an `ExecNode`?
   
   WDYT?



-- 
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] vibhatha commented on a diff in pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
vibhatha commented on code in PR #13401:
URL: https://github.com/apache/arrow/pull/13401#discussion_r957017608


##########
cpp/src/arrow/engine/substrait/relation_internal.cc:
##########
@@ -421,5 +431,171 @@ Result<DeclarationInfo> FromProto(const substrait::Rel& rel, const ExtensionSet&
       rel.DebugString());
 }
 
+Result<std::unique_ptr<substrait::Rel>> ToProto(
+    const compute::Declaration& declr, ExtensionSet* ext_set,
+    const ConversionOptions& conversion_options) {
+  auto rel = make_unique<substrait::Rel>();
+  RETURN_NOT_OK(SerializeAndCombineRelations(declr, ext_set, rel, conversion_options));
+  return std::move(rel);
+}
+
+Status SetRelation(const std::unique_ptr<substrait::Rel>& plan,
+                   const std::unique_ptr<substrait::Rel>& partial_plan,
+                   const std::string& factory_name) {
+  if (factory_name == "scan" && partial_plan->has_read()) {
+    plan->set_allocated_read(partial_plan->release_read());
+  } else if (factory_name == "filter" && partial_plan->has_filter()) {
+    plan->set_allocated_filter(partial_plan->release_filter());
+  } else {
+    return Status::NotImplemented("Substrait converter ", factory_name,
+                                  " not supported.");
+  }
+  return Status::OK();
+}
+
+Result<std::shared_ptr<Schema>> ExtractSchemaToBind(const compute::Declaration& declr) {
+  std::shared_ptr<Schema> bind_schema;
+  if (declr.factory_name == "scan") {
+    const auto& opts = checked_cast<const dataset::ScanNodeOptions&>(*(declr.options));
+    bind_schema = opts.dataset->schema();
+  } else if (declr.factory_name == "filter") {
+    auto input_declr = util::get<compute::Declaration>(declr.inputs[0]);
+    ARROW_ASSIGN_OR_RAISE(bind_schema, ExtractSchemaToBind(input_declr));
+  } else if (declr.factory_name == "sink") {
+    // Note that the sink has no output_schema
+    return bind_schema;
+  } else {
+    return Status::Invalid("Schema extraction failed, unsupported factory ",
+                           declr.factory_name);
+  }
+  return bind_schema;
+}
+
+Status SerializeAndCombineRelations(const compute::Declaration& declaration,
+                                    ExtensionSet* ext_set,
+                                    std::unique_ptr<substrait::Rel>& rel,
+                                    const ConversionOptions& conversion_options) {
+  std::vector<compute::Declaration::Input> inputs = declaration.inputs;
+  for (auto& input : inputs) {
+    auto input_decl = util::get<compute::Declaration>(input);
+    RETURN_NOT_OK(
+        SerializeAndCombineRelations(input_decl, ext_set, rel, conversion_options));

Review Comment:
   You're correct. I updated such that each relation is setting it's own input depending on the nature of the node. For instance `sink` has only single input and `filter` also has a single input. That's handled in the updated logic.



-- 
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] vibhatha commented on a diff in pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
vibhatha commented on code in PR #13401:
URL: https://github.com/apache/arrow/pull/13401#discussion_r918449974


##########
cpp/src/arrow/engine/substrait/serde_test.cc:
##########
@@ -1383,5 +1400,188 @@ TEST(Substrait, JoinPlanInvalidKeys) {
   }
 }
 
+TEST(Substrait, SerializeRelation) {
+#ifdef _WIN32
+  GTEST_SKIP() << "ARROW-16392: Substrait File URI not supported for Windows";
+#else
+  ExtensionSet ext_set;
+  auto dummy_schema = schema({field("foo", binary())});
+  // creating a dummy dataset using a dummy table
+  auto format = std::make_shared<arrow::dataset::ParquetFileFormat>();
+  auto filesystem = std::make_shared<fs::LocalFileSystem>();
+
+  std::vector<fs::FileInfo> files;
+  const std::vector<std::string> f_paths = {"/tmp/data1.parquet", "/tmp/data2.parquet"};

Review Comment:
   There is a missing path support. 
   
   ```c++
   GTEST_SKIP() << "ARROW-16392: Substrait File URI not supported for Windows";
   ```



-- 
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] westonpace commented on a diff in pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
westonpace commented on code in PR #13401:
URL: https://github.com/apache/arrow/pull/13401#discussion_r914302131


##########
cpp/src/arrow/engine/substrait/relation_internal.cc:
##########
@@ -27,10 +27,17 @@
 #include "arrow/engine/substrait/type_internal.h"
 #include "arrow/filesystem/localfs.h"
 #include "arrow/filesystem/util_internal.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/make_unique.h"
 
 namespace arrow {
 namespace engine {
 
+namespace internal {
+using ::arrow::internal::checked_cast;
+using ::arrow::internal::make_unique;
+}  // namespace internal
+

Review Comment:
   ```suggestion
   namespace arrow {
   
   using internal::checked_cast;
   using internal::make_unique;
   
   namespace engine {
   
   ```



##########
cpp/src/arrow/engine/substrait/relation_internal.cc:
##########
@@ -316,5 +323,97 @@ Result<compute::Declaration> FromProto(const substrait::Rel& rel,
       rel.DebugString());
 }
 
+namespace {
+// TODO: add other types
+enum ArrowRelationType : uint8_t {
+  SCAN,
+  FILTER,
+  PROJECT,
+  JOIN,
+  AGGREGATE,
+};
+
+const std::map<std::string, ArrowRelationType> enum_map{
+    {"scan", ArrowRelationType::SCAN},           {"filter", ArrowRelationType::FILTER},
+    {"project", ArrowRelationType::PROJECT},     {"join", ArrowRelationType::JOIN},
+    {"aggregate", ArrowRelationType::AGGREGATE},
+};
+
+struct ExtractRelation {
+  explicit ExtractRelation(substrait::Rel* rel, ExtensionSet* ext_set)
+      : rel_(rel), ext_set_(ext_set) {}
+
+  Status AddRelation(const compute::Declaration& declaration) {
+    const std::string& rel_name = declaration.factory_name;
+    switch (enum_map.find(rel_name)->second) {
+      case ArrowRelationType::SCAN:
+        return AddReadRelation(declaration);
+      case ArrowRelationType::FILTER:
+        return Status::NotImplemented("Filter operator not supported.");
+      case ArrowRelationType::PROJECT:
+        return Status::NotImplemented("Project operator not supported.");
+      case ArrowRelationType::JOIN:
+        return Status::NotImplemented("Join operator not supported.");
+      case ArrowRelationType::AGGREGATE:
+        return Status::NotImplemented("Aggregate operator not supported.");
+      default:
+        return Status::Invalid("Unsupported factory name :", rel_name);
+    }
+  }
+
+  Status AddReadRelation(const compute::Declaration& declaration) {
+    auto read_rel = internal::make_unique<substrait::ReadRel>();
+    const auto& scan_node_options =
+        internal::checked_cast<const dataset::ScanNodeOptions&>(*declaration.options);
+
+    const auto& fds = internal::checked_cast<const dataset::FileSystemDataset&>(
+        *scan_node_options.dataset);
+
+    // set schema
+    ARROW_ASSIGN_OR_RAISE(auto named_struct, ToProto(*fds.schema(), ext_set_));
+    read_rel->set_allocated_base_schema(named_struct.release());
+
+    // set local files
+    auto read_rel_lfs = internal::make_unique<substrait::ReadRel_LocalFiles>();
+    for (const auto& file : fds.files()) {
+      auto read_rel_lfs_ffs =
+          internal::make_unique<substrait::ReadRel_LocalFiles_FileOrFiles>();
+      read_rel_lfs_ffs->set_uri_path("file://" + file);
+
+      // set file format
+      auto format_type_name = fds.format()->type_name();
+      if (format_type_name == "parquet" || format_type_name == "arrow" ||
+          format_type_name == "feather") {
+        read_rel_lfs_ffs->set_format(
+            substrait::ReadRel::LocalFiles::FileOrFiles::FILE_FORMAT_PARQUET);
+      } else {
+        return Status::Invalid("Unsupported file type : ", format_type_name);
+      }
+      read_rel_lfs->mutable_items()->AddAllocated(read_rel_lfs_ffs.release());
+    }
+    *read_rel->mutable_local_files() = *read_rel_lfs.get();
+
+    rel_->set_allocated_read(read_rel.release());
+    return Status::OK();
+  }
+
+  Status operator()(const compute::Declaration& declaration) {
+    return AddRelation(declaration);
+  }
+
+ private:
+  substrait::Rel* rel_;
+  ExtensionSet* ext_set_;
+};
+
+}  // namespace
+
+Result<std::unique_ptr<substrait::Rel>> ToProto(const compute::Declaration& declaration,
+                                                ExtensionSet* ext_set) {
+  auto out = internal::make_unique<substrait::Rel>();
+  RETURN_NOT_OK(ExtractRelation(out.get(), ext_set)(declaration));

Review Comment:
   Let's go with a lightweight function until it no longer works



-- 
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] vibhatha commented on a diff in pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
vibhatha commented on code in PR #13401:
URL: https://github.com/apache/arrow/pull/13401#discussion_r936680902


##########
cpp/src/arrow/engine/substrait/serde_test.cc:
##########
@@ -1731,5 +1808,265 @@ TEST(Substrait, AggregateWithFilter) {
   ASSERT_RAISES(NotImplemented, DeserializePlans(*buf, [] { return kNullConsumer; }));
 }
 
+TEST(Substrait, BasicPlanRoundTripping) {
+#ifdef _WIN32
+  GTEST_SKIP() << "ARROW-16392: Substrait File URI not supported for Windows";
+#else
+  compute::ExecContext exec_context;
+  ExtensionSet ext_set;
+  auto dummy_schema = schema(
+      {field("key", int32()), field("shared", int32()), field("distinct", int32())});
+
+  // creating a dummy dataset using a dummy table
+  auto table = TableFromJSON(dummy_schema, {R"([
+      [1, 1, 10],
+      [3, 4, 20]
+    ])",
+                                            R"([
+      [0, 2, 1],
+      [1, 3, 2],
+      [4, 1, 3],
+      [3, 1, 3],
+      [1, 2, 5]
+    ])",
+                                            R"([
+      [2, 2, 12],
+      [5, 3, 12],
+      [1, 3, 12]
+    ])"});
+
+  auto format = std::make_shared<arrow::dataset::ParquetFileFormat>();
+  auto filesystem = std::make_shared<fs::LocalFileSystem>();
+  const std::string file_name = "serde_test.parquet";
+
+  ASSERT_OK_AND_ASSIGN(auto tempdir,
+                       arrow::internal::TemporaryDir::Make("substrait_tempdir"));
+  ASSERT_OK_AND_ASSIGN(auto file_path, tempdir->path().Join(file_name));
+  std::string file_path_str = file_path.ToString();
+
+  // Note: there is an additional forward slash introduced by the tempdir
+  // it must be replaced to properly load into reading files
+  // TODO: (Review: Jira needs to be reported to handle this properly)

Review Comment:
   @westonpace what is the best here? Any suggestions?



-- 
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] vibhatha commented on pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
vibhatha commented on PR #13401:
URL: https://github.com/apache/arrow/pull/13401#issuecomment-1205452548

   > Thanks for this work! It'll be very useful in testing to have roundtrippable full plans. I do have some concerns wrt the large-scale structure of this patch, though:
   > 
   > Maybe I've missed something, but I don't see the necessity of wrapping this in a registry. We have a number of registries in arrow but it's a pattern with heavy overhead which we follow only when required by constraints of third party extensibility. In this case, since the protobuf message classes are a private/internal implementation detail, adding anything to this registry would require rebuilding arrow anyway. That being the case, please remove the registry and ensure that protobuf message classes remain internal.
   
   Thank you @bkietz for the detailed review. I was inclined towards the registry with the idea of putting the Substrait-to-Acero and Acero-to-Substrait to be unified under a single registry. Not sure if it is a good idea. And this idea is not reflected in this PR. 
   
   We can easily move for a non-registry approach it is no issue. 
   
   cc @westonpace for more thoughts on this. 


-- 
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] vibhatha commented on a diff in pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
vibhatha commented on code in PR #13401:
URL: https://github.com/apache/arrow/pull/13401#discussion_r907462064


##########
cpp/src/arrow/engine/substrait/relation_internal.cc:
##########
@@ -316,5 +323,97 @@ Result<compute::Declaration> FromProto(const substrait::Rel& rel,
       rel.DebugString());
 }
 
+namespace {
+// TODO: add other types
+enum ArrowRelationType : uint8_t {
+  SCAN,
+  FILTER,
+  PROJECT,
+  JOIN,
+  AGGREGATE,
+};
+
+const std::map<std::string, ArrowRelationType> enum_map{
+    {"scan", ArrowRelationType::SCAN},           {"filter", ArrowRelationType::FILTER},
+    {"project", ArrowRelationType::PROJECT},     {"join", ArrowRelationType::JOIN},
+    {"aggregate", ArrowRelationType::AGGREGATE},
+};
+
+struct ExtractRelation {
+  explicit ExtractRelation(substrait::Rel* rel, ExtensionSet* ext_set)
+      : rel_(rel), ext_set_(ext_set) {}
+
+  Status AddRelation(const compute::Declaration& declaration) {
+    const std::string& rel_name = declaration.factory_name;
+    switch (enum_map.find(rel_name)->second) {
+      case ArrowRelationType::SCAN:
+        return AddReadRelation(declaration);
+      case ArrowRelationType::FILTER:
+        return Status::NotImplemented("Filter operator not supported.");
+      case ArrowRelationType::PROJECT:
+        return Status::NotImplemented("Project operator not supported.");
+      case ArrowRelationType::JOIN:
+        return Status::NotImplemented("Join operator not supported.");
+      case ArrowRelationType::AGGREGATE:
+        return Status::NotImplemented("Aggregate operator not supported.");
+      default:
+        return Status::Invalid("Unsupported factory name :", rel_name);
+    }
+  }
+
+  Status AddReadRelation(const compute::Declaration& declaration) {
+    auto read_rel = internal::make_unique<substrait::ReadRel>();
+    const auto& scan_node_options =
+        internal::checked_cast<const dataset::ScanNodeOptions&>(*declaration.options);
+
+    const auto& fds = internal::checked_cast<const dataset::FileSystemDataset&>(
+        *scan_node_options.dataset);
+
+    // set schema
+    ARROW_ASSIGN_OR_RAISE(auto named_struct, ToProto(*fds.schema(), ext_set_));
+    read_rel->set_allocated_base_schema(named_struct.release());
+
+    // set local files
+    auto read_rel_lfs = internal::make_unique<substrait::ReadRel_LocalFiles>();
+    for (const auto& file : fds.files()) {
+      auto read_rel_lfs_ffs =
+          internal::make_unique<substrait::ReadRel_LocalFiles_FileOrFiles>();
+      read_rel_lfs_ffs->set_uri_path("file://" + file);
+
+      // set file format
+      auto format_type_name = fds.format()->type_name();
+      if (format_type_name == "parquet" || format_type_name == "arrow" ||
+          format_type_name == "feather") {
+        read_rel_lfs_ffs->set_format(
+            substrait::ReadRel::LocalFiles::FileOrFiles::FILE_FORMAT_PARQUET);
+      } else {
+        return Status::Invalid("Unsupported file type : ", format_type_name);
+      }
+      read_rel_lfs->mutable_items()->AddAllocated(read_rel_lfs_ffs.release());
+    }
+    *read_rel->mutable_local_files() = *read_rel_lfs.get();
+
+    rel_->set_allocated_read(read_rel.release());
+    return Status::OK();
+  }
+
+  Status operator()(const compute::Declaration& declaration) {
+    return AddRelation(declaration);
+  }
+
+ private:
+  substrait::Rel* rel_;
+  ExtensionSet* ext_set_;
+};
+
+}  // namespace
+
+Result<std::unique_ptr<substrait::Rel>> ToProto(const compute::Declaration& declaration,
+                                                ExtensionSet* ext_set) {
+  auto out = internal::make_unique<substrait::Rel>();
+  RETURN_NOT_OK(ExtractRelation(out.get(), ext_set)(declaration));

Review Comment:
   WDYT?
   



##########
cpp/src/arrow/engine/substrait/serde_test.cc:
##########
@@ -1173,5 +1175,173 @@ TEST(Substrait, JoinPlanInvalidKeys) {
           &ext_set));
 }
 
+TEST(Substrait, SerializeRelation) {
+#ifdef _WIN32
+  GTEST_SKIP() << "ARROW-16392: Substrait File URI not supported for Windows";
+#else
+  ExtensionSet ext_set;
+  auto dummy_schema = schema({field("foo", binary())});
+  // creating a dummy dataset using a dummy table
+  auto format = std::make_shared<arrow::dataset::ParquetFileFormat>();
+  auto filesystem = std::make_shared<fs::LocalFileSystem>();

Review Comment:
   WDYT?



-- 
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] vibhatha commented on a diff in pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
vibhatha commented on code in PR #13401:
URL: https://github.com/apache/arrow/pull/13401#discussion_r907476465


##########
cpp/src/arrow/engine/substrait/serde_test.cc:
##########
@@ -1173,5 +1175,173 @@ TEST(Substrait, JoinPlanInvalidKeys) {
           &ext_set));
 }
 
+TEST(Substrait, SerializeRelation) {
+#ifdef _WIN32
+  GTEST_SKIP() << "ARROW-16392: Substrait File URI not supported for Windows";
+#else
+  ExtensionSet ext_set;
+  auto dummy_schema = schema({field("foo", binary())});
+  // creating a dummy dataset using a dummy table
+  auto format = std::make_shared<arrow::dataset::ParquetFileFormat>();
+  auto filesystem = std::make_shared<fs::LocalFileSystem>();
+
+  ASSERT_OK_AND_ASSIGN(std::string dir_string,
+                       arrow::internal::GetEnvVar("PARQUET_TEST_DATA"));
+  auto file_name =
+      arrow::internal::PlatformFilename::FromString(dir_string)->Join("binary.parquet");
+
+  std::vector<fs::FileInfo> files;
+  const std::vector<std::string> f_paths = {file_name->ToString()};
+
+  for (const auto& f_path : f_paths) {
+    ASSERT_OK_AND_ASSIGN(auto f_file, filesystem->GetFileInfo(f_path));
+    files.push_back(std::move(f_file));
+  }
+
+  ASSERT_OK_AND_ASSIGN(auto ds_factory, dataset::FileSystemDatasetFactory::Make(
+                                            std::move(filesystem), std::move(files),
+                                            std::move(format), {}));
+  ASSERT_OK_AND_ASSIGN(auto dataset, ds_factory->Finish(dummy_schema));
+
+  auto options = std::make_shared<dataset::ScanOptions>();
+  options->projection = compute::project({}, {});
+  auto scan_node_options = dataset::ScanNodeOptions{dataset, options};
+
+  auto scan_declaration = compute::Declaration({"scan", scan_node_options});
+
+  ASSERT_OK_AND_ASSIGN(auto serialized_rel,
+                       SerializeRelation(scan_declaration, &ext_set));
+  ASSERT_OK_AND_ASSIGN(auto deserialized_decl,
+                       DeserializeRelation(*serialized_rel, ext_set));
+
+  auto dataset_comparator = [](std::shared_ptr<dataset::Dataset> ds_lhs,
+                               std::shared_ptr<dataset::Dataset> ds_rhs) -> bool {
+    const auto& fds_lhs = checked_cast<const dataset::FileSystemDataset&>(*ds_lhs);
+    const auto& fds_rhs = checked_cast<const dataset::FileSystemDataset&>(*ds_lhs);
+    const auto& files_lhs = fds_lhs.files();
+    const auto& files_rhs = fds_rhs.files();
+
+    bool cmp_fsize = files_lhs.size() == files_rhs.size();
+    uint64_t fidx = 0;
+    for (const auto& l_file : files_lhs) {
+      if (l_file != files_rhs[fidx]) {
+        return false;
+      }
+      fidx++;
+    }
+    bool cmp_file_format = fds_lhs.format()->Equals(*fds_lhs.format());
+    bool cmp_file_system = fds_lhs.filesystem()->Equals(fds_rhs.filesystem());
+    return cmp_fsize && cmp_file_format && cmp_file_system;
+  };
+
+  auto scan_option_comparator = [dataset_comparator](
+                                    const dataset::ScanNodeOptions& lhs,
+                                    const dataset::ScanNodeOptions& rhs) -> bool {
+    bool cmp_rso = lhs.require_sequenced_output == rhs.require_sequenced_output;
+    bool cmp_ds = dataset_comparator(lhs.dataset, rhs.dataset);
+    return cmp_rso && cmp_ds;
+  };
+
+  EXPECT_EQ(deserialized_decl.factory_name, scan_declaration.factory_name);
+  const auto& lhs =
+      checked_cast<const dataset::ScanNodeOptions&>(*deserialized_decl.options);
+  const auto& rhs =
+      checked_cast<const dataset::ScanNodeOptions&>(*scan_declaration.options);
+  ASSERT_TRUE(scan_option_comparator(lhs, rhs));
+#endif
+}
+
+TEST(Substrait, SerializeRelationEndToEnd) {

Review Comment:
   @westonpace 
   Added JIRAs for `Equals` methods.
   
   1. https://issues.apache.org/jira/browse/ARROW-16911
   2. https://issues.apache.org/jira/browse/ARROW-16910



-- 
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] westonpace commented on a diff in pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
westonpace commented on code in PR #13401:
URL: https://github.com/apache/arrow/pull/13401#discussion_r965395555


##########
cpp/src/arrow/engine/substrait/plan_internal.h:
##########
@@ -51,5 +53,17 @@ Result<ExtensionSet> GetExtensionSetFromPlan(
     const substrait::Plan& plan,
     const ExtensionIdRegistry* registry = default_extension_id_registry());
 
+/// \brief Serialize a declaration and into a substrait::Plan.

Review Comment:
   ```suggestion
   /// \brief Serialize a declaration into a substrait::Plan.
   ```



##########
cpp/src/arrow/engine/substrait/relation_internal.cc:
##########
@@ -421,5 +433,144 @@ Result<DeclarationInfo> FromProto(const substrait::Rel& rel, const ExtensionSet&
       rel.DebugString());
 }
 
+namespace {
+
+Result<std::shared_ptr<Schema>> ExtractSchemaToBind(const compute::Declaration& declr) {
+  std::shared_ptr<Schema> bind_schema;
+  if (declr.factory_name == "scan") {
+    const auto& opts = checked_cast<const dataset::ScanNodeOptions&>(*(declr.options));
+    bind_schema = opts.dataset->schema();
+  } else if (declr.factory_name == "filter") {
+    auto input_declr = util::get<compute::Declaration>(declr.inputs[0]);
+    ARROW_ASSIGN_OR_RAISE(bind_schema, ExtractSchemaToBind(input_declr));
+  } else if (declr.factory_name == "sink") {
+    // Note that the sink has no output_schema
+    return bind_schema;
+  } else {
+    return Status::Invalid("Schema extraction failed, unsupported factory ",
+                           declr.factory_name);
+  }
+  return bind_schema;
+}
+
+Result<std::unique_ptr<substrait::ReadRel>> ScanRelationConverter(
+    const std::shared_ptr<Schema>& schema, const compute::Declaration& declaration,
+    ExtensionSet* ext_set, const ConversionOptions& conversion_options) {
+  auto read_rel = make_unique<substrait::ReadRel>();
+  const auto& scan_node_options =
+      checked_cast<const dataset::ScanNodeOptions&>(*declaration.options);
+  auto dataset =
+      dynamic_cast<dataset::FileSystemDataset*>(scan_node_options.dataset.get());
+  if (dataset == nullptr) {
+    return Status::Invalid(
+        "Can only convert scan node with FileSystemDataset to a Substrait plan.");
+  }
+  // set schema
+  ARROW_ASSIGN_OR_RAISE(auto named_struct,
+                        ToProto(*dataset->schema(), ext_set, conversion_options));
+  read_rel->set_allocated_base_schema(named_struct.release());
+
+  // set local files
+  auto read_rel_lfs = make_unique<substrait::ReadRel_LocalFiles>();
+  for (const auto& file : dataset->files()) {
+    auto read_rel_lfs_ffs = make_unique<substrait::ReadRel_LocalFiles_FileOrFiles>();
+    read_rel_lfs_ffs->set_uri_path(UriFromAbsolutePath(file));
+    // set file format
+    auto format_type_name = dataset->format()->type_name();
+    if (format_type_name == "parquet") {
+      auto parquet_fmt =
+          make_unique<substrait::ReadRel_LocalFiles_FileOrFiles_ParquetReadOptions>();
+      read_rel_lfs_ffs->set_allocated_parquet(parquet_fmt.release());

Review Comment:
   Minor nit but creating a unique pointer only to release it on the next line seems a bit odd.  Maybe just:
   
   ```
   read_rel_lfs_ffs->set_allocated_parquet(new substrait::ReadRel_LocalFiles_FileOrFiles_ParquetReadOptions>());
   ```



##########
cpp/src/arrow/engine/substrait/relation_internal.cc:
##########
@@ -162,36 +170,40 @@ Result<DeclarationInfo> FromProto(const substrait::Rel& rel, const ExtensionSet&
         }
 
         path = path.substr(7);
-        if (item.path_type_case() ==
-            substrait::ReadRel_LocalFiles_FileOrFiles::kUriPath) {
-          ARROW_ASSIGN_OR_RAISE(auto file, filesystem->GetFileInfo(path));
-          if (file.type() == fs::FileType::File) {
-            files.push_back(std::move(file));
-          } else if (file.type() == fs::FileType::Directory) {
+        switch (item.path_type_case()) {
+          case substrait::ReadRel_LocalFiles_FileOrFiles::kUriPath: {
+            ARROW_ASSIGN_OR_RAISE(auto file, filesystem->GetFileInfo(path));
+            if (file.type() == fs::FileType::File) {
+              files.push_back(std::move(file));
+            } else if (file.type() == fs::FileType::Directory) {
+              fs::FileSelector selector;
+              selector.base_dir = path;
+              selector.recursive = true;
+              ARROW_ASSIGN_OR_RAISE(auto discovered_files,
+                                    filesystem->GetFileInfo(selector));
+              std::move(files.begin(), files.end(), std::back_inserter(discovered_files));
+            }
+            break;
+          }
+          case substrait::ReadRel_LocalFiles_FileOrFiles::kUriFile: {
+            files.emplace_back(path, fs::FileType::File);
+            break;
+          }
+          case substrait::ReadRel_LocalFiles_FileOrFiles::kUriFolder: {
             fs::FileSelector selector;
             selector.base_dir = path;
             selector.recursive = true;
             ARROW_ASSIGN_OR_RAISE(auto discovered_files,
                                   filesystem->GetFileInfo(selector));
-            std::move(files.begin(), files.end(), std::back_inserter(discovered_files));
+            std::move(discovered_files.begin(), discovered_files.end(),
+                      std::back_inserter(files));
+            break;
           }
-        }
-        if (item.path_type_case() ==
-            substrait::ReadRel_LocalFiles_FileOrFiles::kUriFile) {
-          files.emplace_back(path, fs::FileType::File);
-        } else if (item.path_type_case() ==
-                   substrait::ReadRel_LocalFiles_FileOrFiles::kUriFolder) {
-          fs::FileSelector selector;
-          selector.base_dir = path;
-          selector.recursive = true;
-          ARROW_ASSIGN_OR_RAISE(auto discovered_files, filesystem->GetFileInfo(selector));
-          std::move(discovered_files.begin(), discovered_files.end(),
-                    std::back_inserter(files));
-        } else {
-          ARROW_ASSIGN_OR_RAISE(auto discovered_files,
-                                fs::internal::GlobFiles(filesystem, path));
-          std::move(discovered_files.begin(), discovered_files.end(),
-                    std::back_inserter(files));
+          default:

Review Comment:
   Don't use `default` for the glob case.  Instead do something like...
   
   ```
   case substrait::ReadRel_LocalFiles_FileOrFiles::kUriGlob: {
     ...
     break;
   }
   default: {
     return Status::Invalid("Unrecognized file type in LocalFiles");
   }
   ```
   
   This will future-proof us against additions to the enum.



##########
cpp/src/arrow/util/io_util.cc:
##########
@@ -1867,7 +1867,16 @@ Result<std::unique_ptr<TemporaryDir>> TemporaryDir::Make(const std::string& pref
       [&](const NativePathString& base_dir) -> Result<std::unique_ptr<TemporaryDir>> {
     Status st;
     for (int attempt = 0; attempt < 3; ++attempt) {
-      PlatformFilename fn(base_dir + kNativeSep + base_name + kNativeSep);
+      // Note: certain temporary directories of MacOS contains a trailing slash
+      // Handling the base_dir with trailing slash
+      PlatformFilename fn;
+      if (base_dir.back() == kNativeSep) {
+        PlatformFilename fn_base_dir(base_dir);
+        PlatformFilename fn_base_name(base_name + kNativeSep);
+        fn = fn_base_dir.Join(fn_base_name);
+      } else {
+        fn = PlatformFilename(base_dir + kNativeSep + base_name + kNativeSep);
+      }

Review Comment:
   ```suggestion
         PlatformFilename fn_base_dir(base_dir);
         fn = fn_base_dir.Join(base_name) + kNativeSep;
   ```
   
   Does this work?  `Join` already has logic in place to test for a trailing separator so I'm not sure why we need to also test for one here.



##########
cpp/src/arrow/engine/substrait/plan_internal.h:
##########
@@ -51,5 +53,17 @@ Result<ExtensionSet> GetExtensionSetFromPlan(
     const substrait::Plan& plan,
     const ExtensionIdRegistry* registry = default_extension_id_registry());
 
+/// \brief Serialize a declaration and into a substrait::Plan.
+///
+/// Note that, this is a part of roundtripping test API and not

Review Comment:
   ```suggestion
   /// Note that, this is a part of a roundtripping test API and not
   ```



##########
cpp/src/arrow/util/uri.h:
##########
@@ -104,5 +104,10 @@ std::string UriEncodeHost(const std::string& host);
 ARROW_EXPORT
 bool IsValidUriScheme(const arrow::util::string_view s);
 
+/// Create a file uri from a given URI

Review Comment:
   ```suggestion
   /// Create a file uri from a given absolute path
   ```



##########
cpp/src/arrow/engine/substrait/relation_internal.cc:
##########
@@ -162,36 +170,40 @@ Result<DeclarationInfo> FromProto(const substrait::Rel& rel, const ExtensionSet&
         }
 
         path = path.substr(7);
-        if (item.path_type_case() ==
-            substrait::ReadRel_LocalFiles_FileOrFiles::kUriPath) {
-          ARROW_ASSIGN_OR_RAISE(auto file, filesystem->GetFileInfo(path));
-          if (file.type() == fs::FileType::File) {
-            files.push_back(std::move(file));
-          } else if (file.type() == fs::FileType::Directory) {
+        switch (item.path_type_case()) {
+          case substrait::ReadRel_LocalFiles_FileOrFiles::kUriPath: {

Review Comment:
   I kind of prefer `substrait::ReadRel::LocalFiles::FileOrFiles` over `substrait::ReadRel_LocalFiles_FileOrFiles` but I see we have some precedence for this case so I don't think you have to fix it for this PR.



##########
cpp/src/arrow/engine/substrait/relation_internal.cc:
##########
@@ -421,5 +433,144 @@ Result<DeclarationInfo> FromProto(const substrait::Rel& rel, const ExtensionSet&
       rel.DebugString());
 }
 
+namespace {
+
+Result<std::shared_ptr<Schema>> ExtractSchemaToBind(const compute::Declaration& declr) {
+  std::shared_ptr<Schema> bind_schema;
+  if (declr.factory_name == "scan") {
+    const auto& opts = checked_cast<const dataset::ScanNodeOptions&>(*(declr.options));
+    bind_schema = opts.dataset->schema();
+  } else if (declr.factory_name == "filter") {
+    auto input_declr = util::get<compute::Declaration>(declr.inputs[0]);
+    ARROW_ASSIGN_OR_RAISE(bind_schema, ExtractSchemaToBind(input_declr));
+  } else if (declr.factory_name == "sink") {
+    // Note that the sink has no output_schema
+    return bind_schema;
+  } else {
+    return Status::Invalid("Schema extraction failed, unsupported factory ",
+                           declr.factory_name);
+  }
+  return bind_schema;
+}
+
+Result<std::unique_ptr<substrait::ReadRel>> ScanRelationConverter(
+    const std::shared_ptr<Schema>& schema, const compute::Declaration& declaration,
+    ExtensionSet* ext_set, const ConversionOptions& conversion_options) {
+  auto read_rel = make_unique<substrait::ReadRel>();
+  const auto& scan_node_options =
+      checked_cast<const dataset::ScanNodeOptions&>(*declaration.options);
+  auto dataset =
+      dynamic_cast<dataset::FileSystemDataset*>(scan_node_options.dataset.get());
+  if (dataset == nullptr) {
+    return Status::Invalid(
+        "Can only convert scan node with FileSystemDataset to a Substrait plan.");
+  }
+  // set schema
+  ARROW_ASSIGN_OR_RAISE(auto named_struct,
+                        ToProto(*dataset->schema(), ext_set, conversion_options));
+  read_rel->set_allocated_base_schema(named_struct.release());
+
+  // set local files
+  auto read_rel_lfs = make_unique<substrait::ReadRel_LocalFiles>();
+  for (const auto& file : dataset->files()) {
+    auto read_rel_lfs_ffs = make_unique<substrait::ReadRel_LocalFiles_FileOrFiles>();
+    read_rel_lfs_ffs->set_uri_path(UriFromAbsolutePath(file));
+    // set file format
+    auto format_type_name = dataset->format()->type_name();
+    if (format_type_name == "parquet") {
+      auto parquet_fmt =
+          make_unique<substrait::ReadRel_LocalFiles_FileOrFiles_ParquetReadOptions>();
+      read_rel_lfs_ffs->set_allocated_parquet(parquet_fmt.release());
+    } else if (format_type_name == "ipc") {
+      auto arrow_fmt =
+          make_unique<substrait::ReadRel_LocalFiles_FileOrFiles_ArrowReadOptions>();
+      read_rel_lfs_ffs->set_allocated_arrow(arrow_fmt.release());
+    } else if (format_type_name == "orc") {
+      auto orc_fmt =
+          make_unique<substrait::ReadRel_LocalFiles_FileOrFiles_OrcReadOptions>();
+      read_rel_lfs_ffs->set_allocated_orc(orc_fmt.release());
+    } else {
+      return Status::NotImplemented("Unsupported file type: ", format_type_name);
+    }
+    read_rel_lfs->mutable_items()->AddAllocated(read_rel_lfs_ffs.release());
+  }
+  read_rel->set_allocated_local_files(read_rel_lfs.release());

Review Comment:
   Can we have a follow-up JIRA to add support for scan options projection & filter?  I don't think it should be done as part of this JIRA since it is changing.



##########
cpp/src/arrow/engine/substrait/relation_internal.h:
##########
@@ -40,9 +40,19 @@ struct DeclarationInfo {
   int num_columns;
 };
 
+/// \brief Convert a Substrait Rel object to an Acero declaration
 ARROW_ENGINE_EXPORT
 Result<DeclarationInfo> FromProto(const substrait::Rel&, const ExtensionSet&,
                                   const ConversionOptions&);
 
+/// \brief Convert an Acero Declaration to a Substrait Rel
+///
+/// Note that in order to provide a generic interface for ToProto for
+/// declaration. The ExecNode or ExecPlan is not used in this context as Declaration

Review Comment:
   ```suggestion
   /// Note that, in order to provide a generic interface for ToProto,
   /// the ExecNode or ExecPlan are not used in this context as Declaration
   ```



##########
cpp/src/arrow/util/uri.h:
##########
@@ -104,5 +104,10 @@ std::string UriEncodeHost(const std::string& host);
 ARROW_EXPORT
 bool IsValidUriScheme(const arrow::util::string_view s);
 
+/// Create a file uri from a given URI
+/// file:///<some_path>

Review Comment:
   ```suggestion
   ```



##########
cpp/src/arrow/engine/substrait/plan_internal.h:
##########
@@ -51,5 +53,17 @@ Result<ExtensionSet> GetExtensionSetFromPlan(
     const substrait::Plan& plan,
     const ExtensionIdRegistry* registry = default_extension_id_registry());
 
+/// \brief Serialize a declaration and into a substrait::Plan.
+///
+/// Note that, this is a part of roundtripping test API and not
+/// designed to use in production

Review Comment:
   ```suggestion
   /// designed for use in production
   ```



-- 
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] vibhatha commented on a diff in pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
vibhatha commented on code in PR #13401:
URL: https://github.com/apache/arrow/pull/13401#discussion_r965410625


##########
cpp/src/arrow/engine/substrait/relation_internal.cc:
##########
@@ -162,36 +170,40 @@ Result<DeclarationInfo> FromProto(const substrait::Rel& rel, const ExtensionSet&
         }
 
         path = path.substr(7);
-        if (item.path_type_case() ==
-            substrait::ReadRel_LocalFiles_FileOrFiles::kUriPath) {
-          ARROW_ASSIGN_OR_RAISE(auto file, filesystem->GetFileInfo(path));
-          if (file.type() == fs::FileType::File) {
-            files.push_back(std::move(file));
-          } else if (file.type() == fs::FileType::Directory) {
+        switch (item.path_type_case()) {
+          case substrait::ReadRel_LocalFiles_FileOrFiles::kUriPath: {

Review Comment:
   Good note. I also saw this kind of style in a few places. May be I will make a cleanup PR after to make them uniform across Substrait. 
   
   JIRA: https://issues.apache.org/jira/browse/ARROW-17647



-- 
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] vibhatha commented on a diff in pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
vibhatha commented on code in PR #13401:
URL: https://github.com/apache/arrow/pull/13401#discussion_r965463633


##########
cpp/src/arrow/engine/substrait/relation_internal.cc:
##########
@@ -421,5 +433,144 @@ Result<DeclarationInfo> FromProto(const substrait::Rel& rel, const ExtensionSet&
       rel.DebugString());
 }
 
+namespace {
+
+Result<std::shared_ptr<Schema>> ExtractSchemaToBind(const compute::Declaration& declr) {
+  std::shared_ptr<Schema> bind_schema;
+  if (declr.factory_name == "scan") {
+    const auto& opts = checked_cast<const dataset::ScanNodeOptions&>(*(declr.options));
+    bind_schema = opts.dataset->schema();
+  } else if (declr.factory_name == "filter") {
+    auto input_declr = util::get<compute::Declaration>(declr.inputs[0]);
+    ARROW_ASSIGN_OR_RAISE(bind_schema, ExtractSchemaToBind(input_declr));
+  } else if (declr.factory_name == "sink") {
+    // Note that the sink has no output_schema
+    return bind_schema;
+  } else {
+    return Status::Invalid("Schema extraction failed, unsupported factory ",
+                           declr.factory_name);
+  }
+  return bind_schema;
+}
+
+Result<std::unique_ptr<substrait::ReadRel>> ScanRelationConverter(
+    const std::shared_ptr<Schema>& schema, const compute::Declaration& declaration,
+    ExtensionSet* ext_set, const ConversionOptions& conversion_options) {
+  auto read_rel = make_unique<substrait::ReadRel>();
+  const auto& scan_node_options =
+      checked_cast<const dataset::ScanNodeOptions&>(*declaration.options);
+  auto dataset =
+      dynamic_cast<dataset::FileSystemDataset*>(scan_node_options.dataset.get());
+  if (dataset == nullptr) {
+    return Status::Invalid(
+        "Can only convert scan node with FileSystemDataset to a Substrait plan.");
+  }
+  // set schema
+  ARROW_ASSIGN_OR_RAISE(auto named_struct,
+                        ToProto(*dataset->schema(), ext_set, conversion_options));
+  read_rel->set_allocated_base_schema(named_struct.release());
+
+  // set local files
+  auto read_rel_lfs = make_unique<substrait::ReadRel_LocalFiles>();
+  for (const auto& file : dataset->files()) {
+    auto read_rel_lfs_ffs = make_unique<substrait::ReadRel_LocalFiles_FileOrFiles>();
+    read_rel_lfs_ffs->set_uri_path(UriFromAbsolutePath(file));
+    // set file format
+    auto format_type_name = dataset->format()->type_name();
+    if (format_type_name == "parquet") {
+      auto parquet_fmt =
+          make_unique<substrait::ReadRel_LocalFiles_FileOrFiles_ParquetReadOptions>();
+      read_rel_lfs_ffs->set_allocated_parquet(parquet_fmt.release());
+    } else if (format_type_name == "ipc") {
+      auto arrow_fmt =
+          make_unique<substrait::ReadRel_LocalFiles_FileOrFiles_ArrowReadOptions>();
+      read_rel_lfs_ffs->set_allocated_arrow(arrow_fmt.release());
+    } else if (format_type_name == "orc") {
+      auto orc_fmt =
+          make_unique<substrait::ReadRel_LocalFiles_FileOrFiles_OrcReadOptions>();
+      read_rel_lfs_ffs->set_allocated_orc(orc_fmt.release());
+    } else {
+      return Status::NotImplemented("Unsupported file type: ", format_type_name);
+    }
+    read_rel_lfs->mutable_items()->AddAllocated(read_rel_lfs_ffs.release());
+  }
+  read_rel->set_allocated_local_files(read_rel_lfs.release());

Review Comment:
   Nice catch. Jira created: https://issues.apache.org/jira/browse/ARROW-17647



-- 
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] vibhatha commented on a diff in pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
vibhatha commented on code in PR #13401:
URL: https://github.com/apache/arrow/pull/13401#discussion_r904965152


##########
cpp/src/arrow/engine/substrait/serde_test.cc:
##########
@@ -1173,5 +1175,168 @@ TEST(Substrait, JoinPlanInvalidKeys) {
           &ext_set));
 }
 
+TEST(Substrait, SerializeRelation) {
+#ifdef _WIN32
+  GTEST_SKIP() << "ARROW-16392: Substrait File URI not supported for Windows";
+#else
+  ExtensionSet ext_set;
+  auto dummy_schema = schema({field("f1", int32()), field("f2", int32())});
+  // creating a dummy dataset using a dummy table
+  auto format = std::make_shared<arrow::dataset::ParquetFileFormat>();
+  auto filesystem = std::make_shared<fs::LocalFileSystem>();
+
+  std::vector<fs::FileInfo> files;
+  const std::vector<std::string> f_paths = {"/tmp/data1.parquet", "/tmp/data2.parquet"};
+
+  for (const auto& f_path : f_paths) {
+    ASSERT_OK_AND_ASSIGN(auto f_file, filesystem->GetFileInfo(f_path));
+    files.push_back(std::move(f_file));
+  }
+
+  ASSERT_OK_AND_ASSIGN(auto ds_factory, dataset::FileSystemDatasetFactory::Make(
+                                            std::move(filesystem), std::move(files),
+                                            std::move(format), {}));
+  ASSERT_OK_AND_ASSIGN(auto dataset, ds_factory->Finish(dummy_schema));
+
+  auto options = std::make_shared<dataset::ScanOptions>();
+  options->projection = compute::project({}, {});
+  auto scan_node_options = dataset::ScanNodeOptions{dataset, options};
+
+  auto scan_declaration = compute::Declaration({"scan", scan_node_options});
+
+  ASSERT_OK_AND_ASSIGN(auto serialized_rel,
+                       SerializeRelation(scan_declaration, &ext_set));
+  ASSERT_OK_AND_ASSIGN(auto deserialized_decl,
+                       DeserializeRelation(*serialized_rel, ext_set));
+
+  auto dataset_comparator = [](std::shared_ptr<dataset::Dataset> ds_lhs,
+                               std::shared_ptr<dataset::Dataset> ds_rhs) -> bool {
+    const auto& fds_lhs = checked_cast<const dataset::FileSystemDataset&>(*ds_lhs);
+    const auto& fds_rhs = checked_cast<const dataset::FileSystemDataset&>(*ds_lhs);
+    const auto& files_lhs = fds_lhs.files();
+    const auto& files_rhs = fds_rhs.files();
+
+    bool cmp_fsize = files_lhs.size() == files_rhs.size();
+    uint64_t fidx = 0;
+    for (const auto& l_file : files_lhs) {
+      if (l_file != files_rhs[fidx]) {
+        return false;
+      }
+      fidx++;
+    }
+    bool cmp_file_format = fds_lhs.format()->Equals(*fds_lhs.format());
+    bool cmp_file_system = fds_lhs.filesystem()->Equals(fds_rhs.filesystem());
+    return cmp_fsize && cmp_file_format && cmp_file_system;
+  };
+
+  auto scan_option_comparator = [dataset_comparator](
+                                    const dataset::ScanNodeOptions& lhs,
+                                    const dataset::ScanNodeOptions& rhs) -> bool {
+    bool cmp_rso = lhs.require_sequenced_output == rhs.require_sequenced_output;
+    bool cmp_ds = dataset_comparator(lhs.dataset, rhs.dataset);
+    return cmp_rso && cmp_ds;
+  };

Review Comment:
   On second thoughts, may be for this PR the comparison required is satisfactory with those lambdas. We compare the tabel content in one test using Tables and the other test case just need to compare the configs. 
   
   May be we can create JIRAs to include `Equals` for these classes. 
   
   cc @westonpace 



-- 
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] jvanstraten commented on pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
jvanstraten commented on PR #13401:
URL: https://github.com/apache/arrow/pull/13401#issuecomment-1160229357

   I don't feel qualified to comment on those design questions, but FWIW, I ran the serialized output of the test case through the validator and it's okay-ish (the validator doesn't like the lack of a `NULLABILITY_REQUIRED` in the struct that represents the schema, but that's pretty pedantic I guess), and the code looks fine to me.


-- 
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] lidavidm commented on a diff in pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
lidavidm commented on code in PR #13401:
URL: https://github.com/apache/arrow/pull/13401#discussion_r901579403


##########
cpp/src/arrow/engine/substrait/serde_test.cc:
##########
@@ -1173,5 +1175,168 @@ TEST(Substrait, JoinPlanInvalidKeys) {
           &ext_set));
 }
 
+TEST(Substrait, SerializeRelation) {
+#ifdef _WIN32
+  GTEST_SKIP() << "ARROW-16392: Substrait File URI not supported for Windows";
+#else
+  ExtensionSet ext_set;
+  auto dummy_schema = schema({field("f1", int32()), field("f2", int32())});
+  // creating a dummy dataset using a dummy table
+  auto format = std::make_shared<arrow::dataset::ParquetFileFormat>();
+  auto filesystem = std::make_shared<fs::LocalFileSystem>();
+
+  std::vector<fs::FileInfo> files;
+  const std::vector<std::string> f_paths = {"/tmp/data1.parquet", "/tmp/data2.parquet"};
+
+  for (const auto& f_path : f_paths) {
+    ASSERT_OK_AND_ASSIGN(auto f_file, filesystem->GetFileInfo(f_path));
+    files.push_back(std::move(f_file));
+  }
+
+  ASSERT_OK_AND_ASSIGN(auto ds_factory, dataset::FileSystemDatasetFactory::Make(
+                                            std::move(filesystem), std::move(files),
+                                            std::move(format), {}));
+  ASSERT_OK_AND_ASSIGN(auto dataset, ds_factory->Finish(dummy_schema));
+
+  auto options = std::make_shared<dataset::ScanOptions>();
+  options->projection = compute::project({}, {});
+  auto scan_node_options = dataset::ScanNodeOptions{dataset, options};
+
+  auto scan_declaration = compute::Declaration({"scan", scan_node_options});
+
+  ASSERT_OK_AND_ASSIGN(auto serialized_rel,
+                       SerializeRelation(scan_declaration, &ext_set));
+  ASSERT_OK_AND_ASSIGN(auto deserialized_decl,
+                       DeserializeRelation(*serialized_rel, ext_set));
+
+  auto dataset_comparator = [](std::shared_ptr<dataset::Dataset> ds_lhs,
+                               std::shared_ptr<dataset::Dataset> ds_rhs) -> bool {
+    const auto& fds_lhs = checked_cast<const dataset::FileSystemDataset&>(*ds_lhs);
+    const auto& fds_rhs = checked_cast<const dataset::FileSystemDataset&>(*ds_lhs);
+    const auto& files_lhs = fds_lhs.files();
+    const auto& files_rhs = fds_rhs.files();
+
+    bool cmp_fsize = files_lhs.size() == files_rhs.size();
+    uint64_t fidx = 0;
+    for (const auto& l_file : files_lhs) {
+      if (l_file != files_rhs[fidx]) {
+        return false;
+      }
+      fidx++;
+    }
+    bool cmp_file_format = fds_lhs.format()->Equals(*fds_lhs.format());
+    bool cmp_file_system = fds_lhs.filesystem()->Equals(fds_rhs.filesystem());
+    return cmp_fsize && cmp_file_format && cmp_file_system;
+  };
+
+  auto scan_option_comparator = [dataset_comparator](
+                                    const dataset::ScanNodeOptions& lhs,
+                                    const dataset::ScanNodeOptions& rhs) -> bool {
+    bool cmp_rso = lhs.require_sequenced_output == rhs.require_sequenced_output;
+    bool cmp_ds = dataset_comparator(lhs.dataset, rhs.dataset);
+    return cmp_rso && cmp_ds;
+  };

Review Comment:
   Yes, let's add Equals where necessary. You can also inherit from [EqualityComparable](https://github.com/apache/arrow/blob/master/cpp/src/arrow/util/compare.h)



-- 
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] bkietz commented on pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

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

   I think this PR provides useful functionality for round trip testing. I also agree that we want to support custom relations (which will in all likelihood require a registry). I think trying to address both goals with the same API will compromise its efficacy for either.
   
   <hr/>
   
   Re extension relations: I think `google.protobuf.Any` is simple enough that we don't really need to expose it since it is just a buffer and a type_url. I'd propose the following structure:
   
   ```c++
   class ExtensionRelationRegistry {
    public:
     using Factory = std::function<Result<Declaration>(const Buffer& bytes, std::vector<Declaration> inputs)>;
   
     enum RelationType {
       /// substrait.ExtensionTable: the relation is a literal and requires no processing to emit data
       kTable,
       /// substrait.ExtensionLeafRel: the relation has no inputs, for example may stream data from disk
       kLeaf,
       /// substrait.ExtensionSingleRel: the relation has a single input, for example may filter out rows
       kSingle,
       /// substrait.ExtensionMultiRel: the relation has multiple inputs, for example may be a HighFolutinJoin
       kMulti,
     };
   
     virtual Status AddFactory(std::string type_url, RelationType, Factory) = 0;
     virtual Result<Factory> GetFactory(const std::string& type_url, RelationType) = 0;
   };
   ```
   
   When adding the above registry, we can extend ToProto with a lambda fallback like `struct { RelationType type; std::string type_url; std::shared_ptr<Buffer> bytes; } custom_to_proto(const Declaration&)` which should be sufficient for round trip testing purposes.


-- 
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] github-actions[bot] commented on pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

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

   :warning: Ticket **has no components in JIRA**, make sure you assign one.


-- 
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] github-actions[bot] commented on pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

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

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


-- 
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] vibhatha commented on a diff in pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
vibhatha commented on code in PR #13401:
URL: https://github.com/apache/arrow/pull/13401#discussion_r907461716


##########
cpp/src/arrow/engine/substrait/relation_internal.cc:
##########
@@ -316,5 +323,97 @@ Result<compute::Declaration> FromProto(const substrait::Rel& rel,
       rel.DebugString());
 }
 
+namespace {
+// TODO: add other types
+enum ArrowRelationType : uint8_t {
+  SCAN,
+  FILTER,
+  PROJECT,
+  JOIN,
+  AGGREGATE,
+};
+
+const std::map<std::string, ArrowRelationType> enum_map{
+    {"scan", ArrowRelationType::SCAN},           {"filter", ArrowRelationType::FILTER},
+    {"project", ArrowRelationType::PROJECT},     {"join", ArrowRelationType::JOIN},
+    {"aggregate", ArrowRelationType::AGGREGATE},
+};
+
+struct ExtractRelation {
+  explicit ExtractRelation(substrait::Rel* rel, ExtensionSet* ext_set)
+      : rel_(rel), ext_set_(ext_set) {}
+
+  Status AddRelation(const compute::Declaration& declaration) {
+    const std::string& rel_name = declaration.factory_name;
+    switch (enum_map.find(rel_name)->second) {
+      case ArrowRelationType::SCAN:
+        return AddReadRelation(declaration);
+      case ArrowRelationType::FILTER:
+        return Status::NotImplemented("Filter operator not supported.");
+      case ArrowRelationType::PROJECT:
+        return Status::NotImplemented("Project operator not supported.");
+      case ArrowRelationType::JOIN:
+        return Status::NotImplemented("Join operator not supported.");
+      case ArrowRelationType::AGGREGATE:
+        return Status::NotImplemented("Aggregate operator not supported.");
+      default:
+        return Status::Invalid("Unsupported factory name :", rel_name);
+    }
+  }
+
+  Status AddReadRelation(const compute::Declaration& declaration) {
+    auto read_rel = internal::make_unique<substrait::ReadRel>();
+    const auto& scan_node_options =
+        internal::checked_cast<const dataset::ScanNodeOptions&>(*declaration.options);
+
+    const auto& fds = internal::checked_cast<const dataset::FileSystemDataset&>(
+        *scan_node_options.dataset);
+
+    // set schema
+    ARROW_ASSIGN_OR_RAISE(auto named_struct, ToProto(*fds.schema(), ext_set_));
+    read_rel->set_allocated_base_schema(named_struct.release());
+
+    // set local files
+    auto read_rel_lfs = internal::make_unique<substrait::ReadRel_LocalFiles>();
+    for (const auto& file : fds.files()) {
+      auto read_rel_lfs_ffs =
+          internal::make_unique<substrait::ReadRel_LocalFiles_FileOrFiles>();
+      read_rel_lfs_ffs->set_uri_path("file://" + file);
+
+      // set file format
+      auto format_type_name = fds.format()->type_name();
+      if (format_type_name == "parquet" || format_type_name == "arrow" ||
+          format_type_name == "feather") {
+        read_rel_lfs_ffs->set_format(
+            substrait::ReadRel::LocalFiles::FileOrFiles::FILE_FORMAT_PARQUET);
+      } else {
+        return Status::Invalid("Unsupported file type : ", format_type_name);
+      }
+      read_rel_lfs->mutable_items()->AddAllocated(read_rel_lfs_ffs.release());
+    }
+    *read_rel->mutable_local_files() = *read_rel_lfs.get();
+
+    rel_->set_allocated_read(read_rel.release());

Review Comment:
   JIRA created here: https://issues.apache.org/jira/browse/ARROW-16909



-- 
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] vibhatha commented on pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
vibhatha commented on PR #13401:
URL: https://github.com/apache/arrow/pull/13401#issuecomment-1175710995

   > > Intersting thoughts. I will take a look at the tool. It would be better if we can use it to validate things. But I am not sure if it needs to be inside the Arrow source or should it be a plugin for Apache Arrow. cc @westonpace
   > 
   > I think it would make a lot of sense for unit tests to bring in the validator as a C dependency.
   
   Should we create a JIRA for this? 


-- 
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] vibhatha commented on a diff in pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
vibhatha commented on code in PR #13401:
URL: https://github.com/apache/arrow/pull/13401#discussion_r918450190


##########
cpp/src/arrow/engine/substrait/serde_test.cc:
##########
@@ -1383,5 +1400,188 @@ TEST(Substrait, JoinPlanInvalidKeys) {
   }
 }
 
+TEST(Substrait, SerializeRelation) {
+#ifdef _WIN32
+  GTEST_SKIP() << "ARROW-16392: Substrait File URI not supported for Windows";
+#else
+  ExtensionSet ext_set;
+  auto dummy_schema = schema({field("foo", binary())});
+  // creating a dummy dataset using a dummy table
+  auto format = std::make_shared<arrow::dataset::ParquetFileFormat>();
+  auto filesystem = std::make_shared<fs::LocalFileSystem>();
+
+  std::vector<fs::FileInfo> files;
+  const std::vector<std::string> f_paths = {"/tmp/data1.parquet", "/tmp/data2.parquet"};
+
+  for (const auto& f_path : f_paths) {
+    ASSERT_OK_AND_ASSIGN(auto f_file, filesystem->GetFileInfo(f_path));
+    files.push_back(std::move(f_file));
+  }
+
+  ASSERT_OK_AND_ASSIGN(auto ds_factory, dataset::FileSystemDatasetFactory::Make(
+                                            std::move(filesystem), std::move(files),
+                                            std::move(format), {}));
+  ASSERT_OK_AND_ASSIGN(auto dataset, ds_factory->Finish(dummy_schema));
+
+  auto options = std::make_shared<dataset::ScanOptions>();
+  options->projection = compute::project({}, {});
+  auto scan_node_options = dataset::ScanNodeOptions{dataset, options};
+
+  auto scan_declaration = compute::Declaration({"scan", scan_node_options});
+
+  ASSERT_OK_AND_ASSIGN(auto serialized_rel,
+                       SerializeRelation(scan_declaration, &ext_set));
+  ASSERT_OK_AND_ASSIGN(auto deserialized_decl,
+                       DeserializeRelation(*serialized_rel, ext_set));
+
+  auto dataset_comparator = [](std::shared_ptr<dataset::Dataset> ds_lhs,
+                               std::shared_ptr<dataset::Dataset> ds_rhs) -> bool {
+    const auto& fsd_lhs = checked_cast<const dataset::FileSystemDataset&>(*ds_lhs);
+    const auto& fsd_rhs = checked_cast<const dataset::FileSystemDataset&>(*ds_lhs);
+    const auto& files_lhs = fsd_lhs.files();
+    const auto& files_rhs = fsd_rhs.files();
+
+    if (files_lhs.size() != files_rhs.size()) {
+      return false;
+    }
+    uint64_t fidx = 0;
+    for (const auto& l_file : files_lhs) {
+      if (l_file != files_rhs[fidx]) {
+        return false;
+      }
+      fidx++;
+    }
+    bool cmp_file_format = fsd_lhs.format()->Equals(*fsd_lhs.format());
+    bool cmp_file_system = fsd_lhs.filesystem()->Equals(fsd_rhs.filesystem());
+    return cmp_file_format && cmp_file_system;
+  };
+
+  auto scan_option_comparator = [dataset_comparator](
+                                    const dataset::ScanNodeOptions& lhs,
+                                    const dataset::ScanNodeOptions& rhs) -> bool {
+    bool cmp_rso = lhs.require_sequenced_output == rhs.require_sequenced_output;
+    bool cmp_ds = dataset_comparator(lhs.dataset, rhs.dataset);
+    return cmp_rso && cmp_ds;
+  };
+
+  EXPECT_EQ(deserialized_decl.factory_name, scan_declaration.factory_name);
+  const auto& lhs =
+      checked_cast<const dataset::ScanNodeOptions&>(*deserialized_decl.options);
+  const auto& rhs =
+      checked_cast<const dataset::ScanNodeOptions&>(*scan_declaration.options);
+  ASSERT_TRUE(scan_option_comparator(lhs, rhs));
+#endif
+}
+
+TEST(Substrait, SerializeRelationEndToEnd) {
+#ifdef _WIN32
+  GTEST_SKIP() << "ARROW-16392: Substrait File URI not supported for Windows";
+#else
+  ExtensionSet ext_set;
+  compute::ExecContext exec_context;
+
+  auto dummy_schema = schema({field("a", int32()), field("b", int32())});
+  auto table = TableFromJSON(dummy_schema, {R"([
+    [1, 1],
+    [3, 4]
+                        ])",
+                                            R"([
+    [0, 2],
+    [1, 3],
+    [4, 1],
+    [3, 1],
+    [1, 2]
+                        ])",
+                                            R"([
+    [2, 2],
+    [5, 3],
+    [1, 3]
+                        ])"});
+  // Note:: adding `/` to support the currently supported file read format from file
+  // system for Substrait

Review Comment:
   Ah instead of just saying `test.parquet` we say ```/test.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.

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

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


[GitHub] [arrow] vibhatha commented on a diff in pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
vibhatha commented on code in PR #13401:
URL: https://github.com/apache/arrow/pull/13401#discussion_r918449687


##########
cpp/src/arrow/engine/substrait/serde_test.cc:
##########
@@ -1383,5 +1400,188 @@ TEST(Substrait, JoinPlanInvalidKeys) {
   }
 }
 
+TEST(Substrait, SerializeRelation) {
+#ifdef _WIN32
+  GTEST_SKIP() << "ARROW-16392: Substrait File URI not supported for Windows";
+#else
+  ExtensionSet ext_set;
+  auto dummy_schema = schema({field("foo", binary())});
+  // creating a dummy dataset using a dummy table
+  auto format = std::make_shared<arrow::dataset::ParquetFileFormat>();
+  auto filesystem = std::make_shared<fs::LocalFileSystem>();
+
+  std::vector<fs::FileInfo> files;
+  const std::vector<std::string> f_paths = {"/tmp/data1.parquet", "/tmp/data2.parquet"};

Review Comment:
   Didn't we got an issue with Windows paths already in Substrait? We have added a skip for Windows tests. 



-- 
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] vibhatha commented on a diff in pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
vibhatha commented on code in PR #13401:
URL: https://github.com/apache/arrow/pull/13401#discussion_r956888894


##########
cpp/src/arrow/engine/substrait/serde.h:
##########
@@ -202,6 +207,18 @@ Result<std::shared_ptr<Buffer>> SerializeExpression(
     const compute::Expression& expr, ExtensionSet* ext_set,
     const ConversionOptions& conversion_options = {});
 
+/// \brief Serializes an Arrow compute Declaration to a Substrait Relation message
+///
+/// \param[in] declaration the Arrow compute declaration to serialize
+/// \param[in,out] ext_set the extension mapping to use; may be updated to add

Review Comment:
   the latter part was not added. I added the full description. Thanks for noting this typo.



-- 
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] vibhatha commented on a diff in pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
vibhatha commented on code in PR #13401:
URL: https://github.com/apache/arrow/pull/13401#discussion_r956904559


##########
cpp/src/arrow/engine/substrait/relation_internal.h:
##########
@@ -40,9 +40,46 @@ struct DeclarationInfo {
   int num_columns;
 };
 
+/// \brief A function to extract Acero Declaration from a Substrait Rel object
 ARROW_ENGINE_EXPORT
 Result<DeclarationInfo> FromProto(const substrait::Rel&, const ExtensionSet&,
                                   const ConversionOptions&);
 
+/// \brief Serializes a Declaration, produce a Substrait Rel and update the global
+/// Substrait plan. A Substrait Rel is passed as a the plan and it is updated with
+/// corresponding Declaration passed for serialization.
+///
+/// Note that this is a rather a helper method useful to fuse a partially serialized
+/// plan with another plan. The reason for having a partially serialized plan is to
+/// avoid unnecessary complication and enable partial plan serialization without
+/// affecting a global plan. Since kept as unique_ptr resources are relased efficiently
+/// upon releasing for the global plan.
+ARROW_ENGINE_EXPORT Status SerializeAndCombineRelations(const compute::Declaration&,
+                                                        ExtensionSet*,
+                                                        std::unique_ptr<substrait::Rel>&,
+                                                        const ConversionOptions&);
+
+/// \brief Serialize a Declaration and produces a Substrait Rel.
+///
+/// Note that in order to provide a generic interface for ToProto for
+/// declaration it is not specialized for each relation within the Substrait Rel.
+/// Rather a serialized relation is set as a member for the Substrait Rel
+/// (partial Relation) which is later on extracted to update a Substrait Rel
+/// which would be included in the fully serialized Acero Exec Plan.
+/// The ExecNode or ExecPlan is not used in this context as Declaration is preferred
+/// in the Substrait space rather than internal components of Acero execution engine.

Review Comment:
   Ah this was part of what I did to write the generic interface for registry. I didn't update this properly. Now it is simplified.



-- 
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] vibhatha commented on a diff in pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
vibhatha commented on code in PR #13401:
URL: https://github.com/apache/arrow/pull/13401#discussion_r956907399


##########
cpp/src/arrow/engine/substrait/relation_internal.h:
##########
@@ -40,9 +40,46 @@ struct DeclarationInfo {
   int num_columns;
 };
 
+/// \brief A function to extract Acero Declaration from a Substrait Rel object
 ARROW_ENGINE_EXPORT
 Result<DeclarationInfo> FromProto(const substrait::Rel&, const ExtensionSet&,
                                   const ConversionOptions&);
 
+/// \brief Serializes a Declaration, produce a Substrait Rel and update the global
+/// Substrait plan. A Substrait Rel is passed as a the plan and it is updated with
+/// corresponding Declaration passed for serialization.
+///
+/// Note that this is a rather a helper method useful to fuse a partially serialized
+/// plan with another plan. The reason for having a partially serialized plan is to
+/// avoid unnecessary complication and enable partial plan serialization without
+/// affecting a global plan. Since kept as unique_ptr resources are relased efficiently
+/// upon releasing for the global plan.
+ARROW_ENGINE_EXPORT Status SerializeAndCombineRelations(const compute::Declaration&,
+                                                        ExtensionSet*,
+                                                        std::unique_ptr<substrait::Rel>&,

Review Comment:
   Thanks for noting this. 



-- 
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] vibhatha commented on a diff in pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
vibhatha commented on code in PR #13401:
URL: https://github.com/apache/arrow/pull/13401#discussion_r957069255


##########
cpp/src/arrow/engine/substrait/serde_test.cc:
##########
@@ -42,9 +55,35 @@ using testing::UnorderedElementsAre;
 namespace arrow {
 
 using internal::checked_cast;
-
+using internal::hash_combine;
 namespace engine {
 
+Status WriteParquetData(const std::string& path,
+                        const std::shared_ptr<fs::FileSystem> file_system,
+                        const std::shared_ptr<Table> input) {
+  EXPECT_OK_AND_ASSIGN(auto buffer_writer, file_system->OpenOutputStream(path));
+  PARQUET_THROW_NOT_OK(parquet::arrow::WriteTable(*input, arrow::default_memory_pool(),
+                                                  buffer_writer, /*chunk_size*/ 1));
+  return buffer_writer->Close();
+}

Review Comment:
   You mean `ORC`?



-- 
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] westonpace merged pull request #13401: ARROW-16855: [C++] Adding Read Relation ToProto

Posted by GitBox <gi...@apache.org>.
westonpace merged PR #13401:
URL: https://github.com/apache/arrow/pull/13401


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