blob: f1c18d765d465d4e34f0bdfb2b2ce92367c99743 [file] [log] [blame]
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
include "Schema.fbs";
include "SparseTensor.fbs";
include "Tensor.fbs";
namespace org.apache.arrow.flatbuf;
/// ----------------------------------------------------------------------
/// Data structures for describing a table row batch (a collection of
/// equal-length Arrow arrays)
/// Metadata about a field at some level of a nested type tree (but not
/// its children).
///
/// For example, a List<Int16> with values `[[1, 2, 3], null, [4], [5, 6], null]`
/// would have {length: 5, null_count: 2} for its List node, and {length: 6,
/// null_count: 0} for its Int16 node, as separate FieldNode structs
struct FieldNode {
/// The number of value slots in the Arrow array at this level of a nested
/// tree
length: long;
/// The number of observed nulls. Fields with null_count == 0 may choose not
/// to write their physical validity bitmap out as a materialized buffer,
/// instead setting the length of the bitmap buffer to 0.
null_count: long;
}
enum CompressionType:byte {
// LZ4 frame format, for portability, as provided by lz4frame.h or wrappers
// thereof. Not to be confused with "raw" (also called "block") format
// provided by lz4.h
LZ4_FRAME,
// Zstandard
ZSTD
}
/// Provided for forward compatibility in case we need to support different
/// strategies for compressing the IPC message body (like whole-body
/// compression rather than buffer-level) in the future
enum BodyCompressionMethod:byte {
/// Each constituent buffer is first compressed with the indicated
/// compressor, and then written with the uncompressed length in the first 8
/// bytes as a 64-bit little-endian signed integer followed by the compressed
/// buffer bytes (and then padding as required by the protocol). The
/// uncompressed length may be set to -1 to indicate that the data that
/// follows is not compressed, which can be useful for cases where
/// compression does not yield appreciable savings.
BUFFER
}
/// Optional compression for the memory buffers constituting IPC message
/// bodies. Intended for use with RecordBatch but could be used for other
/// message types
table BodyCompression {
/// Compressor library
codec: CompressionType = LZ4_FRAME;
/// Indicates the way the record batch body was compressed
method: BodyCompressionMethod = BUFFER;
}
/// A data header describing the shared memory layout of a "record" or "row"
/// batch. Some systems call this a "row batch" internally and others a "record
/// batch".
table RecordBatch {
/// number of records / rows. The arrays in the batch should all have this
/// length
length: long;
/// Nodes correspond to the pre-ordered flattened logical schema
nodes: [FieldNode];
/// Buffers correspond to the pre-ordered flattened buffer tree
///
/// The number of buffers appended to this list depends on the schema. For
/// example, most primitive arrays will have 2 buffers, 1 for the validity
/// bitmap and 1 for the values. For struct arrays, there will only be a
/// single buffer for the validity (nulls) bitmap
buffers: [Buffer];
/// Optional compression of the message body
compression: BodyCompression;
}
/// For sending dictionary encoding information. Any Field can be
/// dictionary-encoded, but in this case none of its children may be
/// dictionary-encoded.
/// There is one vector / column per dictionary, but that vector / column
/// may be spread across multiple dictionary batches by using the isDelta
/// flag
table DictionaryBatch {
id: long;
data: RecordBatch;
/// If isDelta is true the values in the dictionary are to be appended to a
/// dictionary with the indicated id. If isDelta is false this dictionary
/// should replace the existing dictionary.
isDelta: bool = false;
}
/// ----------------------------------------------------------------------
/// The root Message type
/// This union enables us to easily send different message types without
/// redundant storage, and in the future we can easily add new message types.
///
/// Arrow implementations do not need to implement all of the message types,
/// which may include experimental metadata types. For maximum compatibility,
/// it is best to send data using RecordBatch
union MessageHeader {
Schema, DictionaryBatch, RecordBatch, Tensor, SparseTensor
}
table Message {
version: org.apache.arrow.flatbuf.MetadataVersion;
header: MessageHeader;
bodyLength: long;
custom_metadata: [ KeyValue ];
}
root_type Message;