|
| 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 "SparseTensor.fbs"; |
| 20 | +include "Tensor.fbs"; |
| 21 | + |
| 22 | +namespace org.apache.arrow.flatbuf; |
| 23 | + |
| 24 | +/// ---------------------------------------------------------------------- |
| 25 | +/// Data structures for describing a table row batch (a collection of |
| 26 | +/// equal-length Arrow arrays) |
| 27 | + |
| 28 | +/// Metadata about a field at some level of a nested type tree (but not |
| 29 | +/// its children). |
| 30 | +/// |
| 31 | +/// For example, a List<Int16> with values `[[1, 2, 3], null, [4], [5, 6], null]` |
| 32 | +/// would have {length: 5, null_count: 2} for its List node, and {length: 6, |
| 33 | +/// null_count: 0} for its Int16 node, as separate FieldNode structs |
| 34 | +struct FieldNode { |
| 35 | + /// The number of value slots in the Arrow array at this level of a nested |
| 36 | + /// tree |
| 37 | + length: long; |
| 38 | + |
| 39 | + /// The number of observed nulls. Fields with null_count == 0 may choose not |
| 40 | + /// to write their physical validity bitmap out as a materialized buffer, |
| 41 | + /// instead setting the length of the bitmap buffer to 0. |
| 42 | + null_count: long; |
| 43 | +} |
| 44 | + |
| 45 | +enum CompressionType:byte { |
| 46 | + // LZ4 frame format, for portability, as provided by lz4frame.h or wrappers |
| 47 | + // thereof. Not to be confused with "raw" (also called "block") format |
| 48 | + // provided by lz4.h |
| 49 | + LZ4_FRAME, |
| 50 | + |
| 51 | + // Zstandard |
| 52 | + ZSTD |
| 53 | +} |
| 54 | + |
| 55 | +/// Provided for forward compatibility in case we need to support different |
| 56 | +/// strategies for compressing the IPC message body (like whole-body |
| 57 | +/// compression rather than buffer-level) in the future |
| 58 | +enum BodyCompressionMethod:byte { |
| 59 | + /// Each constituent buffer is first compressed with the indicated |
| 60 | + /// compressor, and then written with the uncompressed length in the first 8 |
| 61 | + /// bytes as a 64-bit little-endian signed integer followed by the compressed |
| 62 | + /// buffer bytes (and then padding as required by the protocol). The |
| 63 | + /// uncompressed length may be set to -1 to indicate that the data that |
| 64 | + /// follows is not compressed, which can be useful for cases where |
| 65 | + /// compression does not yield appreciable savings. |
| 66 | + BUFFER |
| 67 | +} |
| 68 | + |
| 69 | +/// Optional compression for the memory buffers constituting IPC message |
| 70 | +/// bodies. Intended for use with RecordBatch but could be used for other |
| 71 | +/// message types |
| 72 | +table BodyCompression { |
| 73 | + /// Compressor library. |
| 74 | + /// For LZ4_FRAME, each compressed buffer must consist of a single frame. |
| 75 | + codec: CompressionType = LZ4_FRAME; |
| 76 | + |
| 77 | + /// Indicates the way the record batch body was compressed |
| 78 | + method: BodyCompressionMethod = BUFFER; |
| 79 | +} |
| 80 | + |
| 81 | +/// A data header describing the shared memory layout of a "record" or "row" |
| 82 | +/// batch. Some systems call this a "row batch" internally and others a "record |
| 83 | +/// batch". |
| 84 | +table RecordBatch { |
| 85 | + /// number of records / rows. The arrays in the batch should all have this |
| 86 | + /// length |
| 87 | + length: long; |
| 88 | + |
| 89 | + /// Nodes correspond to the pre-ordered flattened logical schema |
| 90 | + nodes: [FieldNode]; |
| 91 | + |
| 92 | + /// Buffers correspond to the pre-ordered flattened buffer tree |
| 93 | + /// |
| 94 | + /// The number of buffers appended to this list depends on the schema. For |
| 95 | + /// example, most primitive arrays will have 2 buffers, 1 for the validity |
| 96 | + /// bitmap and 1 for the values. For struct arrays, there will only be a |
| 97 | + /// single buffer for the validity (nulls) bitmap |
| 98 | + buffers: [Buffer]; |
| 99 | + |
| 100 | + /// Optional compression of the message body |
| 101 | + compression: BodyCompression; |
| 102 | + |
| 103 | + /// Some types such as Utf8View are represented using a variable number of buffers. |
| 104 | + /// For each such Field in the pre-ordered flattened logical schema, there will be |
| 105 | + /// an entry in variadicBufferCounts to indicate the number of number of variadic |
| 106 | + /// buffers which belong to that Field in the current RecordBatch. |
| 107 | + /// |
| 108 | + /// For example, the schema |
| 109 | + /// col1: Struct<alpha: Int32, beta: BinaryView, gamma: Float64> |
| 110 | + /// col2: Utf8View |
| 111 | + /// contains two Fields with variadic buffers so variadicBufferCounts will have |
| 112 | + /// two entries, the first counting the variadic buffers of `col1.beta` and the |
| 113 | + /// second counting `col2`'s. |
| 114 | + /// |
| 115 | + /// This field may be omitted if and only if the schema contains no Fields with |
| 116 | + /// a variable number of buffers, such as BinaryView and Utf8View. |
| 117 | + variadicBufferCounts: [long]; |
| 118 | +} |
| 119 | + |
| 120 | +/// For sending dictionary encoding information. Any Field can be |
| 121 | +/// dictionary-encoded, but in this case none of its children may be |
| 122 | +/// dictionary-encoded. |
| 123 | +/// There is one vector / column per dictionary, but that vector / column |
| 124 | +/// may be spread across multiple dictionary batches by using the isDelta |
| 125 | +/// flag |
| 126 | + |
| 127 | +table DictionaryBatch { |
| 128 | + id: long; |
| 129 | + data: RecordBatch; |
| 130 | + |
| 131 | + /// If isDelta is true the values in the dictionary are to be appended to a |
| 132 | + /// dictionary with the indicated id. If isDelta is false this dictionary |
| 133 | + /// should replace the existing dictionary. |
| 134 | + isDelta: bool = false; |
| 135 | +} |
| 136 | + |
| 137 | +/// ---------------------------------------------------------------------- |
| 138 | +/// The root Message type |
| 139 | + |
| 140 | +/// This union enables us to easily send different message types without |
| 141 | +/// redundant storage, and in the future we can easily add new message types. |
| 142 | +/// |
| 143 | +/// Arrow implementations do not need to implement all of the message types, |
| 144 | +/// which may include experimental metadata types. For maximum compatibility, |
| 145 | +/// it is best to send data using RecordBatch |
| 146 | +union MessageHeader { |
| 147 | + Schema, DictionaryBatch, RecordBatch, Tensor, SparseTensor |
| 148 | +} |
| 149 | + |
| 150 | +table Message { |
| 151 | + version: org.apache.arrow.flatbuf.MetadataVersion; |
| 152 | + header: MessageHeader; |
| 153 | + bodyLength: long; |
| 154 | + custom_metadata: [ KeyValue ]; |
| 155 | +} |
| 156 | + |
| 157 | +root_type Message; |
0 commit comments