You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@arrow.apache.org by "David Li (Jira)" <ji...@apache.org> on 2020/05/10 00:39:00 UTC

[jira] [Comment Edited] (ARROW-8749) [C++] IpcFormatWriter writes dictionary batches with wrong ID

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

David Li edited comment on ARROW-8749 at 5/10/20, 12:38 AM:
------------------------------------------------------------

So the important thing here is that in this test, the schema comes from a different (but identical value/schema-wise) batch. This is something the Flight tests do but the IPC tests don't do.

{code:cpp}
      ASSERT_OK_AND_ASSIGN(auto sink, arrow::io::BufferOutputStream::Create());
      BatchVector expected_batches;
      std::shared_ptr<RecordBatch> batch;
      ASSERT_OK(MakeDictionary(&batch));
      std::shared_ptr<Schema> schema = batch->schema();
      // Uncomment this and the test will fail
      // ASSERT_OK(MakeDictionary(&batch));
      expected_batches.push_back(batch);
      ASSERT_OK_AND_ASSIGN(auto writer, arrow::ipc::NewStreamWriter(sink.get(), schema));
      for (auto& batch : expected_batches) {
        ASSERT_OK(writer->WriteRecordBatch(*batch));
      }
      ASSERT_OK_AND_ASSIGN(auto buf, sink->Finish());
      arrow::io::BufferReader source(buf);
      ASSERT_OK_AND_ASSIGN(auto reader, arrow::ipc::RecordBatchStreamReader::Open(&source));
      AssertSchemaEqual(schema, reader->schema());
      for (auto& batch : expected_batches) {
        ASSERT_OK_AND_ASSIGN(auto actual, reader->Next());
        AssertBatchesEqual(*actual, *batch);
      }
{code}


was (Author: lidavidm):
So the important thing here is that in this test, the schema comes from a different (but identical value/schema-wise) batch.

{code:cpp}
      ASSERT_OK_AND_ASSIGN(auto sink, arrow::io::BufferOutputStream::Create());
      BatchVector expected_batches;
      std::shared_ptr<RecordBatch> batch;
      ASSERT_OK(MakeDictionary(&batch));
      std::shared_ptr<Schema> schema = batch->schema();
      // Uncomment this and the test will fail
      // ASSERT_OK(MakeDictionary(&batch));
      expected_batches.push_back(batch);
      ASSERT_OK_AND_ASSIGN(auto writer, arrow::ipc::NewStreamWriter(sink.get(), schema));
      for (auto& batch : expected_batches) {
        ASSERT_OK(writer->WriteRecordBatch(*batch));
      }
      ASSERT_OK_AND_ASSIGN(auto buf, sink->Finish());
      arrow::io::BufferReader source(buf);
      ASSERT_OK_AND_ASSIGN(auto reader, arrow::ipc::RecordBatchStreamReader::Open(&source));
      AssertSchemaEqual(schema, reader->schema());
      for (auto& batch : expected_batches) {
        ASSERT_OK_AND_ASSIGN(auto actual, reader->Next());
        AssertBatchesEqual(*actual, *batch);
      }
{code}

> [C++] IpcFormatWriter writes dictionary batches with wrong ID
> -------------------------------------------------------------
>
>                 Key: ARROW-8749
>                 URL: https://issues.apache.org/jira/browse/ARROW-8749
>             Project: Apache Arrow
>          Issue Type: Bug
>          Components: C++
>    Affects Versions: 0.16.0, 0.17.0
>            Reporter: David Li
>            Priority: Major
>             Fix For: 1.0.0
>
>
> IpcFormatWriter assigns dictionary IDs once when it writes the schema message. Then, when it writes dictionary batches, it assigns dictionary IDs again because it re-collects dictionaries from the given batch. So for example, if you have 5 dictionaries, the first dictionary will end up with ID 0 but be written with ID 5.
> For example, this will fail with "'_error_or_value11.status()' failed with Key error: No record of dictionary type with id 9"
> {code:cpp}
> TEST_F(TestMetadata, DoPutDictionaries) {
>   ASSERT_OK_AND_ASSIGN(auto sink, arrow::io::BufferOutputStream::Create());
>   std::shared_ptr<Schema> schema = ExampleDictSchema();
>   BatchVector expected_batches;
>   ASSERT_OK(ExampleDictBatches(&expected_batches));
>   ASSERT_OK_AND_ASSIGN(auto writer, arrow::ipc::NewStreamWriter(sink.get(), schema));
>   for (auto& batch : expected_batches) {
>     ASSERT_OK(writer->WriteRecordBatch(*batch));
>   }
>   ASSERT_OK_AND_ASSIGN(auto buf, sink->Finish());
>   arrow::io::BufferReader source(buf);
>   ASSERT_OK_AND_ASSIGN(auto reader, arrow::ipc::RecordBatchStreamReader::Open(&source));
>   AssertSchemaEqual(schema, reader->schema());
>   for (auto& batch : expected_batches) {
>     ASSERT_OK_AND_ASSIGN(auto actual, reader->Next());
>     AssertBatchesEqual(*actual, *batch);
>   }
> }{code}



--
This message was sent by Atlassian Jira
(v8.3.4#803005)