-
Notifications
You must be signed in to change notification settings - Fork 3.6k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
GH-40860: [GLib][Parquet] Add `gparquet_arrow_file_writer_write_recor…
…d_batch()` (#44001) ### Rationale for this change We don't need to create a `GArrowTable` only for writing a `GArrowRecordBatch`. ### What changes are included in this PR? The following APIs are also added: * `gparquet_arrow_file_writer_get_schema()` * Parquet::ArrowFileWriter#write` in Ruby ### Are these changes tested? Yes. ### Are there any user-facing changes? Yes. * GitHub Issue: #40860 Authored-by: Sutou Kouhei <[email protected]> Signed-off-by: Sutou Kouhei <[email protected]>
- Loading branch information
Showing
6 changed files
with
262 additions
and
7 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,98 @@ | ||
# 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. | ||
|
||
module Parquet | ||
class ArrowFileWriter | ||
# Write data to Apache Parquet. | ||
# | ||
# @return [void] | ||
# | ||
# @overload write(record_batch) | ||
# | ||
# @param record_batch [Arrow::RecordBatch] The record batch to | ||
# be written. | ||
# | ||
# @example Write a record batch | ||
# record_batch = Arrow::RecordBatch.new(enabled: [true, false]) | ||
# schema = record_batch.schema | ||
# Parquet::ArrowFileWriter.open(schema, "data.parquet") do |writer| | ||
# writer.write(record_batch) | ||
# end | ||
# | ||
# @overload write(table, chunk_size: nil) | ||
# | ||
# @param table [Arrow::Table] The table to be written. | ||
# | ||
# @param chunk_size [nil, Integer] (nil) The maximum number of | ||
# rows to write per row group. | ||
# | ||
# If this is `nil`, the default value (`1024 * 1024`) is used. | ||
# | ||
# @example Write a record batch with the default chunk size | ||
# table = Arrow::Table.new(enabled: [true, false]) | ||
# schema = table.schema | ||
# Parquet::ArrowFileWriter.open(schema, "data.parquet") do |writer| | ||
# writer.write(table) | ||
# end | ||
# | ||
# @example Write a record batch with the specified chunk size | ||
# table = Arrow::Table.new(enabled: [true, false]) | ||
# schema = table.schema | ||
# Parquet::ArrowFileWriter.open(schema, "data.parquet") do |writer| | ||
# writer.write(table, chunk_size: 1) | ||
# end | ||
# | ||
# @overload write(raw_records) | ||
# | ||
# @param data [Array<Hash>, Array<Array>] The data to be written | ||
# as primitive Ruby objects. | ||
# | ||
# @example Write a record batch with Array<Array> based data | ||
# schema = Arrow::Schema.new(enabled: :boolean) | ||
# raw_records = [ | ||
# [true], | ||
# [false], | ||
# ] | ||
# Parquet::ArrowFileWriter.open(schema, "data.parquet") do |writer| | ||
# writer.write(raw_records) | ||
# end | ||
# | ||
# @example Write a record batch with Array<Hash> based data | ||
# schema = Arrow::Schema.new(enabled: :boolean) | ||
# raw_columns = [ | ||
# enabled: [true, false], | ||
# ] | ||
# Parquet::ArrowFileWriter.open(schema, "data.parquet") do |writer| | ||
# writer.write(raw_columns) | ||
# end | ||
# | ||
# @since 18.0.0 | ||
def write(target, chunk_size: nil) | ||
case target | ||
when Arrow::RecordBatch | ||
write_record_batch(target) | ||
when Arrow::Table | ||
# Same as parquet::DEFAULT_MAX_ROW_GROUP_LENGTH in C++ | ||
chunk_size ||= 1024 * 1024 | ||
write_table(target, chunk_size) | ||
else | ||
record_batch = Arrow::RecordBatch.new(schema, target) | ||
write_record_batch(record_batch) | ||
end | ||
end | ||
end | ||
end |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,76 @@ | ||
# 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. | ||
|
||
class TestArrowFileWriter < Test::Unit::TestCase | ||
def open_buffer_output_stream | ||
buffer = Arrow::ResizableBuffer.new(4096) | ||
Arrow::BufferOutputStream.open(buffer) do |output| | ||
yield(output) | ||
end | ||
buffer | ||
end | ||
|
||
sub_test_case("#write") do | ||
test("RecordBatch") do | ||
schema = Arrow::Schema.new(visible: :boolean) | ||
record_batch = Arrow::RecordBatch.new(schema, [[true], [false]]) | ||
buffer = open_buffer_output_stream do |output| | ||
Parquet::ArrowFileWriter.open(record_batch.schema, output) do |writer| | ||
writer.write(record_batch) | ||
end | ||
end | ||
assert_equal(record_batch.to_table, | ||
Arrow::Table.load(buffer, format: :parquet)) | ||
end | ||
|
||
test("Table") do | ||
schema = Arrow::Schema.new(visible: :boolean) | ||
table = Arrow::Table.new(schema, [[true], [false]]) | ||
buffer = open_buffer_output_stream do |output| | ||
Parquet::ArrowFileWriter.open(table.schema, output) do |writer| | ||
writer.write(table) | ||
end | ||
end | ||
assert_equal(table, | ||
Arrow::Table.load(buffer, format: :parquet)) | ||
end | ||
|
||
test("[[]]") do | ||
schema = Arrow::Schema.new(visible: :boolean) | ||
raw_records = [[true], [false]] | ||
buffer = open_buffer_output_stream do |output| | ||
Parquet::ArrowFileWriter.open(schema, output) do |writer| | ||
writer.write(raw_records) | ||
end | ||
end | ||
assert_equal(Arrow::RecordBatch.new(schema, raw_records).to_table, | ||
Arrow::Table.load(buffer, format: :parquet)) | ||
end | ||
|
||
test("[{}]") do | ||
schema = Arrow::Schema.new(visible: :boolean) | ||
raw_columns = [visible: [true, false]] | ||
buffer = open_buffer_output_stream do |output| | ||
Parquet::ArrowFileWriter.open(schema, output) do |writer| | ||
writer.write(raw_columns) | ||
end | ||
end | ||
assert_equal(Arrow::RecordBatch.new(schema, raw_columns).to_table, | ||
Arrow::Table.load(buffer, format: :parquet)) | ||
end | ||
end | ||
end |