You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@arrow.apache.org by "ASF GitHub Bot (JIRA)" <ji...@apache.org> on 2018/04/02 10:17:00 UTC

[jira] [Commented] (ARROW-2308) Serialized tensor data should be 64-byte aligned.

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

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

pitrou 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_r178520694
 
 

 ##########
 File path: cpp/src/arrow/ipc/reader.cc
 ##########
 @@ -715,7 +716,7 @@ Status ReadTensor(int64_t offset, io::RandomAccessFile* file,
   RETURN_NOT_OK(file->Seek(offset));
 
   std::unique_ptr<Message> message;
-  RETURN_NOT_OK(ReadContiguousPayload(file, &message));
+  RETURN_NOT_OK(ReadContiguousPayload(file, &message, true));
 
 Review comment:
   When passing bare bool literals like that as argument value, is it possible to add a comment with the argument name?
   Something like:
   ```cpp
     RETURN_NOT_OK(ReadContiguousPayload(file, &message, true /* aligned */));
   ```
   (not sure the above passes linting, to be honest)

----------------------------------------------------------------
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:
users@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
>            Priority: Major
>              Labels: pull-request-available
>
> 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)