Skip to content

Conversation

@tianchen92
Copy link
Contributor

Related to ARROW-5861.
Initial implement to support convert Avro record with primitive types to Arrow objects.

@codecov-io
Copy link

codecov-io commented Jul 6, 2019

Codecov Report

Merging #4812 into master will increase coverage by 2.16%.
The diff coverage is n/a.

Impacted file tree graph

@@            Coverage Diff             @@
##           master    #4812      +/-   ##
==========================================
+ Coverage   87.42%   89.58%   +2.16%     
==========================================
  Files         994      661     -333     
  Lines      140102    96617   -43485     
  Branches     1418        0    -1418     
==========================================
- Hits       122481    86557   -35924     
+ Misses      17259    10060    -7199     
+ Partials      362        0     -362
Impacted Files Coverage Δ
cpp/src/gandiva/precompiled/decimal_ops.cc 97.17% <0%> (ø) ⬆️
cpp/src/gandiva/precompiled/decimal_ops_test.cc 97.43% <0%> (ø) ⬆️
r/src/recordbatch.cpp
r/R/Table.R
js/src/util/fn.ts
go/arrow/array/bufferbuilder.go
r/src/symbols.cpp
rust/datafusion/src/execution/projection.rs
rust/datafusion/src/execution/filter.rs
rust/arrow/src/csv/writer.rs
... and 327 more

Continue to review full report at Codecov.

Legend - Click here to learn more
Δ = absolute <relative> (impact), ø = not affected, ? = missing data
Powered by Codecov. Last update c350bba...2439478. Read the comment docs.


allocateVectors(root, DEFAULT_BUFFER_SIZE);

List<Consumer> consumers = createAvroConsumers(root);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

it seems like we might need to understand more about decoders and the order they are expected to [decode(https://avro.apache.org/docs/1.8.2/api/java/org/apache/avro/io/ValidatingDecoder.html) in. This can also be for a follow-up PR

@Override
public void consume(Decoder decoder) throws IOException {
VarBinaryHolder holder = new VarBinaryHolder();
ByteBuffer byteBuffer = decoder.readBytes(null);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

we should probably keep a cached byte buffer for values below a certain size.

Even better would be if we could create a ByteBuffer that wraps the Vector and puts bytes directly where they need to go. This can be done in a separate PR.

Copy link
Contributor

@emkornfield emkornfield Jul 16, 2019

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can we address the first part of my comment (caching a ByteBuffer instead of creating a new one each time)?

Otherwise LGTM as start. @tianchen92

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sure, I will create a new JIRA to address this, thanks!

Copy link
Contributor

@emkornfield emkornfield left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks this looks a lot closer to what I was expecting. It still seems like there might be some additional details we need to learn about the AVRO API but I think this can be done as a follow-up

@tianchen92
Copy link
Contributor Author

Thanks this looks a lot closer to what I was expecting. It still seems like there might be some additional details we need to learn about the AVRO API but I think this can be done as a follow-up

Many thanks for your feedback :), revise would be provided later.

@tianchen92
Copy link
Contributor Author

tianchen92 commented Jul 18, 2019

@emkornfield Hi, Micah, could we get this PR merged? thanks :)
BTW, ask an unrelated question, seems theres no adapter to read csv to ARROW in Java which C++ has, is it necessary to implement it ?

And in the follow PR, I resolve most the comments and still have few questions, would you mind explaining a lot? thanks a lot!
#4846

import org.apache.avro.Schema.Type;
import org.apache.avro.io.Decoder;

import sun.reflect.generics.reflectiveObjects.NotImplementedException;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Fixed, thanks! hope could merge soon then I could start follow-up works

@emkornfield
Copy link
Contributor

An adapter for CSV would probably be nice as well. I don't know if there is an efficient CSV parser in java that we could potentially already leverage (or if we want bindings to the C++ version). might be worth an e-mail to the mailing list to get opinions.

@tianchen92
Copy link
Contributor Author

An adapter for CSV would probably be nice as well. I don't know if there is an efficient CSV parser in java that we could potentially already leverage (or if we want bindings to the C++ version). might be worth an e-mail to the mailing list to get opinions.

Maybe https://commons.apache.org/proper/commons-csv/ is a good choice.

@emkornfield
Copy link
Contributor

Maybe https://commons.apache.org/proper/commons-csv/ is a good choice.

Seems reasonable to me. Please ask on the mailing list in case anyone else has input.

@tianchen92
Copy link
Contributor Author

Maybe https://commons.apache.org/proper/commons-csv/ is a good choice.

Seems reasonable to me. Please ask on the mailing list in case anyone else has input.

Thanks for your effort, I will start discuss later :)

@tianchen92
Copy link
Contributor Author

@emkornfield Build passed :)

@emkornfield
Copy link
Contributor

+1, thank you.

kszucs pushed a commit that referenced this pull request Jul 22, 2019
…itive types

Related to [ARROW-5861](https://issues.apache.org/jira/browse/ARROW-5861).
Initial implement to support convert Avro record with primitive types to Arrow objects.

Author: tianchen <niki.lj@alibaba-inc.com>

Closes #4812 from tianchen92/ARROW-5861 and squashes the following commits:

2439478 <tianchen> use UnsupportedOperationException
fa3f39a <tianchen> resolve comments
7c3a730 <tianchen> add consumers and use GenericDatumReader
61d2dac <tianchen> fix style
54479c8 <tianchen> Initial implement to convert Avro record with primitive types
pribor pushed a commit to GlobalWebIndex/arrow that referenced this pull request Oct 24, 2025
…itive types

Related to [ARROW-5861](https://issues.apache.org/jira/browse/ARROW-5861).
Initial implement to support convert Avro record with primitive types to Arrow objects.

Author: tianchen <niki.lj@alibaba-inc.com>

Closes apache#4812 from tianchen92/ARROW-5861 and squashes the following commits:

2439478 <tianchen> use UnsupportedOperationException
fa3f39a <tianchen> resolve comments
7c3a730 <tianchen> add consumers and use GenericDatumReader
61d2dac <tianchen> fix style
54479c8 <tianchen> Initial implement to convert Avro record with primitive types
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

None yet

Development

Successfully merging this pull request may close these issues.

4 participants