Hi Andrew,

I do not see the attached code, maybe the attachments got stripped?  Is it
small enough to just inline in the message?

Bryan

On Wed, Sep 27, 2017 at 12:26 PM, Andrew Pham (BLOOMBERG/ 731 LEX) <
[email protected]> wrote:

> Also for reference, this is apparently Arrow Schema used by the
> ArrowFileWriter to write to the output stream (given by
> root.getSchema().toString() and root.getSchema().toJson()):
>
> Schema<price: FloatingPoint(DOUBLE), numShares: Int(32, true)>
> {
>   "fields" : [ {
>     "name" : "price",
>     "nullable" : true,
>     "type" : {
>       "name" : "floatingpoint",
>       "precision" : "DOUBLE"
>     },
>     "children" : [ ],
>     "typeLayout" : {
>       "vectors" : [ {
>         "type" : "VALIDITY",
>         "typeBitWidth" : 1
>       }, {
>         "type" : "DATA",
>         "typeBitWidth" : 64
>       } ]
>     }
>   }, {
>     "name" : "numShares",
>     "nullable" : true,
>     "type" : {
>       "name" : "int",
>       "bitWidth" : 32,
>       "isSigned" : true
>     },
>     "children" : [ ],
>     "typeLayout" : {
>       "vectors" : [ {
>         "type" : "VALIDITY",
>         "typeBitWidth" : 1
>       }, {
>         "type" : "DATA",
>         "typeBitWidth" : 32
>       } ]
>     }
>   } ]
> }
>
>
> Given our bytes (wrapped by a SeekableByteChannel), the reader is unable
> to obtain the schema from this.  Any ideas as to what could be happening?
> Cheers!
>
> From: [email protected] At: 09/26/17 18:59:18To:  Andrew Pham
> (BLOOMBERG/ 731 LEX ) ,  [email protected]
> Subject: Re: ArrowFileReader failing to read bytes written to Java output
> stream
>
> Andrew,
>
> Seems like it fails to read the schema. It has reached the data part yet.
> Can you share your reader/writer code?
>
> On Tue, Sep 26, 2017 at 6:37 PM, Andrew Pham (BLOOMBERG/ 731 LEX) <
> [email protected]> wrote:
>
> > Hello there, I've written something that behaves similarly to:
> >
> > https://github.com/apache/spark/blob/master/sql/core/
> > src/main/scala/org/apache/spark/sql/execution/arrow/
> > ArrowConverters.scala#L73
> >
> > Except that for proof of concept purposes, it transforms Java objects
> with
> > data into a byte[] payload.  The ArrowFileWriter log statements indicate
> > that data is getting written to the output stream:
> >
> > 17:53:16.759 [main] DEBUG org.apache.arrow.vector.file.WriteChannel -
> > Writing buffer with size: 6
> > 17:53:16.759 [main] DEBUG org.apache.arrow.vector.file.WriteChannel -
> > Writing buffer with size: 2
> > 17:53:16.766 [main] DEBUG org.apache.arrow.vector.file.WriteChannel -
> > Writing buffer with size: 4
> > 17:53:16.766 [main] DEBUG org.apache.arrow.vector.file.WriteChannel -
> > Writing buffer with size: 288
> > 17:53:16.766 [main] DEBUG org.apache.arrow.vector.file.WriteChannel -
> > Writing buffer with size: 4
> > 17:53:16.769 [main] DEBUG org.apache.arrow.vector.
> schema.ArrowRecordBatch
> > - Buffer in RecordBatch at 0, length: 1
> > 17:53:16.769 [main] DEBUG org.apache.arrow.vector.
> schema.ArrowRecordBatch
> > - Buffer in RecordBatch at 8, length: 24
> > 17:53:16.770 [main] DEBUG org.apache.arrow.vector.
> schema.ArrowRecordBatch
> > - Buffer in RecordBatch at 32, length: 1
> > 17:53:16.770 [main] DEBUG org.apache.arrow.vector.
> schema.ArrowRecordBatch
> > - Buffer in RecordBatch at 40, length: 12
> > 17:53:16.771 [main] DEBUG org.apache.arrow.vector.file.WriteChannel -
> > Writing buffer with size: 4
> > 17:53:16.771 [main] DEBUG org.apache.arrow.vector.file.WriteChannel -
> > Writing buffer with size: 216
> > 17:53:16.771 [main] DEBUG org.apache.arrow.vector.file.WriteChannel -
> > Writing buffer with size: 4
> > 17:53:16.771 [main] DEBUG org.apache.arrow.vector.file.WriteChannel -
> > Writing buffer with size: 1
> > 17:53:16.771 [main] DEBUG org.apache.arrow.vector.file.WriteChannel -
> > Writing buffer with size: 7
> > 17:53:16.771 [main] DEBUG org.apache.arrow.vector.file.WriteChannel -
> > Writing buffer with size: 24
> > 17:53:16.771 [main] DEBUG org.apache.arrow.vector.file.WriteChannel -
> > Writing buffer with size: 1
> > 17:53:16.771 [main] DEBUG org.apache.arrow.vector.file.WriteChannel -
> > Writing buffer with size: 7
> > 17:53:16.771 [main] DEBUG org.apache.arrow.vector.file.WriteChannel -
> > Writing buffer with size: 12
> > 17:53:16.771 [main] DEBUG org.apache.arrow.vector.file.WriteChannel -
> > Writing buffer with size: 4
> > 17:53:16.772 [main] DEBUG org.apache.arrow.vector.file.ArrowWriter -
> > RecordBatch at 304, metadata: 224, body: 56
> >
> >
> > However, when I wrap that payload into a ByteArrayReadableSeekableByteC
> hannel
> > and use ArrowFileReader (along with a BufferAllocator) to read it,
> > ArrowFileReader is complaining that it's reading an invalid format, right
> > at the point where I call reader.getVectorSchemaRoot():
> >
> > Exception in thread "main" org.apache.arrow.vector.file.
> InvalidArrowFileException:
> > missing Magic number [0, 0, 42, 0, 0, 0, 0, 0, 0, 0]
> >      at org.apache.arrow.vector.file.ArrowFileReader.readSchema(
> > ArrowFileReader.java:66)
> >   at org.apache.arrow.vector.file.ArrowFileReader.readSchema(
> > ArrowFileReader.java:37)
> >   at org.apache.arrow.vector.file.ArrowReader.initialize(
> > ArrowReader.java:162)
> >  at org.apache.arrow.vector.file.ArrowReader.ensureInitialized(
> > ArrowReader.java:153)
> >   at org.apache.arrow.vector.file.ArrowReader.getVectorSchemaRoot(
> > ArrowReader.java:67)
> >  at com.bloomberg.andrew.sql.execution.arrow.ArrowConverters.
> > byteArrayToBatch(ArrowConverters.java:89)
> >         at com.bloomberg.andrew.sql.execution.arrow.ArrowPayload.
> > loadBatch(ArrowPayload.java:18)
> >      at com.bloomberg.andrew.test.arrow.ArrowPublisher.main(
> > ArrowPublisher.java:28)
> >
> >
> > I'm noticing that the number 42 is exactly the same as the value of the
> > very last field/member in the very last object in our list (or
> equivalent,
> > the very last column of the very last row of our table), and if I try
> out a
> > bunch of different cases, this appears to be the case.  Clearly, I'm
> > writing stuff to the output stream...but any ideas as to why ArrowReader
> is
> > struggling?  There's some ideas regarding big endian/little endian stuff,
> > but I'm not sure if that was addressed or not.  Thanks!
>
>
>

Reply via email to