Skip to content

Commit a40a702

Browse files
westonpacethisisnicjorisvandenbosscheanjakefalapitrou
authored andcommitted
GH-35730: [C++] Add the ability to specify custom schema on a dataset write (#35860)
### Rationale for this change The dataset write node previously allowed you to specify custom key/value metadata on a write node. This was added to support saving schema metadata. However, it doesn't capture field metadata or field nullability. This PR replaces that capability with the ability to specify a custom schema instead. The custom schema must have the same number of fields as the input to the write node and each field must have the same type. ### What changes are included in this PR? Added `custom_schema` to `WriteNodeOptions` and removed `custom_metadata`. ### Are these changes tested? Yes, I added a new C++ unit test to verify that the custom info is applied to written files. ### Are there any user-facing changes? No. Only new functionality (which is user facing) * Closes: #35730 Lead-authored-by: Weston Pace <[email protected]> Co-authored-by: Nic Crane <[email protected]> Co-authored-by: Joris Van den Bossche <[email protected]> Co-authored-by: anjakefala <[email protected]> Co-authored-by: Antoine Pitrou <[email protected]> Signed-off-by: Weston Pace <[email protected]>
1 parent 0334820 commit a40a702

File tree

10 files changed

+310
-29
lines changed

10 files changed

+310
-29
lines changed

cpp/src/arrow/dataset/CMakeLists.txt

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -151,6 +151,7 @@ add_arrow_dataset_test(file_test)
151151
add_arrow_dataset_test(partition_test)
152152
add_arrow_dataset_test(scanner_test)
153153
add_arrow_dataset_test(subtree_test)
154+
add_arrow_dataset_test(write_node_test)
154155

155156
if(ARROW_CSV)
156157
add_arrow_dataset_test(file_csv_test)

cpp/src/arrow/dataset/file_base.cc

Lines changed: 45 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -387,16 +387,16 @@ Status WriteBatch(
387387

388388
class DatasetWritingSinkNodeConsumer : public acero::SinkNodeConsumer {
389389
public:
390-
DatasetWritingSinkNodeConsumer(std::shared_ptr<const KeyValueMetadata> custom_metadata,
390+
DatasetWritingSinkNodeConsumer(std::shared_ptr<Schema> custom_schema,
391391
FileSystemDatasetWriteOptions write_options)
392-
: custom_metadata_(std::move(custom_metadata)),
392+
: custom_schema_(std::move(custom_schema)),
393393
write_options_(std::move(write_options)) {}
394394

395395
Status Init(const std::shared_ptr<Schema>& schema,
396396
acero::BackpressureControl* backpressure_control,
397397
acero::ExecPlan* plan) override {
398-
if (custom_metadata_) {
399-
schema_ = schema->WithMetadata(custom_metadata_);
398+
if (custom_schema_) {
399+
schema_ = custom_schema_;
400400
} else {
401401
schema_ = schema;
402402
}
@@ -434,7 +434,7 @@ class DatasetWritingSinkNodeConsumer : public acero::SinkNodeConsumer {
434434
});
435435
}
436436

437-
std::shared_ptr<const KeyValueMetadata> custom_metadata_;
437+
std::shared_ptr<Schema> custom_schema_;
438438
std::unique_ptr<internal::DatasetWriter> dataset_writer_;
439439
FileSystemDatasetWriteOptions write_options_;
440440
Future<> finished_ = Future<>::Make();
@@ -453,13 +453,16 @@ Status FileSystemDataset::Write(const FileSystemDatasetWriteOptions& write_optio
453453

454454
// The projected_schema is currently used by pyarrow to preserve the custom metadata
455455
// when reading from a single input file.
456-
const auto& custom_metadata = scanner->options()->projected_schema->metadata();
456+
const auto& custom_schema = scanner->options()->projected_schema;
457+
458+
WriteNodeOptions write_node_options(write_options);
459+
write_node_options.custom_schema = custom_schema;
457460

458461
acero::Declaration plan = acero::Declaration::Sequence({
459462
{"scan", ScanNodeOptions{dataset, scanner->options()}},
460463
{"filter", acero::FilterNodeOptions{scanner->options()->filter}},
461464
{"project", acero::ProjectNodeOptions{std::move(exprs), std::move(names)}},
462-
{"write", WriteNodeOptions{write_options, custom_metadata}},
465+
{"write", std::move(write_node_options)},
463466
});
464467

465468
return acero::DeclarationToStatus(std::move(plan), scanner->options()->use_threads);
@@ -475,16 +478,50 @@ Result<acero::ExecNode*> MakeWriteNode(acero::ExecPlan* plan,
475478

476479
const WriteNodeOptions write_node_options =
477480
checked_cast<const WriteNodeOptions&>(options);
481+
std::shared_ptr<Schema> custom_schema = write_node_options.custom_schema;
478482
const std::shared_ptr<const KeyValueMetadata>& custom_metadata =
479483
write_node_options.custom_metadata;
480484
const FileSystemDatasetWriteOptions& write_options = write_node_options.write_options;
481485

486+
const std::shared_ptr<Schema>& input_schema = inputs[0]->output_schema();
487+
488+
if (custom_schema != nullptr) {
489+
if (custom_metadata) {
490+
return Status::TypeError(
491+
"Do not provide both custom_metadata and custom_schema. If custom_schema is "
492+
"used then custom_schema->metadata should be used instead of custom_metadata");
493+
}
494+
495+
if (custom_schema->num_fields() != input_schema->num_fields()) {
496+
return Status::TypeError(
497+
"The provided custom_schema did not have the same number of fields as the "
498+
"data. The custom schema can only be used to add metadata / nullability to "
499+
"fields and cannot change the type or number of fields.");
500+
}
501+
for (int field_idx = 0; field_idx < input_schema->num_fields(); field_idx++) {
502+
if (!input_schema->field(field_idx)->type()->Equals(
503+
custom_schema->field(field_idx)->type())) {
504+
return Status::TypeError("The provided custom_schema specified type ",
505+
custom_schema->field(field_idx)->type()->ToString(),
506+
" for field ", field_idx, "and the input data has type ",
507+
input_schema->field(field_idx),
508+
"The custom schema can only be used to add metadata / "
509+
"nullability to fields and "
510+
"cannot change the type or number of fields.");
511+
}
512+
}
513+
}
514+
515+
if (custom_metadata) {
516+
custom_schema = input_schema->WithMetadata(custom_metadata);
517+
}
518+
482519
if (!write_options.partitioning) {
483520
return Status::Invalid("Must provide partitioning");
484521
}
485522

486523
std::shared_ptr<DatasetWritingSinkNodeConsumer> consumer =
487-
std::make_shared<DatasetWritingSinkNodeConsumer>(custom_metadata, write_options);
524+
std::make_shared<DatasetWritingSinkNodeConsumer>(custom_schema, write_options);
488525

489526
ARROW_ASSIGN_OR_RAISE(
490527
auto node,

cpp/src/arrow/dataset/file_base.h

Lines changed: 10 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -33,6 +33,7 @@
3333
#include "arrow/dataset/visibility.h"
3434
#include "arrow/filesystem/filesystem.h"
3535
#include "arrow/io/file.h"
36+
#include "arrow/type_fwd.h"
3637
#include "arrow/util/compression.h"
3738

3839
namespace arrow {
@@ -470,6 +471,15 @@ class ARROW_DS_EXPORT WriteNodeOptions : public acero::ExecNodeOptions {
470471

471472
/// \brief Options to control how to write the dataset
472473
FileSystemDatasetWriteOptions write_options;
474+
/// \brief Optional schema to attach to all written batches
475+
///
476+
/// By default, we will use the output schema of the input.
477+
///
478+
/// This can be used to alter schema metadata, field nullability, or field metadata.
479+
/// However, this cannot be used to change the type of data. If the custom schema does
480+
/// not have the same number of fields and the same data types as the input then the
481+
/// plan will fail.
482+
std::shared_ptr<Schema> custom_schema;
473483
/// \brief Optional metadata to attach to written batches
474484
std::shared_ptr<const KeyValueMetadata> custom_metadata;
475485
};
Lines changed: 174 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,174 @@
1+
// Licensed to the Apache Software Foundation (ASF) under one
2+
// or more contributor license agreements. See the NOTICE file
3+
// distributed with this work for additional information
4+
// regarding copyright ownership. The ASF licenses this file
5+
// to you under the Apache License, Version 2.0 (the
6+
// "License"); you may not use this file except in compliance
7+
// with the License. You may obtain a copy of the License at
8+
//
9+
// http://www.apache.org/licenses/LICENSE-2.0
10+
//
11+
// Unless required by applicable law or agreed to in writing,
12+
// software distributed under the License is distributed on an
13+
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
14+
// KIND, either express or implied. See the License for the
15+
// specific language governing permissions and limitations
16+
// under the License.
17+
18+
#include <gmock/gmock-matchers.h>
19+
#include <gtest/gtest.h>
20+
21+
#include <memory>
22+
23+
#include "arrow/acero/exec_plan.h"
24+
#include "arrow/acero/options.h"
25+
#include "arrow/dataset/file_base.h"
26+
#include "arrow/dataset/file_ipc.h"
27+
#include "arrow/dataset/partition.h"
28+
#include "arrow/dataset/plan.h"
29+
#include "arrow/filesystem/filesystem.h"
30+
#include "arrow/filesystem/mockfs.h"
31+
#include "arrow/io/interfaces.h"
32+
#include "arrow/ipc/reader.h"
33+
#include "arrow/testing/generator.h"
34+
#include "arrow/testing/gtest_util.h"
35+
#include "arrow/testing/matchers.h"
36+
37+
#include "arrow/table.h"
38+
#include "arrow/util/key_value_metadata.h"
39+
40+
namespace arrow {
41+
42+
namespace dataset {
43+
44+
class SimpleWriteNodeTest : public ::testing::Test {
45+
protected:
46+
void SetUp() override {
47+
internal::Initialize();
48+
mock_fs_ = std::make_shared<fs::internal::MockFileSystem>(fs::kNoTime);
49+
auto ipc_format = std::make_shared<dataset::IpcFileFormat>();
50+
51+
fs_write_options_.filesystem = mock_fs_;
52+
fs_write_options_.base_dir = "/my_dataset";
53+
fs_write_options_.basename_template = "{i}.arrow";
54+
fs_write_options_.file_write_options = ipc_format->DefaultWriteOptions();
55+
fs_write_options_.partitioning = dataset::Partitioning::Default();
56+
}
57+
58+
std::shared_ptr<fs::internal::MockFileSystem> mock_fs_;
59+
dataset::FileSystemDatasetWriteOptions fs_write_options_;
60+
};
61+
62+
TEST_F(SimpleWriteNodeTest, CustomNullability) {
63+
// Create an input table with a nullable and a non-nullable type
64+
ExecBatch batch = gen::Gen({gen::Step()})->FailOnError()->ExecBatch(/*num_rows=*/1);
65+
std::shared_ptr<Schema> test_schema =
66+
schema({field("nullable_i32", uint32(), /*nullable=*/true),
67+
field("non_nullable_i32", uint32(), /*nullable=*/false)});
68+
std::shared_ptr<RecordBatch> record_batch =
69+
RecordBatch::Make(test_schema, /*num_rows=*/1,
70+
{batch.values[0].make_array(), batch.values[0].make_array()});
71+
ASSERT_OK_AND_ASSIGN(std::shared_ptr<Table> table,
72+
Table::FromRecordBatches({std::move(record_batch)}));
73+
74+
ASSERT_TRUE(table->field(0)->nullable());
75+
ASSERT_FALSE(table->field(1)->nullable());
76+
77+
dataset::WriteNodeOptions write_options(fs_write_options_);
78+
write_options.custom_schema = test_schema;
79+
80+
// Write the data to disk (these plans use a project because it destroys whatever
81+
// metadata happened to be in the table source node's output schema). This more
82+
// accurately simulates reading from a dataset.
83+
acero::Declaration plan = acero::Declaration::Sequence(
84+
{{"table_source", acero::TableSourceNodeOptions(table)},
85+
{"project",
86+
acero::ProjectNodeOptions({compute::field_ref(0), compute::field_ref(1)})},
87+
{"write", write_options}});
88+
89+
ASSERT_OK(DeclarationToStatus(plan));
90+
91+
// Read the file back out and verify the nullability
92+
ASSERT_OK_AND_ASSIGN(std::shared_ptr<io::RandomAccessFile> file,
93+
mock_fs_->OpenInputFile("/my_dataset/0.arrow"));
94+
ASSERT_OK_AND_ASSIGN(std::shared_ptr<ipc::RecordBatchFileReader> file_reader,
95+
ipc::RecordBatchFileReader::Open(file));
96+
std::shared_ptr<Schema> file_schema = file_reader->schema();
97+
98+
ASSERT_TRUE(file_schema->field(0)->nullable());
99+
ASSERT_FALSE(file_schema->field(1)->nullable());
100+
101+
// Invalid custom schema
102+
103+
// Incorrect # of fields
104+
write_options.custom_schema = schema({});
105+
plan = acero::Declaration::Sequence(
106+
{{"table_source", acero::TableSourceNodeOptions(table)},
107+
{"project",
108+
acero::ProjectNodeOptions({compute::field_ref(0), compute::field_ref(1)})},
109+
{"write", write_options}});
110+
111+
ASSERT_THAT(
112+
DeclarationToStatus(plan),
113+
Raises(StatusCode::TypeError,
114+
::testing::HasSubstr("did not have the same number of fields as the data")));
115+
116+
// Incorrect types
117+
write_options.custom_schema =
118+
schema({field("nullable_i32", int32()), field("non_nullable_i32", int32())});
119+
plan = acero::Declaration::Sequence(
120+
{{"table_source", acero::TableSourceNodeOptions(table)},
121+
{"project",
122+
acero::ProjectNodeOptions({compute::field_ref(0), compute::field_ref(1)})},
123+
{"write", write_options}});
124+
ASSERT_THAT(
125+
DeclarationToStatus(plan),
126+
Raises(StatusCode::TypeError, ::testing::HasSubstr("and the input data has type")));
127+
128+
// Cannot have both custom_schema and custom_metadata
129+
write_options.custom_schema = test_schema;
130+
write_options.custom_metadata = key_value_metadata({{"foo", "bar"}});
131+
plan = acero::Declaration::Sequence(
132+
{{"table_source", acero::TableSourceNodeOptions(std::move(table))},
133+
{"project",
134+
acero::ProjectNodeOptions({compute::field_ref(0), compute::field_ref(1)})},
135+
{"write", write_options}});
136+
ASSERT_THAT(DeclarationToStatus(plan),
137+
Raises(StatusCode::TypeError,
138+
::testing::HasSubstr(
139+
"Do not provide both custom_metadata and custom_schema")));
140+
}
141+
142+
TEST_F(SimpleWriteNodeTest, CustomMetadata) {
143+
constexpr int64_t kRowsPerChunk = 1;
144+
constexpr int64_t kNumChunks = 1;
145+
// Create an input table with no schema metadata
146+
std::shared_ptr<Table> table =
147+
gen::Gen({gen::Step()})->FailOnError()->Table(kRowsPerChunk, kNumChunks);
148+
149+
std::shared_ptr<KeyValueMetadata> custom_metadata =
150+
key_value_metadata({{"foo", "bar"}});
151+
152+
dataset::WriteNodeOptions write_options(fs_write_options_);
153+
write_options.custom_metadata = custom_metadata;
154+
155+
// Write the data to disk
156+
acero::Declaration plan = acero::Declaration::Sequence(
157+
{{"table_source", acero::TableSourceNodeOptions(table)},
158+
{"project", acero::ProjectNodeOptions({compute::field_ref(0)})},
159+
{"write", write_options}});
160+
161+
ASSERT_OK(DeclarationToStatus(plan));
162+
163+
// Read the file back out and verify the schema metadata
164+
ASSERT_OK_AND_ASSIGN(std::shared_ptr<io::RandomAccessFile> file,
165+
mock_fs_->OpenInputFile("/my_dataset/0.arrow"));
166+
ASSERT_OK_AND_ASSIGN(std::shared_ptr<ipc::RecordBatchFileReader> file_reader,
167+
ipc::RecordBatchFileReader::Open(file));
168+
std::shared_ptr<Schema> file_schema = file_reader->schema();
169+
170+
ASSERT_TRUE(custom_metadata->Equals(*file_schema->metadata()));
171+
}
172+
173+
} // namespace dataset
174+
} // namespace arrow

python/pyarrow/tests/test_dataset.py

Lines changed: 53 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -5074,6 +5074,59 @@ def test_dataset_partition_with_slash(tmpdir):
50745074
assert encoded_paths == file_paths
50755075

50765076

5077+
@pytest.mark.parquet
5078+
def test_write_dataset_preserve_nullability(tempdir):
5079+
# GH-35730
5080+
schema_nullable = pa.schema([
5081+
pa.field("x", pa.int64(), nullable=False),
5082+
pa.field("y", pa.int64(), nullable=True)])
5083+
5084+
arrays = [[1, 2, 3], [None, 5, None]]
5085+
table = pa.Table.from_arrays(arrays, schema=schema_nullable)
5086+
5087+
pq.write_to_dataset(table, tempdir / "nulltest1")
5088+
dataset = ds.dataset(tempdir / "nulltest1", format="parquet")
5089+
# nullability of field is preserved
5090+
assert dataset.to_table().schema.equals(schema_nullable)
5091+
5092+
ds.write_dataset(table, tempdir / "nulltest2", format="parquet")
5093+
dataset = ds.dataset(tempdir / "nulltest2", format="parquet")
5094+
assert dataset.to_table().schema.equals(schema_nullable)
5095+
5096+
ds.write_dataset([table, table], tempdir / "nulltest3", format="parquet")
5097+
dataset = ds.dataset(tempdir / "nulltest3", format="parquet")
5098+
assert dataset.to_table().schema.equals(schema_nullable)
5099+
5100+
5101+
def test_write_dataset_preserve_field_metadata(tempdir):
5102+
schema_metadata = pa.schema([
5103+
pa.field("x", pa.int64(), metadata={b'foo': b'bar'}),
5104+
pa.field("y", pa.int64())])
5105+
5106+
schema_no_meta = pa.schema([
5107+
pa.field("x", pa.int64()),
5108+
pa.field("y", pa.int64())])
5109+
5110+
arrays = [[1, 2, 3], [None, 5, None]]
5111+
table = pa.Table.from_arrays(arrays, schema=schema_metadata)
5112+
table_no_meta = pa.Table.from_arrays(arrays, schema=schema_no_meta)
5113+
5114+
# If no schema is provided the schema of the first table will be used
5115+
ds.write_dataset([table, table_no_meta], tempdir / "test1", format="parquet")
5116+
dataset = ds.dataset(tempdir / "test1", format="parquet")
5117+
assert dataset.to_table().schema.equals(schema_metadata, check_metadata=True)
5118+
5119+
ds.write_dataset([table_no_meta, table], tempdir / "test2", format="parquet")
5120+
dataset = ds.dataset(tempdir / "test2", format="parquet")
5121+
assert dataset.to_table().schema.equals(schema_no_meta, check_metadata=True)
5122+
5123+
# If a schema is provided it will override the schema of the input
5124+
ds.write_dataset([table_no_meta, table], tempdir / "test3", format="parquet",
5125+
schema=schema_metadata)
5126+
dataset = ds.dataset(tempdir / "test3", format="parquet")
5127+
assert dataset.to_table().schema.equals(schema_metadata, check_metadata=True)
5128+
5129+
50775130
@pytest.mark.parametrize('dstype', [
50785131
"fs", "mem"
50795132
])

r/R/arrowExports.R

Lines changed: 2 additions & 2 deletions
Some generated files are not rendered by default. Learn more about customizing how changed files appear on GitHub.

r/R/query-engine.R

Lines changed: 3 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -236,10 +236,12 @@ ExecPlan <- R6Class("ExecPlan",
236236
},
237237
Write = function(node, ...) {
238238
# TODO(ARROW-16200): take FileSystemDatasetWriteOptions not ...
239+
final_metadata <- prepare_key_value_metadata(node$final_metadata())
240+
239241
ExecPlan_Write(
240242
self,
241243
node,
242-
prepare_key_value_metadata(node$final_metadata()),
244+
node$schema$WithMetadata(final_metadata),
243245
...
244246
)
245247
},

0 commit comments

Comments
 (0)