You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@arrow.apache.org by "David Li (Jira)" <ji...@apache.org> on 2022/07/01 14:51:00 UTC

[jira] [Created] (ARROW-16958) [C++][FlightRPC] Flight generates misaligned buffers

David Li created ARROW-16958:
--------------------------------

             Summary: [C++][FlightRPC] Flight generates misaligned buffers
                 Key: ARROW-16958
                 URL: https://issues.apache.org/jira/browse/ARROW-16958
             Project: Apache Arrow
          Issue Type: Bug
          Components: C++, FlightRPC
            Reporter: David Li


Protobuf's wire format design + our zero-copy serializer/deserializer mean that buffers can end up misaligned. On some Arrow versions, this can cause segfaults in kernels assuming alignment (and generally violates expectations). 

We should:
* Possibly include buffer alignment in array validation
* See if we can adjust the serializer to somehow pad things properly
* See if we can do anything about this in the deserializer

Example:
{code:python}
import pyarrow as pa
import pyarrow.flight as flight

class TestServer(flight.FlightServerBase):
    def do_get(self, context, ticket):
        schema = pa.schema(
            [
                ("index", pa.int64()),
                ("int8", pa.float64()),
                ("int16", pa.float64()),
                ("int32", pa.float64()),
            ]
        )
        return flight.RecordBatchStream(pa.table([
            [0, 1, 2, 3],
            [0, 1, None, 3],
            [0, 1, 2, None],
            [0, None, 2, 3],
        ], schema=schema))


with TestServer() as server:
    client = flight.connect(f"grpc://localhost:{server.port}")
    table = client.do_get(flight.Ticket(b"")).read_all()
    for col in table:
        print(col.type)
        for chunk in col.chunks:
            for buf in chunk.buffers():
                if not buf: continue
                print("buffer is 8-byte aligned?", buf.address % 8)
            chunk.cast(pa.float32())
{code}

On Arrow 8
{noformat}
int64
buffer is 8-byte aligned? 1
double
buffer is 8-byte aligned? 1
buffer is 8-byte aligned? 1
double
buffer is 8-byte aligned? 1
buffer is 8-byte aligned? 1
double
buffer is 8-byte aligned? 1
buffer is 8-byte aligned? 1
{noformat}
On Arrow 7
{noformat}
int64
buffer is 8-byte aligned? 4
double
buffer is 8-byte aligned? 4
buffer is 8-byte aligned? 4
fish: Job 1, 'python ../test.py' terminated by signal SIGSEGV (Address boundary error)
{noformat}



--
This message was sent by Atlassian Jira
(v8.20.10#820010)