Uploaded image for project: 'Apache Arrow'
  1. Apache Arrow
  2. ARROW-1194

Getting record batch size with pa.get_record_batch_size returns a size that is too small for pandas DataFrame.

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • 0.5.0
    • 0.5.0
    • Python
    • None
    • Ubuntu 16.04
      Python 3.6

    Description

      I'm running into the following problem.

      Suppose I create a dataframe and turn it into a record batch.

      import pyarrow as pa
      import pandas as pd
      
      df = pd.DataFrame({"a": [1, 2, 3]})
      record_batch = pa.RecordBatch.from_pandas(df)
      

      It's size is 352 according to

      pa.get_record_batch_size(record_batch)  # This is 352.
      

      However, if I write it using a stream_writer and then attempt to read it, the resulting buffer has size 928.

      sink = pa.BufferOutputStream()
      stream_writer = pa.RecordBatchStreamWriter(sink, record_batch.schema)
      stream_writer.write_batch(record_batch)
      new_buf = sink.get_result()
      new_buf.size  # This is 928.
      

      I'm running into this problem because I'm attempting to write the pandas DataFrame to the Plasma object store as follows (after Plasma has been started and a client has been created), so I need to know the size ahead of time.

      data_size = pa.get_record_batch_size(record_batch)
      object_id = plasma.ObjectID(np.random.bytes(20))
      
      buf = client.create(object_id, data_size)  # Note that if I replace "data_size" by "data_size + 1000" then it works.
      stream = plasma.FixedSizeBufferOutputStream(buf)
      stream_writer = pa.RecordBatchStreamWriter(stream, record_batch.schema)
      stream_writer.write_batch(record_batch)
      

      The above fails because the buffer allocated in Plasma only has size 352, but 928 bytes are needed.

      So my question is, am I determining the size of the record batch incorrectly? Or could there be a bug in pa.get_record_batch_size?

      Attachments

        Issue Links

          Activity

            People

              Unassigned Unassigned
              robertnishihara Robert Nishihara
              Votes:
              0 Vote for this issue
              Watchers:
              3 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: