[ 
https://issues.apache.org/jira/browse/ARROW-2308?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16426389#comment-16426389
 ] 

ASF GitHub Bot commented on ARROW-2308:
---------------------------------------

wesm commented on a change in pull request #1802: ARROW-2308: [Python] Make 
deserialized numpy arrays 64-byte aligned.
URL: https://github.com/apache/arrow/pull/1802#discussion_r179325308
 
 

 ##########
 File path: cpp/src/arrow/ipc/message.cc
 ##########
 @@ -233,6 +235,16 @@ Status ReadMessage(io::InputStream* file, 
std::unique_ptr<Message>* message) {
     return Status::Invalid(ss.str());
   }
 
+  // If requested, align the file before reading the message.
+  if (aligned) {
+    int64_t offset;
+    RETURN_NOT_OK(file->Tell(&offset));
+    int64_t aligned_offset = PaddedLength(offset);
+    int64_t num_extra_bytes = aligned_offset - offset;
+    std::shared_ptr<Buffer> dummy_buffer;
+    RETURN_NOT_OK(file->Read(num_extra_bytes, &dummy_buffer));
 
 Review comment:
   I'm not sure this is the right change. Shouldn't the alignment come from 
padding after the metadata? 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Serialized tensor data should be 64-byte aligned.
> -------------------------------------------------
>
>                 Key: ARROW-2308
>                 URL: https://issues.apache.org/jira/browse/ARROW-2308
>             Project: Apache Arrow
>          Issue Type: Improvement
>          Components: Python
>            Reporter: Robert Nishihara
>            Assignee: Robert Nishihara
>            Priority: Major
>              Labels: pull-request-available
>             Fix For: 0.10.0
>
>
> See [https://github.com/ray-project/ray/issues/1658] for an example of this 
> issue. Non-aligned data can trigger a copy when fed into TensorFlow and 
> things like that.
> {code}
> import pyarrow as pa
> import numpy as np
> x = np.zeros(10)
> y = pa.deserialize(pa.serialize(x).to_buffer())
> x.ctypes.data % 64  # 0 (it starts out aligned)
> y.ctypes.data % 64  # 48 (it is no longer aligned)
> {code}
> It should be possible to fix this by calling something like 
> {{RETURN_NOT_OK(AlignStreamPosition(dst));}} before writing the array data. 
> Note that we already do this before writing the tensor header, but the tensor 
> header is not necessarily a multiple of 64 bytes, so the subsequent data can 
> be unaligned.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to