github.com/cockroachdb/cockroach@v20.2.0-alpha.1+incompatible/pkg/col/colserde/arrowserde/message.fbs (about)

     1  // Licensed to the Apache Software Foundation (ASF) under one
     2  // or more contributor license agreements.  See the NOTICE file
     3  // distributed with this work for additional information
     4  // regarding copyright ownership.  The ASF licenses this file
     5  // to you under the Apache License, Version 2.0 (the
     6  // "License"); you may not use this file except in compliance
     7  // with the License.  You may obtain a copy of the License at
     8  //
     9  //   http://www.apache.org/licenses/LICENSE-2.0
    10  //
    11  // Unless required by applicable law or agreed to in writing,
    12  // software distributed under the License is distributed on an
    13  // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    14  // KIND, either express or implied.  See the License for the
    15  // specific language governing permissions and limitations
    16  // under the License.
    17  
    18  include "schema.fbs";
    19  include "tensor.fbs";
    20  
    21  namespace org.apache.arrow.flatbuf;
    22  
    23  /// ----------------------------------------------------------------------
    24  /// Data structures for describing a table row batch (a collection of
    25  /// equal-length Arrow arrays)
    26  
    27  /// Metadata about a field at some level of a nested type tree (but not
    28  /// its children).
    29  ///
    30  /// For example, a List<Int16> with values [[1, 2, 3], null, [4], [5, 6], null]
    31  /// would have {length: 5, null_count: 2} for its List node, and {length: 6,
    32  /// null_count: 0} for its Int16 node, as separate FieldNode structs
    33  struct FieldNode {
    34    /// The number of value slots in the Arrow array at this level of a nested
    35    /// tree
    36    length: long;
    37  
    38    /// The number of observed nulls. Fields with null_count == 0 may choose not
    39    /// to write their physical validity bitmap out as a materialized buffer,
    40    /// instead setting the length of the bitmap buffer to 0.
    41    null_count: long;
    42  }
    43  
    44  /// A data header describing the shared memory layout of a "record" or "row"
    45  /// batch. Some systems call this a "row batch" internally and others a "record
    46  /// batch".
    47  table RecordBatch {
    48    /// number of records / rows. The arrays in the batch should all have this
    49    /// length
    50    length: long;
    51  
    52    /// Nodes correspond to the pre-ordered flattened logical schema
    53    nodes: [FieldNode];
    54  
    55    /// Buffers correspond to the pre-ordered flattened buffer tree
    56    ///
    57    /// The number of buffers appended to this list depends on the schema. For
    58    /// example, most primitive arrays will have 2 buffers, 1 for the validity
    59    /// bitmap and 1 for the values. For struct arrays, there will only be a
    60    /// single buffer for the validity (nulls) bitmap
    61    buffers: [Buffer];
    62  }
    63  
    64  /// For sending dictionary encoding information. Any Field can be
    65  /// dictionary-encoded, but in this case none of its children may be
    66  /// dictionary-encoded.
    67  /// There is one vector / column per dictionary, but that vector / column
    68  /// may be spread across multiple dictionary batches by using the isDelta
    69  /// flag
    70  
    71  table DictionaryBatch {
    72    id: long;
    73    data: RecordBatch;
    74  
    75    /// If isDelta is true the values in the dictionary are to be appended to a
    76    /// dictionary with the indicated id
    77    isDelta: bool = false;
    78  }
    79  
    80  /// ----------------------------------------------------------------------
    81  /// The root Message type
    82  
    83  /// This union enables us to easily send different message types without
    84  /// redundant storage, and in the future we can easily add new message types.
    85  ///
    86  /// Arrow implementations do not need to implement all of the message types,
    87  /// which may include experimental metadata types. For maximum compatibility,
    88  /// it is best to send data using RecordBatch
    89  union MessageHeader {
    90    Schema, DictionaryBatch, RecordBatch, Tensor, SparseTensor
    91  }
    92  
    93  table Message {
    94    version: org.apache.arrow.flatbuf.MetadataVersion;
    95    header: MessageHeader;
    96    bodyLength: long;
    97  }
    98  
    99  root_type Message;