You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by ks...@apache.org on 2019/04/08 11:04:59 UTC

[arrow] branch master updated: ARROW-2467: [Rust] Add generated IPC code

This is an automated email from the ASF dual-hosted git repository.

kszucs pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/arrow.git


The following commit(s) were added to refs/heads/master by this push:
     new e881aef  ARROW-2467: [Rust] Add generated IPC code
e881aef is described below

commit e881aef02eb24248bde309e3c37fd28967ff7f01
Author: Andy Grove <an...@gmail.com>
AuthorDate: Mon Apr 8 13:04:40 2019 +0200

    ARROW-2467: [Rust] Add generated IPC code
    
    I generated the IPC code after installing the flatbuffers release available via conda but still had to manually modify the code to make it compile due to these issues with flatbuffers:
    
    https://github.com/google/flatbuffers/issues/5052
    https://github.com/google/flatbuffers/issues/5053
    https://github.com/google/flatbuffers/issues/5054
    
    I think this is the best we can do for now.
    
    Author: Andy Grove <an...@gmail.com>
    
    Closes #4021 from andygrove/rust-ipc and squashes the following commits:
    
    7a549be9e <Andy Grove> encode field data type
    ab769bba5 <Andy Grove> delete file
    a81bee9b3 <Andy Grove> fix CI
    1bbc4a76b <Andy Grove> Update README
    83f3b4c06 <Andy Grove> Add generated IPC code
---
 rust/arrow/Cargo.toml                     |    1 +
 rust/arrow/README.md                      |   14 +
 rust/arrow/src/ipc/convert.rs             |   76 +
 rust/arrow/src/ipc/gen/File.rs            |  282 +++
 rust/arrow/src/ipc/gen/Message.rs         |  698 ++++++++
 rust/arrow/src/ipc/gen/Schema.rs          | 2766 +++++++++++++++++++++++++++++
 rust/arrow/src/ipc/gen/SparseTensor.rs    |  760 ++++++++
 rust/arrow/src/ipc/gen/Tensor.rs          |  494 ++++++
 rust/arrow/src/{lib.rs => ipc/gen/mod.rs} |   29 +-
 rust/arrow/src/{lib.rs => ipc/mod.rs}     |   29 +-
 rust/arrow/src/lib.rs                     |    1 +
 11 files changed, 5106 insertions(+), 44 deletions(-)

diff --git a/rust/arrow/Cargo.toml b/rust/arrow/Cargo.toml
index 47d5f78..70d2ddb 100644
--- a/rust/arrow/Cargo.toml
+++ b/rust/arrow/Cargo.toml
@@ -47,6 +47,7 @@ regex = "1.1"
 lazy_static = "1.2"
 packed_simd = "0.3.1"
 chrono = "0.4"
+flatbuffers = "0.5.0"
 
 [dev-dependencies]
 criterion = "0.2"
diff --git a/rust/arrow/README.md b/rust/arrow/README.md
index b2fd520..f3d6184 100644
--- a/rust/arrow/README.md
+++ b/rust/arrow/README.md
@@ -50,6 +50,20 @@ cargo run --example dynamic_types
 cargo run --example read_csv
 ```
 
+## IPC
+
+The IPC flatbuffer code was generated by running this command from the root of the project, using flatc version 1.10.0:
+
+```bash
+flatc --rust -o rust/arrow/src/ipc/gen/ format/*.fbs
+```
+
+Some manual steps were then performed:
+
+- Replace `type__type` with `type_type`
+- Remove `org::apache::arrow::flatbuffers` namespace
+- Add includes to each generated file
+
 # Publishing to crates.io
 
 An Arrow committer can publish this crate after an official project release has
diff --git a/rust/arrow/src/ipc/convert.rs b/rust/arrow/src/ipc/convert.rs
new file mode 100644
index 0000000..abcabd7
--- /dev/null
+++ b/rust/arrow/src/ipc/convert.rs
@@ -0,0 +1,76 @@
+// 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.
+
+//! Utilities for converting between IPC types and native Arrow types
+
+use crate::datatypes::DataType::*;
+use crate::datatypes::Schema;
+use crate::ipc;
+
+use flatbuffers::FlatBufferBuilder;
+
+/// Serialize a schema in IPC format
+fn schema_to_fb(schema: &Schema) -> FlatBufferBuilder {
+    let mut fbb = FlatBufferBuilder::new();
+
+    let mut fields = vec![];
+    for field in schema.fields() {
+        let fb_field_name = fbb.create_string(field.name().as_str());
+        let mut field_builder = ipc::FieldBuilder::new(&mut fbb);
+        field_builder.add_name(fb_field_name);
+        let ipc_type = match field.data_type() {
+            Boolean => ipc::Type::Bool,
+            UInt8 | UInt16 | UInt32 | UInt64 => ipc::Type::Int,
+            Int8 | Int16 | Int32 | Int64 => ipc::Type::Int,
+            Float32 | Float64 => ipc::Type::FloatingPoint,
+            Utf8 => ipc::Type::Utf8,
+            Date32(_) | Date64(_) => ipc::Type::Date,
+            Time32(_) | Time64(_) => ipc::Type::Time,
+            Timestamp(_) => ipc::Type::Timestamp,
+            _ => ipc::Type::NONE,
+        };
+        field_builder.add_type_type(ipc_type);
+        field_builder.add_nullable(field.is_nullable());
+        fields.push(field_builder.finish());
+    }
+
+    let fb_field_list = fbb.create_vector(&fields);
+
+    let root = {
+        let mut builder = ipc::SchemaBuilder::new(&mut fbb);
+        builder.add_fields(fb_field_list);
+        builder.finish()
+    };
+
+    fbb.finish(root, None);
+
+    fbb
+}
+
+#[cfg(test)]
+mod tests {
+    use super::*;
+    use crate::datatypes::{DataType, Field, Schema};
+
+    #[test]
+    fn convert_schema() {
+        let schema = Schema::new(vec![Field::new("a", DataType::UInt32, false)]);
+
+        let ipc = schema_to_fb(&schema);
+        assert_eq!(60, ipc.finished_data().len());
+    }
+}
diff --git a/rust/arrow/src/ipc/gen/File.rs b/rust/arrow/src/ipc/gen/File.rs
new file mode 100644
index 0000000..f808bd6
--- /dev/null
+++ b/rust/arrow/src/ipc/gen/File.rs
@@ -0,0 +1,282 @@
+// 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.
+
+// automatically generated by the FlatBuffers compiler, do not modify
+
+
+#![allow(dead_code)]
+#![allow(unused_imports)]
+
+
+use crate::ipc::gen::Schema::*;
+
+use std::cmp::Ordering;
+use std::mem;
+
+use flatbuffers::EndianScalar;
+
+// struct Block, aligned to 8
+#[repr(C, align(8))]
+#[derive(Clone, Copy, Debug, PartialEq)]
+pub struct Block {
+    offset_: i64,
+    metaDataLength_: i32,
+    padding0__: u32,
+    bodyLength_: i64,
+} // pub struct Block
+impl flatbuffers::SafeSliceAccess for Block {}
+impl<'a> flatbuffers::Follow<'a> for Block {
+    type Inner = &'a Block;
+    #[inline]
+    fn follow(buf: &'a [u8], loc: usize) -> Self::Inner {
+        <&'a Block>::follow(buf, loc)
+    }
+}
+impl<'a> flatbuffers::Follow<'a> for &'a Block {
+    type Inner = &'a Block;
+    #[inline]
+    fn follow(buf: &'a [u8], loc: usize) -> Self::Inner {
+        flatbuffers::follow_cast_ref::<Block>(buf, loc)
+    }
+}
+impl<'b> flatbuffers::Push for Block {
+    type Output = Block;
+    #[inline]
+    fn push(&self, dst: &mut [u8], _rest: &[u8]) {
+        let src = unsafe {
+            ::std::slice::from_raw_parts(self as *const Block as *const u8, Self::size())
+        };
+        dst.copy_from_slice(src);
+    }
+}
+impl<'b> flatbuffers::Push for &'b Block {
+    type Output = Block;
+
+    #[inline]
+    fn push(&self, dst: &mut [u8], _rest: &[u8]) {
+        let src = unsafe {
+            ::std::slice::from_raw_parts(*self as *const Block as *const u8, Self::size())
+        };
+        dst.copy_from_slice(src);
+    }
+}
+
+impl Block {
+    pub fn new<'a>(_offset: i64, _metaDataLength: i32, _bodyLength: i64) -> Self {
+        Block {
+            offset_: _offset.to_little_endian(),
+            metaDataLength_: _metaDataLength.to_little_endian(),
+            bodyLength_: _bodyLength.to_little_endian(),
+
+            padding0__: 0,
+        }
+    }
+    /// Index to the start of the RecordBlock (note this is past the Message header)
+    pub fn offset<'a>(&'a self) -> i64 {
+        self.offset_.from_little_endian()
+    }
+    /// Length of the metadata
+    pub fn metaDataLength<'a>(&'a self) -> i32 {
+        self.metaDataLength_.from_little_endian()
+    }
+    /// Length of the data (this is aligned so there can be a gap between this and
+    /// the metatdata).
+    pub fn bodyLength<'a>(&'a self) -> i64 {
+        self.bodyLength_.from_little_endian()
+    }
+}
+
+/// ----------------------------------------------------------------------
+/// Arrow File metadata
+///
+pub enum FooterOffset {}
+#[derive(Copy, Clone, Debug, PartialEq)]
+
+pub struct Footer<'a> {
+    pub _tab: flatbuffers::Table<'a>,
+}
+
+impl<'a> flatbuffers::Follow<'a> for Footer<'a> {
+    type Inner = Footer<'a>;
+    #[inline]
+    fn follow(buf: &'a [u8], loc: usize) -> Self::Inner {
+        Self {
+            _tab: flatbuffers::Table { buf: buf, loc: loc },
+        }
+    }
+}
+
+impl<'a> Footer<'a> {
+    #[inline]
+    pub fn init_from_table(table: flatbuffers::Table<'a>) -> Self {
+        Footer { _tab: table }
+    }
+    #[allow(unused_mut)]
+    pub fn create<'bldr: 'args, 'args: 'mut_bldr, 'mut_bldr>(
+        _fbb: &'mut_bldr mut flatbuffers::FlatBufferBuilder<'bldr>,
+        args: &'args FooterArgs<'args>,
+    ) -> flatbuffers::WIPOffset<Footer<'bldr>> {
+        let mut builder = FooterBuilder::new(_fbb);
+        if let Some(x) = args.recordBatches {
+            builder.add_recordBatches(x);
+        }
+        if let Some(x) = args.dictionaries {
+            builder.add_dictionaries(x);
+        }
+        if let Some(x) = args.schema {
+            builder.add_schema(x);
+        }
+        builder.add_version(args.version);
+        builder.finish()
+    }
+
+    pub const VT_VERSION: flatbuffers::VOffsetT = 4;
+    pub const VT_SCHEMA: flatbuffers::VOffsetT = 6;
+    pub const VT_DICTIONARIES: flatbuffers::VOffsetT = 8;
+    pub const VT_RECORDBATCHES: flatbuffers::VOffsetT = 10;
+
+    #[inline]
+    pub fn version(&self) -> MetadataVersion {
+        self._tab
+            .get::<MetadataVersion>(Footer::VT_VERSION, Some(MetadataVersion::V1))
+            .unwrap()
+    }
+    #[inline]
+    pub fn schema(&self) -> Option<Schema<'a>> {
+        self._tab
+            .get::<flatbuffers::ForwardsUOffset<Schema<'a>>>(Footer::VT_SCHEMA, None)
+    }
+    #[inline]
+    pub fn dictionaries(&self) -> Option<&'a [Block]> {
+        self._tab
+            .get::<flatbuffers::ForwardsUOffset<flatbuffers::Vector<Block>>>(
+                Footer::VT_DICTIONARIES,
+                None,
+            )
+            .map(|v| v.safe_slice())
+    }
+    #[inline]
+    pub fn recordBatches(&self) -> Option<&'a [Block]> {
+        self._tab
+            .get::<flatbuffers::ForwardsUOffset<flatbuffers::Vector<Block>>>(
+                Footer::VT_RECORDBATCHES,
+                None,
+            )
+            .map(|v| v.safe_slice())
+    }
+}
+
+pub struct FooterArgs<'a> {
+    pub version: MetadataVersion,
+    pub schema: Option<flatbuffers::WIPOffset<Schema<'a>>>,
+    pub dictionaries: Option<flatbuffers::WIPOffset<flatbuffers::Vector<'a, Block>>>,
+    pub recordBatches: Option<flatbuffers::WIPOffset<flatbuffers::Vector<'a, Block>>>,
+}
+impl<'a> Default for FooterArgs<'a> {
+    #[inline]
+    fn default() -> Self {
+        FooterArgs {
+            version: MetadataVersion::V1,
+            schema: None,
+            dictionaries: None,
+            recordBatches: None,
+        }
+    }
+}
+pub struct FooterBuilder<'a: 'b, 'b> {
+    fbb_: &'b mut flatbuffers::FlatBufferBuilder<'a>,
+    start_: flatbuffers::WIPOffset<flatbuffers::TableUnfinishedWIPOffset>,
+}
+impl<'a: 'b, 'b> FooterBuilder<'a, 'b> {
+    #[inline]
+    pub fn add_version(&mut self, version: MetadataVersion) {
+        self.fbb_.push_slot::<MetadataVersion>(
+            Footer::VT_VERSION,
+            version,
+            MetadataVersion::V1,
+        );
+    }
+    #[inline]
+    pub fn add_schema(&mut self, schema: flatbuffers::WIPOffset<Schema<'b>>) {
+        self.fbb_
+            .push_slot_always::<flatbuffers::WIPOffset<Schema>>(
+                Footer::VT_SCHEMA,
+                schema,
+            );
+    }
+    #[inline]
+    pub fn add_dictionaries(
+        &mut self,
+        dictionaries: flatbuffers::WIPOffset<flatbuffers::Vector<'b, Block>>,
+    ) {
+        self.fbb_.push_slot_always::<flatbuffers::WIPOffset<_>>(
+            Footer::VT_DICTIONARIES,
+            dictionaries,
+        );
+    }
+    #[inline]
+    pub fn add_recordBatches(
+        &mut self,
+        recordBatches: flatbuffers::WIPOffset<flatbuffers::Vector<'b, Block>>,
+    ) {
+        self.fbb_.push_slot_always::<flatbuffers::WIPOffset<_>>(
+            Footer::VT_RECORDBATCHES,
+            recordBatches,
+        );
+    }
+    #[inline]
+    pub fn new(
+        _fbb: &'b mut flatbuffers::FlatBufferBuilder<'a>,
+    ) -> FooterBuilder<'a, 'b> {
+        let start = _fbb.start_table();
+        FooterBuilder {
+            fbb_: _fbb,
+            start_: start,
+        }
+    }
+    #[inline]
+    pub fn finish(self) -> flatbuffers::WIPOffset<Footer<'a>> {
+        let o = self.fbb_.end_table(self.start_);
+        flatbuffers::WIPOffset::new(o.value())
+    }
+}
+
+#[inline]
+pub fn get_root_as_footer<'a>(buf: &'a [u8]) -> Footer<'a> {
+    flatbuffers::get_root::<Footer<'a>>(buf)
+}
+
+#[inline]
+pub fn get_size_prefixed_root_as_footer<'a>(buf: &'a [u8]) -> Footer<'a> {
+    flatbuffers::get_size_prefixed_root::<Footer<'a>>(buf)
+}
+
+#[inline]
+pub fn finish_footer_buffer<'a, 'b>(
+    fbb: &'b mut flatbuffers::FlatBufferBuilder<'a>,
+    root: flatbuffers::WIPOffset<Footer<'a>>,
+) {
+    fbb.finish(root, None);
+}
+
+#[inline]
+pub fn finish_size_prefixed_footer_buffer<'a, 'b>(
+    fbb: &'b mut flatbuffers::FlatBufferBuilder<'a>,
+    root: flatbuffers::WIPOffset<Footer<'a>>,
+) {
+    fbb.finish_size_prefixed(root, None);
+}
diff --git a/rust/arrow/src/ipc/gen/Message.rs b/rust/arrow/src/ipc/gen/Message.rs
new file mode 100644
index 0000000..5a753f3
--- /dev/null
+++ b/rust/arrow/src/ipc/gen/Message.rs
@@ -0,0 +1,698 @@
+// 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.
+
+// automatically generated by the FlatBuffers compiler, do not modify
+
+
+#![allow(dead_code)]
+#![allow(unused_imports)]
+
+
+use crate::ipc::gen::Schema::*;
+use crate::ipc::gen::SparseTensor::*;
+use crate::ipc::gen::Tensor::*;
+
+use std::cmp::Ordering;
+use std::mem;
+
+use flatbuffers::EndianScalar;
+
+/// ----------------------------------------------------------------------
+/// 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
+#[allow(non_camel_case_types)]
+#[repr(u8)]
+#[derive(Clone, Copy, PartialEq, Debug)]
+pub enum MessageHeader {
+    NONE = 0,
+    Schema = 1,
+    DictionaryBatch = 2,
+    RecordBatch = 3,
+    Tensor = 4,
+    SparseTensor = 5,
+}
+
+const ENUM_MIN_MESSAGE_HEADER: u8 = 0;
+const ENUM_MAX_MESSAGE_HEADER: u8 = 5;
+
+impl<'a> flatbuffers::Follow<'a> for MessageHeader {
+    type Inner = Self;
+    #[inline]
+    fn follow(buf: &'a [u8], loc: usize) -> Self::Inner {
+        flatbuffers::read_scalar_at::<Self>(buf, loc)
+    }
+}
+
+impl flatbuffers::EndianScalar for MessageHeader {
+    #[inline]
+    fn to_little_endian(self) -> Self {
+        let n = u8::to_le(self as u8);
+        let p = &n as *const u8 as *const MessageHeader;
+        unsafe { *p }
+    }
+    #[inline]
+    fn from_little_endian(self) -> Self {
+        let n = u8::from_le(self as u8);
+        let p = &n as *const u8 as *const MessageHeader;
+        unsafe { *p }
+    }
+}
+
+impl flatbuffers::Push for MessageHeader {
+    type Output = MessageHeader;
+    #[inline]
+    fn push(&self, dst: &mut [u8], _rest: &[u8]) {
+        flatbuffers::emplace_scalar::<MessageHeader>(dst, *self);
+    }
+}
+
+#[allow(non_camel_case_types)]
+const ENUM_VALUES_MESSAGE_HEADER: [MessageHeader; 6] = [
+    MessageHeader::NONE,
+    MessageHeader::Schema,
+    MessageHeader::DictionaryBatch,
+    MessageHeader::RecordBatch,
+    MessageHeader::Tensor,
+    MessageHeader::SparseTensor,
+];
+
+#[allow(non_camel_case_types)]
+const ENUM_NAMES_MESSAGE_HEADER: [&'static str; 6] = [
+    "NONE",
+    "Schema",
+    "DictionaryBatch",
+    "RecordBatch",
+    "Tensor",
+    "SparseTensor",
+];
+
+pub fn enum_name_message_header(e: MessageHeader) -> &'static str {
+    let index: usize = e as usize;
+    ENUM_NAMES_MESSAGE_HEADER[index]
+}
+
+pub struct MessageHeaderUnionTableOffset {}
+/// ----------------------------------------------------------------------
+/// 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, aligned to 8
+#[repr(C, align(8))]
+#[derive(Clone, Copy, Debug, PartialEq)]
+pub struct FieldNode {
+    length_: i64,
+    null_count_: i64,
+} // pub struct FieldNode
+impl flatbuffers::SafeSliceAccess for FieldNode {}
+impl<'a> flatbuffers::Follow<'a> for FieldNode {
+    type Inner = &'a FieldNode;
+    #[inline]
+    fn follow(buf: &'a [u8], loc: usize) -> Self::Inner {
+        <&'a FieldNode>::follow(buf, loc)
+    }
+}
+impl<'a> flatbuffers::Follow<'a> for &'a FieldNode {
+    type Inner = &'a FieldNode;
+    #[inline]
+    fn follow(buf: &'a [u8], loc: usize) -> Self::Inner {
+        flatbuffers::follow_cast_ref::<FieldNode>(buf, loc)
+    }
+}
+impl<'b> flatbuffers::Push for FieldNode {
+    type Output = FieldNode;
+    #[inline]
+    fn push(&self, dst: &mut [u8], _rest: &[u8]) {
+        let src = unsafe {
+            ::std::slice::from_raw_parts(
+                self as *const FieldNode as *const u8,
+                Self::size(),
+            )
+        };
+        dst.copy_from_slice(src);
+    }
+}
+impl<'b> flatbuffers::Push for &'b FieldNode {
+    type Output = FieldNode;
+
+    #[inline]
+    fn push(&self, dst: &mut [u8], _rest: &[u8]) {
+        let src = unsafe {
+            ::std::slice::from_raw_parts(
+                *self as *const FieldNode as *const u8,
+                Self::size(),
+            )
+        };
+        dst.copy_from_slice(src);
+    }
+}
+
+impl FieldNode {
+    pub fn new<'a>(_length: i64, _null_count: i64) -> Self {
+        FieldNode {
+            length_: _length.to_little_endian(),
+            null_count_: _null_count.to_little_endian(),
+        }
+    }
+    /// The number of value slots in the Arrow array at this level of a nested
+    /// tree
+    pub fn length<'a>(&'a self) -> i64 {
+        self.length_.from_little_endian()
+    }
+    /// 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.
+    pub fn null_count<'a>(&'a self) -> i64 {
+        self.null_count_.from_little_endian()
+    }
+}
+
+/// 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".
+pub enum RecordBatchOffset {}
+#[derive(Copy, Clone, Debug, PartialEq)]
+
+pub struct RecordBatch<'a> {
+    pub _tab: flatbuffers::Table<'a>,
+}
+
+impl<'a> flatbuffers::Follow<'a> for RecordBatch<'a> {
+    type Inner = RecordBatch<'a>;
+    #[inline]
+    fn follow(buf: &'a [u8], loc: usize) -> Self::Inner {
+        Self {
+            _tab: flatbuffers::Table { buf: buf, loc: loc },
+        }
+    }
+}
+
+impl<'a> RecordBatch<'a> {
+    #[inline]
+    pub fn init_from_table(table: flatbuffers::Table<'a>) -> Self {
+        RecordBatch { _tab: table }
+    }
+    #[allow(unused_mut)]
+    pub fn create<'bldr: 'args, 'args: 'mut_bldr, 'mut_bldr>(
+        _fbb: &'mut_bldr mut flatbuffers::FlatBufferBuilder<'bldr>,
+        args: &'args RecordBatchArgs<'args>,
+    ) -> flatbuffers::WIPOffset<RecordBatch<'bldr>> {
+        let mut builder = RecordBatchBuilder::new(_fbb);
+        builder.add_length(args.length);
+        if let Some(x) = args.buffers {
+            builder.add_buffers(x);
+        }
+        if let Some(x) = args.nodes {
+            builder.add_nodes(x);
+        }
+        builder.finish()
+    }
+
+    pub const VT_LENGTH: flatbuffers::VOffsetT = 4;
+    pub const VT_NODES: flatbuffers::VOffsetT = 6;
+    pub const VT_BUFFERS: flatbuffers::VOffsetT = 8;
+
+    /// number of records / rows. The arrays in the batch should all have this
+    /// length
+    #[inline]
+    pub fn length(&self) -> i64 {
+        self._tab
+            .get::<i64>(RecordBatch::VT_LENGTH, Some(0))
+            .unwrap()
+    }
+    /// Nodes correspond to the pre-ordered flattened logical schema
+    #[inline]
+    pub fn nodes(&self) -> Option<&'a [FieldNode]> {
+        self._tab
+            .get::<flatbuffers::ForwardsUOffset<flatbuffers::Vector<FieldNode>>>(
+                RecordBatch::VT_NODES,
+                None,
+            )
+            .map(|v| v.safe_slice())
+    }
+    /// 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
+    #[inline]
+    pub fn buffers(&self) -> Option<&'a [Buffer]> {
+        self._tab
+            .get::<flatbuffers::ForwardsUOffset<flatbuffers::Vector<Buffer>>>(
+                RecordBatch::VT_BUFFERS,
+                None,
+            )
+            .map(|v| v.safe_slice())
+    }
+}
+
+pub struct RecordBatchArgs<'a> {
+    pub length: i64,
+    pub nodes: Option<flatbuffers::WIPOffset<flatbuffers::Vector<'a, FieldNode>>>,
+    pub buffers: Option<flatbuffers::WIPOffset<flatbuffers::Vector<'a, Buffer>>>,
+}
+impl<'a> Default for RecordBatchArgs<'a> {
+    #[inline]
+    fn default() -> Self {
+        RecordBatchArgs {
+            length: 0,
+            nodes: None,
+            buffers: None,
+        }
+    }
+}
+pub struct RecordBatchBuilder<'a: 'b, 'b> {
+    fbb_: &'b mut flatbuffers::FlatBufferBuilder<'a>,
+    start_: flatbuffers::WIPOffset<flatbuffers::TableUnfinishedWIPOffset>,
+}
+impl<'a: 'b, 'b> RecordBatchBuilder<'a, 'b> {
+    #[inline]
+    pub fn add_length(&mut self, length: i64) {
+        self.fbb_
+            .push_slot::<i64>(RecordBatch::VT_LENGTH, length, 0);
+    }
+    #[inline]
+    pub fn add_nodes(
+        &mut self,
+        nodes: flatbuffers::WIPOffset<flatbuffers::Vector<'b, FieldNode>>,
+    ) {
+        self.fbb_
+            .push_slot_always::<flatbuffers::WIPOffset<_>>(RecordBatch::VT_NODES, nodes);
+    }
+    #[inline]
+    pub fn add_buffers(
+        &mut self,
+        buffers: flatbuffers::WIPOffset<flatbuffers::Vector<'b, Buffer>>,
+    ) {
+        self.fbb_.push_slot_always::<flatbuffers::WIPOffset<_>>(
+            RecordBatch::VT_BUFFERS,
+            buffers,
+        );
+    }
+    #[inline]
+    pub fn new(
+        _fbb: &'b mut flatbuffers::FlatBufferBuilder<'a>,
+    ) -> RecordBatchBuilder<'a, 'b> {
+        let start = _fbb.start_table();
+        RecordBatchBuilder {
+            fbb_: _fbb,
+            start_: start,
+        }
+    }
+    #[inline]
+    pub fn finish(self) -> flatbuffers::WIPOffset<RecordBatch<'a>> {
+        let o = self.fbb_.end_table(self.start_);
+        flatbuffers::WIPOffset::new(o.value())
+    }
+}
+
+/// 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
+pub enum DictionaryBatchOffset {}
+#[derive(Copy, Clone, Debug, PartialEq)]
+
+pub struct DictionaryBatch<'a> {
+    pub _tab: flatbuffers::Table<'a>,
+}
+
+impl<'a> flatbuffers::Follow<'a> for DictionaryBatch<'a> {
+    type Inner = DictionaryBatch<'a>;
+    #[inline]
+    fn follow(buf: &'a [u8], loc: usize) -> Self::Inner {
+        Self {
+            _tab: flatbuffers::Table { buf: buf, loc: loc },
+        }
+    }
+}
+
+impl<'a> DictionaryBatch<'a> {
+    #[inline]
+    pub fn init_from_table(table: flatbuffers::Table<'a>) -> Self {
+        DictionaryBatch { _tab: table }
+    }
+    #[allow(unused_mut)]
+    pub fn create<'bldr: 'args, 'args: 'mut_bldr, 'mut_bldr>(
+        _fbb: &'mut_bldr mut flatbuffers::FlatBufferBuilder<'bldr>,
+        args: &'args DictionaryBatchArgs<'args>,
+    ) -> flatbuffers::WIPOffset<DictionaryBatch<'bldr>> {
+        let mut builder = DictionaryBatchBuilder::new(_fbb);
+        builder.add_id(args.id);
+        if let Some(x) = args.data {
+            builder.add_data(x);
+        }
+        builder.add_isDelta(args.isDelta);
+        builder.finish()
+    }
+
+    pub const VT_ID: flatbuffers::VOffsetT = 4;
+    pub const VT_DATA: flatbuffers::VOffsetT = 6;
+    pub const VT_ISDELTA: flatbuffers::VOffsetT = 8;
+
+    #[inline]
+    pub fn id(&self) -> i64 {
+        self._tab
+            .get::<i64>(DictionaryBatch::VT_ID, Some(0))
+            .unwrap()
+    }
+    #[inline]
+    pub fn data(&self) -> Option<RecordBatch<'a>> {
+        self._tab
+            .get::<flatbuffers::ForwardsUOffset<RecordBatch<'a>>>(
+                DictionaryBatch::VT_DATA,
+                None,
+            )
+    }
+    /// If isDelta is true the values in the dictionary are to be appended to a
+    /// dictionary with the indicated id
+    #[inline]
+    pub fn isDelta(&self) -> bool {
+        self._tab
+            .get::<bool>(DictionaryBatch::VT_ISDELTA, Some(false))
+            .unwrap()
+    }
+}
+
+pub struct DictionaryBatchArgs<'a> {
+    pub id: i64,
+    pub data: Option<flatbuffers::WIPOffset<RecordBatch<'a>>>,
+    pub isDelta: bool,
+}
+impl<'a> Default for DictionaryBatchArgs<'a> {
+    #[inline]
+    fn default() -> Self {
+        DictionaryBatchArgs {
+            id: 0,
+            data: None,
+            isDelta: false,
+        }
+    }
+}
+pub struct DictionaryBatchBuilder<'a: 'b, 'b> {
+    fbb_: &'b mut flatbuffers::FlatBufferBuilder<'a>,
+    start_: flatbuffers::WIPOffset<flatbuffers::TableUnfinishedWIPOffset>,
+}
+impl<'a: 'b, 'b> DictionaryBatchBuilder<'a, 'b> {
+    #[inline]
+    pub fn add_id(&mut self, id: i64) {
+        self.fbb_.push_slot::<i64>(DictionaryBatch::VT_ID, id, 0);
+    }
+    #[inline]
+    pub fn add_data(&mut self, data: flatbuffers::WIPOffset<RecordBatch<'b>>) {
+        self.fbb_
+            .push_slot_always::<flatbuffers::WIPOffset<RecordBatch>>(
+                DictionaryBatch::VT_DATA,
+                data,
+            );
+    }
+    #[inline]
+    pub fn add_isDelta(&mut self, isDelta: bool) {
+        self.fbb_
+            .push_slot::<bool>(DictionaryBatch::VT_ISDELTA, isDelta, false);
+    }
+    #[inline]
+    pub fn new(
+        _fbb: &'b mut flatbuffers::FlatBufferBuilder<'a>,
+    ) -> DictionaryBatchBuilder<'a, 'b> {
+        let start = _fbb.start_table();
+        DictionaryBatchBuilder {
+            fbb_: _fbb,
+            start_: start,
+        }
+    }
+    #[inline]
+    pub fn finish(self) -> flatbuffers::WIPOffset<DictionaryBatch<'a>> {
+        let o = self.fbb_.end_table(self.start_);
+        flatbuffers::WIPOffset::new(o.value())
+    }
+}
+
+pub enum MessageOffset {}
+#[derive(Copy, Clone, Debug, PartialEq)]
+
+pub struct Message<'a> {
+    pub _tab: flatbuffers::Table<'a>,
+}
+
+impl<'a> flatbuffers::Follow<'a> for Message<'a> {
+    type Inner = Message<'a>;
+    #[inline]
+    fn follow(buf: &'a [u8], loc: usize) -> Self::Inner {
+        Self {
+            _tab: flatbuffers::Table { buf: buf, loc: loc },
+        }
+    }
+}
+
+impl<'a> Message<'a> {
+    #[inline]
+    pub fn init_from_table(table: flatbuffers::Table<'a>) -> Self {
+        Message { _tab: table }
+    }
+    #[allow(unused_mut)]
+    pub fn create<'bldr: 'args, 'args: 'mut_bldr, 'mut_bldr>(
+        _fbb: &'mut_bldr mut flatbuffers::FlatBufferBuilder<'bldr>,
+        args: &'args MessageArgs<'args>,
+    ) -> flatbuffers::WIPOffset<Message<'bldr>> {
+        let mut builder = MessageBuilder::new(_fbb);
+        builder.add_bodyLength(args.bodyLength);
+        if let Some(x) = args.custom_metadata {
+            builder.add_custom_metadata(x);
+        }
+        if let Some(x) = args.header {
+            builder.add_header(x);
+        }
+        builder.add_version(args.version);
+        builder.add_header_type(args.header_type);
+        builder.finish()
+    }
+
+    pub const VT_VERSION: flatbuffers::VOffsetT = 4;
+    pub const VT_HEADER_TYPE: flatbuffers::VOffsetT = 6;
+    pub const VT_HEADER: flatbuffers::VOffsetT = 8;
+    pub const VT_BODYLENGTH: flatbuffers::VOffsetT = 10;
+    pub const VT_CUSTOM_METADATA: flatbuffers::VOffsetT = 12;
+
+    #[inline]
+    pub fn version(&self) -> MetadataVersion {
+        self._tab
+            .get::<MetadataVersion>(Message::VT_VERSION, Some(MetadataVersion::V1))
+            .unwrap()
+    }
+    #[inline]
+    pub fn header_type(&self) -> MessageHeader {
+        self._tab
+            .get::<MessageHeader>(Message::VT_HEADER_TYPE, Some(MessageHeader::NONE))
+            .unwrap()
+    }
+    #[inline]
+    pub fn header(&self) -> Option<flatbuffers::Table<'a>> {
+        self._tab
+            .get::<flatbuffers::ForwardsUOffset<flatbuffers::Table<'a>>>(
+                Message::VT_HEADER,
+                None,
+            )
+    }
+    #[inline]
+    pub fn bodyLength(&self) -> i64 {
+        self._tab
+            .get::<i64>(Message::VT_BODYLENGTH, Some(0))
+            .unwrap()
+    }
+    #[inline]
+    pub fn custom_metadata(
+        &self,
+    ) -> Option<flatbuffers::Vector<flatbuffers::ForwardsUOffset<KeyValue<'a>>>> {
+        self._tab.get::<flatbuffers::ForwardsUOffset<
+            flatbuffers::Vector<flatbuffers::ForwardsUOffset<KeyValue<'a>>>,
+        >>(Message::VT_CUSTOM_METADATA, None)
+    }
+    #[inline]
+    #[allow(non_snake_case)]
+    pub fn header_as_schema(&'a self) -> Option<Schema> {
+        if self.header_type() == MessageHeader::Schema {
+            self.header().map(|u| Schema::init_from_table(u))
+        } else {
+            None
+        }
+    }
+
+    #[inline]
+    #[allow(non_snake_case)]
+    pub fn header_as_dictionary_batch(&'a self) -> Option<DictionaryBatch> {
+        if self.header_type() == MessageHeader::DictionaryBatch {
+            self.header().map(|u| DictionaryBatch::init_from_table(u))
+        } else {
+            None
+        }
+    }
+
+    #[inline]
+    #[allow(non_snake_case)]
+    pub fn header_as_record_batch(&'a self) -> Option<RecordBatch> {
+        if self.header_type() == MessageHeader::RecordBatch {
+            self.header().map(|u| RecordBatch::init_from_table(u))
+        } else {
+            None
+        }
+    }
+
+    #[inline]
+    #[allow(non_snake_case)]
+    pub fn header_as_tensor(&'a self) -> Option<Tensor> {
+        if self.header_type() == MessageHeader::Tensor {
+            self.header().map(|u| Tensor::init_from_table(u))
+        } else {
+            None
+        }
+    }
+
+    #[inline]
+    #[allow(non_snake_case)]
+    pub fn header_as_sparse_tensor(&'a self) -> Option<SparseTensor> {
+        if self.header_type() == MessageHeader::SparseTensor {
+            self.header().map(|u| SparseTensor::init_from_table(u))
+        } else {
+            None
+        }
+    }
+}
+
+pub struct MessageArgs<'a> {
+    pub version: MetadataVersion,
+    pub header_type: MessageHeader,
+    pub header: Option<flatbuffers::WIPOffset<flatbuffers::UnionWIPOffset>>,
+    pub bodyLength: i64,
+    pub custom_metadata: Option<
+        flatbuffers::WIPOffset<
+            flatbuffers::Vector<'a, flatbuffers::ForwardsUOffset<KeyValue<'a>>>,
+        >,
+    >,
+}
+impl<'a> Default for MessageArgs<'a> {
+    #[inline]
+    fn default() -> Self {
+        MessageArgs {
+            version: MetadataVersion::V1,
+            header_type: MessageHeader::NONE,
+            header: None,
+            bodyLength: 0,
+            custom_metadata: None,
+        }
+    }
+}
+pub struct MessageBuilder<'a: 'b, 'b> {
+    fbb_: &'b mut flatbuffers::FlatBufferBuilder<'a>,
+    start_: flatbuffers::WIPOffset<flatbuffers::TableUnfinishedWIPOffset>,
+}
+impl<'a: 'b, 'b> MessageBuilder<'a, 'b> {
+    #[inline]
+    pub fn add_version(&mut self, version: MetadataVersion) {
+        self.fbb_.push_slot::<MetadataVersion>(
+            Message::VT_VERSION,
+            version,
+            MetadataVersion::V1,
+        );
+    }
+    #[inline]
+    pub fn add_header_type(&mut self, header_type: MessageHeader) {
+        self.fbb_.push_slot::<MessageHeader>(
+            Message::VT_HEADER_TYPE,
+            header_type,
+            MessageHeader::NONE,
+        );
+    }
+    #[inline]
+    pub fn add_header(
+        &mut self,
+        header: flatbuffers::WIPOffset<flatbuffers::UnionWIPOffset>,
+    ) {
+        self.fbb_
+            .push_slot_always::<flatbuffers::WIPOffset<_>>(Message::VT_HEADER, header);
+    }
+    #[inline]
+    pub fn add_bodyLength(&mut self, bodyLength: i64) {
+        self.fbb_
+            .push_slot::<i64>(Message::VT_BODYLENGTH, bodyLength, 0);
+    }
+    #[inline]
+    pub fn add_custom_metadata(
+        &mut self,
+        custom_metadata: flatbuffers::WIPOffset<
+            flatbuffers::Vector<'b, flatbuffers::ForwardsUOffset<KeyValue<'b>>>,
+        >,
+    ) {
+        self.fbb_.push_slot_always::<flatbuffers::WIPOffset<_>>(
+            Message::VT_CUSTOM_METADATA,
+            custom_metadata,
+        );
+    }
+    #[inline]
+    pub fn new(
+        _fbb: &'b mut flatbuffers::FlatBufferBuilder<'a>,
+    ) -> MessageBuilder<'a, 'b> {
+        let start = _fbb.start_table();
+        MessageBuilder {
+            fbb_: _fbb,
+            start_: start,
+        }
+    }
+    #[inline]
+    pub fn finish(self) -> flatbuffers::WIPOffset<Message<'a>> {
+        let o = self.fbb_.end_table(self.start_);
+        flatbuffers::WIPOffset::new(o.value())
+    }
+}
+
+#[inline]
+pub fn get_root_as_message<'a>(buf: &'a [u8]) -> Message<'a> {
+    flatbuffers::get_root::<Message<'a>>(buf)
+}
+
+#[inline]
+pub fn get_size_prefixed_root_as_message<'a>(buf: &'a [u8]) -> Message<'a> {
+    flatbuffers::get_size_prefixed_root::<Message<'a>>(buf)
+}
+
+#[inline]
+pub fn finish_message_buffer<'a, 'b>(
+    fbb: &'b mut flatbuffers::FlatBufferBuilder<'a>,
+    root: flatbuffers::WIPOffset<Message<'a>>,
+) {
+    fbb.finish(root, None);
+}
+
+#[inline]
+pub fn finish_size_prefixed_message_buffer<'a, 'b>(
+    fbb: &'b mut flatbuffers::FlatBufferBuilder<'a>,
+    root: flatbuffers::WIPOffset<Message<'a>>,
+) {
+    fbb.finish_size_prefixed(root, None);
+}
diff --git a/rust/arrow/src/ipc/gen/Schema.rs b/rust/arrow/src/ipc/gen/Schema.rs
new file mode 100644
index 0000000..6cab35d
--- /dev/null
+++ b/rust/arrow/src/ipc/gen/Schema.rs
@@ -0,0 +1,2766 @@
+// 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.
+
+// automatically generated by the FlatBuffers compiler, do not modify
+
+
+#![allow(dead_code)]
+#![allow(unused_imports)]
+
+
+use std::cmp::Ordering;
+use std::mem;
+
+use flatbuffers::EndianScalar;
+
+#[allow(non_camel_case_types)]
+#[repr(i16)]
+#[derive(Clone, Copy, PartialEq, Debug)]
+pub enum MetadataVersion {
+    /// 0.1.0
+    V1 = 0,
+    /// 0.2.0
+    V2 = 1,
+    /// 0.3.0 -> 0.7.1
+    V3 = 2,
+    /// >= 0.8.0
+    V4 = 3,
+}
+
+const ENUM_MIN_METADATA_VERSION: i16 = 0;
+const ENUM_MAX_METADATA_VERSION: i16 = 3;
+
+impl<'a> flatbuffers::Follow<'a> for MetadataVersion {
+    type Inner = Self;
+    #[inline]
+    fn follow(buf: &'a [u8], loc: usize) -> Self::Inner {
+        flatbuffers::read_scalar_at::<Self>(buf, loc)
+    }
+}
+
+impl flatbuffers::EndianScalar for MetadataVersion {
+    #[inline]
+    fn to_little_endian(self) -> Self {
+        let n = i16::to_le(self as i16);
+        let p = &n as *const i16 as *const MetadataVersion;
+        unsafe { *p }
+    }
+    #[inline]
+    fn from_little_endian(self) -> Self {
+        let n = i16::from_le(self as i16);
+        let p = &n as *const i16 as *const MetadataVersion;
+        unsafe { *p }
+    }
+}
+
+impl flatbuffers::Push for MetadataVersion {
+    type Output = MetadataVersion;
+    #[inline]
+    fn push(&self, dst: &mut [u8], _rest: &[u8]) {
+        flatbuffers::emplace_scalar::<MetadataVersion>(dst, *self);
+    }
+}
+
+#[allow(non_camel_case_types)]
+const ENUM_VALUES_METADATA_VERSION: [MetadataVersion; 4] = [
+    MetadataVersion::V1,
+    MetadataVersion::V2,
+    MetadataVersion::V3,
+    MetadataVersion::V4,
+];
+
+#[allow(non_camel_case_types)]
+const ENUM_NAMES_METADATA_VERSION: [&'static str; 4] = ["V1", "V2", "V3", "V4"];
+
+pub fn enum_name_metadata_version(e: MetadataVersion) -> &'static str {
+    let index: usize = e as usize;
+    ENUM_NAMES_METADATA_VERSION[index]
+}
+
+#[allow(non_camel_case_types)]
+#[repr(i16)]
+#[derive(Clone, Copy, PartialEq, Debug)]
+pub enum UnionMode {
+    Sparse = 0,
+    Dense = 1,
+}
+
+const ENUM_MIN_UNION_MODE: i16 = 0;
+const ENUM_MAX_UNION_MODE: i16 = 1;
+
+impl<'a> flatbuffers::Follow<'a> for UnionMode {
+    type Inner = Self;
+    #[inline]
+    fn follow(buf: &'a [u8], loc: usize) -> Self::Inner {
+        flatbuffers::read_scalar_at::<Self>(buf, loc)
+    }
+}
+
+impl flatbuffers::EndianScalar for UnionMode {
+    #[inline]
+    fn to_little_endian(self) -> Self {
+        let n = i16::to_le(self as i16);
+        let p = &n as *const i16 as *const UnionMode;
+        unsafe { *p }
+    }
+    #[inline]
+    fn from_little_endian(self) -> Self {
+        let n = i16::from_le(self as i16);
+        let p = &n as *const i16 as *const UnionMode;
+        unsafe { *p }
+    }
+}
+
+impl flatbuffers::Push for UnionMode {
+    type Output = UnionMode;
+    #[inline]
+    fn push(&self, dst: &mut [u8], _rest: &[u8]) {
+        flatbuffers::emplace_scalar::<UnionMode>(dst, *self);
+    }
+}
+
+#[allow(non_camel_case_types)]
+const ENUM_VALUES_UNION_MODE: [UnionMode; 2] = [UnionMode::Sparse, UnionMode::Dense];
+
+#[allow(non_camel_case_types)]
+const ENUM_NAMES_UNION_MODE: [&'static str; 2] = ["Sparse", "Dense"];
+
+pub fn enum_name_union_mode(e: UnionMode) -> &'static str {
+    let index: usize = e as usize;
+    ENUM_NAMES_UNION_MODE[index]
+}
+
+#[allow(non_camel_case_types)]
+#[repr(i16)]
+#[derive(Clone, Copy, PartialEq, Debug)]
+pub enum Precision {
+    HALF = 0,
+    SINGLE = 1,
+    DOUBLE = 2,
+}
+
+const ENUM_MIN_PRECISION: i16 = 0;
+const ENUM_MAX_PRECISION: i16 = 2;
+
+impl<'a> flatbuffers::Follow<'a> for Precision {
+    type Inner = Self;
+    #[inline]
+    fn follow(buf: &'a [u8], loc: usize) -> Self::Inner {
+        flatbuffers::read_scalar_at::<Self>(buf, loc)
+    }
+}
+
+impl flatbuffers::EndianScalar for Precision {
+    #[inline]
+    fn to_little_endian(self) -> Self {
+        let n = i16::to_le(self as i16);
+        let p = &n as *const i16 as *const Precision;
+        unsafe { *p }
+    }
+    #[inline]
+    fn from_little_endian(self) -> Self {
+        let n = i16::from_le(self as i16);
+        let p = &n as *const i16 as *const Precision;
+        unsafe { *p }
+    }
+}
+
+impl flatbuffers::Push for Precision {
+    type Output = Precision;
+    #[inline]
+    fn push(&self, dst: &mut [u8], _rest: &[u8]) {
+        flatbuffers::emplace_scalar::<Precision>(dst, *self);
+    }
+}
+
+#[allow(non_camel_case_types)]
+const ENUM_VALUES_PRECISION: [Precision; 3] =
+    [Precision::HALF, Precision::SINGLE, Precision::DOUBLE];
+
+#[allow(non_camel_case_types)]
+const ENUM_NAMES_PRECISION: [&'static str; 3] = ["HALF", "SINGLE", "DOUBLE"];
+
+pub fn enum_name_precision(e: Precision) -> &'static str {
+    let index: usize = e as usize;
+    ENUM_NAMES_PRECISION[index]
+}
+
+#[allow(non_camel_case_types)]
+#[repr(i16)]
+#[derive(Clone, Copy, PartialEq, Debug)]
+pub enum DateUnit {
+    DAY = 0,
+    MILLISECOND = 1,
+}
+
+const ENUM_MIN_DATE_UNIT: i16 = 0;
+const ENUM_MAX_DATE_UNIT: i16 = 1;
+
+impl<'a> flatbuffers::Follow<'a> for DateUnit {
+    type Inner = Self;
+    #[inline]
+    fn follow(buf: &'a [u8], loc: usize) -> Self::Inner {
+        flatbuffers::read_scalar_at::<Self>(buf, loc)
+    }
+}
+
+impl flatbuffers::EndianScalar for DateUnit {
+    #[inline]
+    fn to_little_endian(self) -> Self {
+        let n = i16::to_le(self as i16);
+        let p = &n as *const i16 as *const DateUnit;
+        unsafe { *p }
+    }
+    #[inline]
+    fn from_little_endian(self) -> Self {
+        let n = i16::from_le(self as i16);
+        let p = &n as *const i16 as *const DateUnit;
+        unsafe { *p }
+    }
+}
+
+impl flatbuffers::Push for DateUnit {
+    type Output = DateUnit;
+    #[inline]
+    fn push(&self, dst: &mut [u8], _rest: &[u8]) {
+        flatbuffers::emplace_scalar::<DateUnit>(dst, *self);
+    }
+}
+
+#[allow(non_camel_case_types)]
+const ENUM_VALUES_DATE_UNIT: [DateUnit; 2] = [DateUnit::DAY, DateUnit::MILLISECOND];
+
+#[allow(non_camel_case_types)]
+const ENUM_NAMES_DATE_UNIT: [&'static str; 2] = ["DAY", "MILLISECOND"];
+
+pub fn enum_name_date_unit(e: DateUnit) -> &'static str {
+    let index: usize = e as usize;
+    ENUM_NAMES_DATE_UNIT[index]
+}
+
+#[allow(non_camel_case_types)]
+#[repr(i16)]
+#[derive(Clone, Copy, PartialEq, Debug)]
+pub enum TimeUnit {
+    SECOND = 0,
+    MILLISECOND = 1,
+    MICROSECOND = 2,
+    NANOSECOND = 3,
+}
+
+const ENUM_MIN_TIME_UNIT: i16 = 0;
+const ENUM_MAX_TIME_UNIT: i16 = 3;
+
+impl<'a> flatbuffers::Follow<'a> for TimeUnit {
+    type Inner = Self;
+    #[inline]
+    fn follow(buf: &'a [u8], loc: usize) -> Self::Inner {
+        flatbuffers::read_scalar_at::<Self>(buf, loc)
+    }
+}
+
+impl flatbuffers::EndianScalar for TimeUnit {
+    #[inline]
+    fn to_little_endian(self) -> Self {
+        let n = i16::to_le(self as i16);
+        let p = &n as *const i16 as *const TimeUnit;
+        unsafe { *p }
+    }
+    #[inline]
+    fn from_little_endian(self) -> Self {
+        let n = i16::from_le(self as i16);
+        let p = &n as *const i16 as *const TimeUnit;
+        unsafe { *p }
+    }
+}
+
+impl flatbuffers::Push for TimeUnit {
+    type Output = TimeUnit;
+    #[inline]
+    fn push(&self, dst: &mut [u8], _rest: &[u8]) {
+        flatbuffers::emplace_scalar::<TimeUnit>(dst, *self);
+    }
+}
+
+#[allow(non_camel_case_types)]
+const ENUM_VALUES_TIME_UNIT: [TimeUnit; 4] = [
+    TimeUnit::SECOND,
+    TimeUnit::MILLISECOND,
+    TimeUnit::MICROSECOND,
+    TimeUnit::NANOSECOND,
+];
+
+#[allow(non_camel_case_types)]
+const ENUM_NAMES_TIME_UNIT: [&'static str; 4] =
+    ["SECOND", "MILLISECOND", "MICROSECOND", "NANOSECOND"];
+
+pub fn enum_name_time_unit(e: TimeUnit) -> &'static str {
+    let index: usize = e as usize;
+    ENUM_NAMES_TIME_UNIT[index]
+}
+
+#[allow(non_camel_case_types)]
+#[repr(i16)]
+#[derive(Clone, Copy, PartialEq, Debug)]
+pub enum IntervalUnit {
+    YEAR_MONTH = 0,
+    DAY_TIME = 1,
+}
+
+const ENUM_MIN_INTERVAL_UNIT: i16 = 0;
+const ENUM_MAX_INTERVAL_UNIT: i16 = 1;
+
+impl<'a> flatbuffers::Follow<'a> for IntervalUnit {
+    type Inner = Self;
+    #[inline]
+    fn follow(buf: &'a [u8], loc: usize) -> Self::Inner {
+        flatbuffers::read_scalar_at::<Self>(buf, loc)
+    }
+}
+
+impl flatbuffers::EndianScalar for IntervalUnit {
+    #[inline]
+    fn to_little_endian(self) -> Self {
+        let n = i16::to_le(self as i16);
+        let p = &n as *const i16 as *const IntervalUnit;
+        unsafe { *p }
+    }
+    #[inline]
+    fn from_little_endian(self) -> Self {
+        let n = i16::from_le(self as i16);
+        let p = &n as *const i16 as *const IntervalUnit;
+        unsafe { *p }
+    }
+}
+
+impl flatbuffers::Push for IntervalUnit {
+    type Output = IntervalUnit;
+    #[inline]
+    fn push(&self, dst: &mut [u8], _rest: &[u8]) {
+        flatbuffers::emplace_scalar::<IntervalUnit>(dst, *self);
+    }
+}
+
+#[allow(non_camel_case_types)]
+const ENUM_VALUES_INTERVAL_UNIT: [IntervalUnit; 2] =
+    [IntervalUnit::YEAR_MONTH, IntervalUnit::DAY_TIME];
+
+#[allow(non_camel_case_types)]
+const ENUM_NAMES_INTERVAL_UNIT: [&'static str; 2] = ["YEAR_MONTH", "DAY_TIME"];
+
+pub fn enum_name_interval_unit(e: IntervalUnit) -> &'static str {
+    let index: usize = e as usize;
+    ENUM_NAMES_INTERVAL_UNIT[index]
+}
+
+/// ----------------------------------------------------------------------
+/// Top-level Type value, enabling extensible type-specific metadata. We can
+/// add new logical types to Type without breaking backwards compatibility
+#[allow(non_camel_case_types)]
+#[repr(u8)]
+#[derive(Clone, Copy, PartialEq, Debug)]
+pub enum Type {
+    NONE = 0,
+    Null = 1,
+    Int = 2,
+    FloatingPoint = 3,
+    Binary = 4,
+    Utf8 = 5,
+    Bool = 6,
+    Decimal = 7,
+    Date = 8,
+    Time = 9,
+    Timestamp = 10,
+    Interval = 11,
+    List = 12,
+    Struct_ = 13,
+    Union = 14,
+    FixedSizeBinary = 15,
+    FixedSizeList = 16,
+    Map = 17,
+}
+
+const ENUM_MIN_TYPE: u8 = 0;
+const ENUM_MAX_TYPE: u8 = 17;
+
+impl<'a> flatbuffers::Follow<'a> for Type {
+    type Inner = Self;
+    #[inline]
+    fn follow(buf: &'a [u8], loc: usize) -> Self::Inner {
+        flatbuffers::read_scalar_at::<Self>(buf, loc)
+    }
+}
+
+impl flatbuffers::EndianScalar for Type {
+    #[inline]
+    fn to_little_endian(self) -> Self {
+        let n = u8::to_le(self as u8);
+        let p = &n as *const u8 as *const Type;
+        unsafe { *p }
+    }
+    #[inline]
+    fn from_little_endian(self) -> Self {
+        let n = u8::from_le(self as u8);
+        let p = &n as *const u8 as *const Type;
+        unsafe { *p }
+    }
+}
+
+impl flatbuffers::Push for Type {
+    type Output = Type;
+    #[inline]
+    fn push(&self, dst: &mut [u8], _rest: &[u8]) {
+        flatbuffers::emplace_scalar::<Type>(dst, *self);
+    }
+}
+
+#[allow(non_camel_case_types)]
+const ENUM_VALUES_TYPE: [Type; 18] = [
+    Type::NONE,
+    Type::Null,
+    Type::Int,
+    Type::FloatingPoint,
+    Type::Binary,
+    Type::Utf8,
+    Type::Bool,
+    Type::Decimal,
+    Type::Date,
+    Type::Time,
+    Type::Timestamp,
+    Type::Interval,
+    Type::List,
+    Type::Struct_,
+    Type::Union,
+    Type::FixedSizeBinary,
+    Type::FixedSizeList,
+    Type::Map,
+];
+
+#[allow(non_camel_case_types)]
+const ENUM_NAMES_TYPE: [&'static str; 18] = [
+    "NONE",
+    "Null",
+    "Int",
+    "FloatingPoint",
+    "Binary",
+    "Utf8",
+    "Bool",
+    "Decimal",
+    "Date",
+    "Time",
+    "Timestamp",
+    "Interval",
+    "List",
+    "Struct_",
+    "Union",
+    "FixedSizeBinary",
+    "FixedSizeList",
+    "Map",
+];
+
+pub fn enum_name_type(e: Type) -> &'static str {
+    let index: usize = e as usize;
+    ENUM_NAMES_TYPE[index]
+}
+
+pub struct TypeUnionTableOffset {}
+/// ----------------------------------------------------------------------
+/// Endianness of the platform producing the data
+#[allow(non_camel_case_types)]
+#[repr(i16)]
+#[derive(Clone, Copy, PartialEq, Debug)]
+pub enum Endianness {
+    Little = 0,
+    Big = 1,
+}
+
+const ENUM_MIN_ENDIANNESS: i16 = 0;
+const ENUM_MAX_ENDIANNESS: i16 = 1;
+
+impl<'a> flatbuffers::Follow<'a> for Endianness {
+    type Inner = Self;
+    #[inline]
+    fn follow(buf: &'a [u8], loc: usize) -> Self::Inner {
+        flatbuffers::read_scalar_at::<Self>(buf, loc)
+    }
+}
+
+impl flatbuffers::EndianScalar for Endianness {
+    #[inline]
+    fn to_little_endian(self) -> Self {
+        let n = i16::to_le(self as i16);
+        let p = &n as *const i16 as *const Endianness;
+        unsafe { *p }
+    }
+    #[inline]
+    fn from_little_endian(self) -> Self {
+        let n = i16::from_le(self as i16);
+        let p = &n as *const i16 as *const Endianness;
+        unsafe { *p }
+    }
+}
+
+impl flatbuffers::Push for Endianness {
+    type Output = Endianness;
+    #[inline]
+    fn push(&self, dst: &mut [u8], _rest: &[u8]) {
+        flatbuffers::emplace_scalar::<Endianness>(dst, *self);
+    }
+}
+
+#[allow(non_camel_case_types)]
+const ENUM_VALUES_ENDIANNESS: [Endianness; 2] = [Endianness::Little, Endianness::Big];
+
+#[allow(non_camel_case_types)]
+const ENUM_NAMES_ENDIANNESS: [&'static str; 2] = ["Little", "Big"];
+
+pub fn enum_name_endianness(e: Endianness) -> &'static str {
+    let index: usize = e as usize;
+    ENUM_NAMES_ENDIANNESS[index]
+}
+
+/// ----------------------------------------------------------------------
+/// A Buffer represents a single contiguous memory segment
+// struct Buffer, aligned to 8
+#[repr(C, align(8))]
+#[derive(Clone, Copy, Debug, PartialEq)]
+pub struct Buffer {
+    offset_: i64,
+    length_: i64,
+} // pub struct Buffer
+impl flatbuffers::SafeSliceAccess for Buffer {}
+impl<'a> flatbuffers::Follow<'a> for Buffer {
+    type Inner = &'a Buffer;
+    #[inline]
+    fn follow(buf: &'a [u8], loc: usize) -> Self::Inner {
+        <&'a Buffer>::follow(buf, loc)
+    }
+}
+impl<'a> flatbuffers::Follow<'a> for &'a Buffer {
+    type Inner = &'a Buffer;
+    #[inline]
+    fn follow(buf: &'a [u8], loc: usize) -> Self::Inner {
+        flatbuffers::follow_cast_ref::<Buffer>(buf, loc)
+    }
+}
+impl<'b> flatbuffers::Push for Buffer {
+    type Output = Buffer;
+    #[inline]
+    fn push(&self, dst: &mut [u8], _rest: &[u8]) {
+        let src = unsafe {
+            ::std::slice::from_raw_parts(self as *const Buffer as *const u8, Self::size())
+        };
+        dst.copy_from_slice(src);
+    }
+}
+impl<'b> flatbuffers::Push for &'b Buffer {
+    type Output = Buffer;
+
+    #[inline]
+    fn push(&self, dst: &mut [u8], _rest: &[u8]) {
+        let src = unsafe {
+            ::std::slice::from_raw_parts(
+                *self as *const Buffer as *const u8,
+                Self::size(),
+            )
+        };
+        dst.copy_from_slice(src);
+    }
+}
+
+impl Buffer {
+    pub fn new<'a>(_offset: i64, _length: i64) -> Self {
+        Buffer {
+            offset_: _offset.to_little_endian(),
+            length_: _length.to_little_endian(),
+        }
+    }
+    /// The relative offset into the shared memory page where the bytes for this
+    /// buffer starts
+    pub fn offset<'a>(&'a self) -> i64 {
+        self.offset_.from_little_endian()
+    }
+    /// The absolute length (in bytes) of the memory buffer. The memory is found
+    /// from offset (inclusive) to offset + length (non-inclusive).
+    pub fn length<'a>(&'a self) -> i64 {
+        self.length_.from_little_endian()
+    }
+}
+
+/// These are stored in the flatbuffer in the Type union below
+pub enum NullOffset {}
+#[derive(Copy, Clone, Debug, PartialEq)]
+
+pub struct Null<'a> {
+    pub _tab: flatbuffers::Table<'a>,
+}
+
+impl<'a> flatbuffers::Follow<'a> for Null<'a> {
+    type Inner = Null<'a>;
+    #[inline]
+    fn follow(buf: &'a [u8], loc: usize) -> Self::Inner {
+        Self {
+            _tab: flatbuffers::Table { buf: buf, loc: loc },
+        }
+    }
+}
+
+impl<'a> Null<'a> {
+    #[inline]
+    pub fn init_from_table(table: flatbuffers::Table<'a>) -> Self {
+        Null { _tab: table }
+    }
+    #[allow(unused_mut)]
+    pub fn create<'bldr: 'args, 'args: 'mut_bldr, 'mut_bldr>(
+        _fbb: &'mut_bldr mut flatbuffers::FlatBufferBuilder<'bldr>,
+        _args: &'args NullArgs,
+    ) -> flatbuffers::WIPOffset<Null<'bldr>> {
+        let mut builder = NullBuilder::new(_fbb);
+        builder.finish()
+    }
+}
+
+pub struct NullArgs {}
+impl<'a> Default for NullArgs {
+    #[inline]
+    fn default() -> Self {
+        NullArgs {}
+    }
+}
+pub struct NullBuilder<'a: 'b, 'b> {
+    fbb_: &'b mut flatbuffers::FlatBufferBuilder<'a>,
+    start_: flatbuffers::WIPOffset<flatbuffers::TableUnfinishedWIPOffset>,
+}
+impl<'a: 'b, 'b> NullBuilder<'a, 'b> {
+    #[inline]
+    pub fn new(_fbb: &'b mut flatbuffers::FlatBufferBuilder<'a>) -> NullBuilder<'a, 'b> {
+        let start = _fbb.start_table();
+        NullBuilder {
+            fbb_: _fbb,
+            start_: start,
+        }
+    }
+    #[inline]
+    pub fn finish(self) -> flatbuffers::WIPOffset<Null<'a>> {
+        let o = self.fbb_.end_table(self.start_);
+        flatbuffers::WIPOffset::new(o.value())
+    }
+}
+
+/// A Struct_ in the flatbuffer metadata is the same as an Arrow Struct
+/// (according to the physical memory layout). We used Struct_ here as
+/// Struct is a reserved word in Flatbuffers
+pub enum Struct_Offset {}
+#[derive(Copy, Clone, Debug, PartialEq)]
+
+pub struct Struct_<'a> {
+    pub _tab: flatbuffers::Table<'a>,
+}
+
+impl<'a> flatbuffers::Follow<'a> for Struct_<'a> {
+    type Inner = Struct_<'a>;
+    #[inline]
+    fn follow(buf: &'a [u8], loc: usize) -> Self::Inner {
+        Self {
+            _tab: flatbuffers::Table { buf: buf, loc: loc },
+        }
+    }
+}
+
+impl<'a> Struct_<'a> {
+    #[inline]
+    pub fn init_from_table(table: flatbuffers::Table<'a>) -> Self {
+        Struct_ { _tab: table }
+    }
+    #[allow(unused_mut)]
+    pub fn create<'bldr: 'args, 'args: 'mut_bldr, 'mut_bldr>(
+        _fbb: &'mut_bldr mut flatbuffers::FlatBufferBuilder<'bldr>,
+        _args: &'args Struct_Args,
+    ) -> flatbuffers::WIPOffset<Struct_<'bldr>> {
+        let mut builder = Struct_Builder::new(_fbb);
+        builder.finish()
+    }
+}
+
+pub struct Struct_Args {}
+impl<'a> Default for Struct_Args {
+    #[inline]
+    fn default() -> Self {
+        Struct_Args {}
+    }
+}
+pub struct Struct_Builder<'a: 'b, 'b> {
+    fbb_: &'b mut flatbuffers::FlatBufferBuilder<'a>,
+    start_: flatbuffers::WIPOffset<flatbuffers::TableUnfinishedWIPOffset>,
+}
+impl<'a: 'b, 'b> Struct_Builder<'a, 'b> {
+    #[inline]
+    pub fn new(
+        _fbb: &'b mut flatbuffers::FlatBufferBuilder<'a>,
+    ) -> Struct_Builder<'a, 'b> {
+        let start = _fbb.start_table();
+        Struct_Builder {
+            fbb_: _fbb,
+            start_: start,
+        }
+    }
+    #[inline]
+    pub fn finish(self) -> flatbuffers::WIPOffset<Struct_<'a>> {
+        let o = self.fbb_.end_table(self.start_);
+        flatbuffers::WIPOffset::new(o.value())
+    }
+}
+
+pub enum ListOffset {}
+#[derive(Copy, Clone, Debug, PartialEq)]
+
+pub struct List<'a> {
+    pub _tab: flatbuffers::Table<'a>,
+}
+
+impl<'a> flatbuffers::Follow<'a> for List<'a> {
+    type Inner = List<'a>;
+    #[inline]
+    fn follow(buf: &'a [u8], loc: usize) -> Self::Inner {
+        Self {
+            _tab: flatbuffers::Table { buf: buf, loc: loc },
+        }
+    }
+}
+
+impl<'a> List<'a> {
+    #[inline]
+    pub fn init_from_table(table: flatbuffers::Table<'a>) -> Self {
+        List { _tab: table }
+    }
+    #[allow(unused_mut)]
+    pub fn create<'bldr: 'args, 'args: 'mut_bldr, 'mut_bldr>(
+        _fbb: &'mut_bldr mut flatbuffers::FlatBufferBuilder<'bldr>,
+        _args: &'args ListArgs,
+    ) -> flatbuffers::WIPOffset<List<'bldr>> {
+        let mut builder = ListBuilder::new(_fbb);
+        builder.finish()
+    }
+}
+
+pub struct ListArgs {}
+impl<'a> Default for ListArgs {
+    #[inline]
+    fn default() -> Self {
+        ListArgs {}
+    }
+}
+pub struct ListBuilder<'a: 'b, 'b> {
+    fbb_: &'b mut flatbuffers::FlatBufferBuilder<'a>,
+    start_: flatbuffers::WIPOffset<flatbuffers::TableUnfinishedWIPOffset>,
+}
+impl<'a: 'b, 'b> ListBuilder<'a, 'b> {
+    #[inline]
+    pub fn new(_fbb: &'b mut flatbuffers::FlatBufferBuilder<'a>) -> ListBuilder<'a, 'b> {
+        let start = _fbb.start_table();
+        ListBuilder {
+            fbb_: _fbb,
+            start_: start,
+        }
+    }
+    #[inline]
+    pub fn finish(self) -> flatbuffers::WIPOffset<List<'a>> {
+        let o = self.fbb_.end_table(self.start_);
+        flatbuffers::WIPOffset::new(o.value())
+    }
+}
+
+pub enum FixedSizeListOffset {}
+#[derive(Copy, Clone, Debug, PartialEq)]
+
+pub struct FixedSizeList<'a> {
+    pub _tab: flatbuffers::Table<'a>,
+}
+
+impl<'a> flatbuffers::Follow<'a> for FixedSizeList<'a> {
+    type Inner = FixedSizeList<'a>;
+    #[inline]
+    fn follow(buf: &'a [u8], loc: usize) -> Self::Inner {
+        Self {
+            _tab: flatbuffers::Table { buf: buf, loc: loc },
+        }
+    }
+}
+
+impl<'a> FixedSizeList<'a> {
+    #[inline]
+    pub fn init_from_table(table: flatbuffers::Table<'a>) -> Self {
+        FixedSizeList { _tab: table }
+    }
+    #[allow(unused_mut)]
+    pub fn create<'bldr: 'args, 'args: 'mut_bldr, 'mut_bldr>(
+        _fbb: &'mut_bldr mut flatbuffers::FlatBufferBuilder<'bldr>,
+        args: &'args FixedSizeListArgs,
+    ) -> flatbuffers::WIPOffset<FixedSizeList<'bldr>> {
+        let mut builder = FixedSizeListBuilder::new(_fbb);
+        builder.add_listSize(args.listSize);
+        builder.finish()
+    }
+
+    pub const VT_LISTSIZE: flatbuffers::VOffsetT = 4;
+
+    /// Number of list items per value
+    #[inline]
+    pub fn listSize(&self) -> i32 {
+        self._tab
+            .get::<i32>(FixedSizeList::VT_LISTSIZE, Some(0))
+            .unwrap()
+    }
+}
+
+pub struct FixedSizeListArgs {
+    pub listSize: i32,
+}
+impl<'a> Default for FixedSizeListArgs {
+    #[inline]
+    fn default() -> Self {
+        FixedSizeListArgs { listSize: 0 }
+    }
+}
+pub struct FixedSizeListBuilder<'a: 'b, 'b> {
+    fbb_: &'b mut flatbuffers::FlatBufferBuilder<'a>,
+    start_: flatbuffers::WIPOffset<flatbuffers::TableUnfinishedWIPOffset>,
+}
+impl<'a: 'b, 'b> FixedSizeListBuilder<'a, 'b> {
+    #[inline]
+    pub fn add_listSize(&mut self, listSize: i32) {
+        self.fbb_
+            .push_slot::<i32>(FixedSizeList::VT_LISTSIZE, listSize, 0);
+    }
+    #[inline]
+    pub fn new(
+        _fbb: &'b mut flatbuffers::FlatBufferBuilder<'a>,
+    ) -> FixedSizeListBuilder<'a, 'b> {
+        let start = _fbb.start_table();
+        FixedSizeListBuilder {
+            fbb_: _fbb,
+            start_: start,
+        }
+    }
+    #[inline]
+    pub fn finish(self) -> flatbuffers::WIPOffset<FixedSizeList<'a>> {
+        let o = self.fbb_.end_table(self.start_);
+        flatbuffers::WIPOffset::new(o.value())
+    }
+}
+
+/// A Map is a logical nested type that is represented as
+///
+/// List<entry: Struct<key: K, value: V>>
+///
+/// In this layout, the keys and values are each respectively contiguous. We do
+/// not constrain the key and value types, so the application is responsible
+/// for ensuring that the keys are hashable and unique. Whether the keys are sorted
+/// may be set in the metadata for this field
+///
+/// In a Field with Map type, the Field has a child Struct field, which then
+/// has two children: key type and the second the value type. The names of the
+/// child fields may be respectively "entry", "key", and "value", but this is
+/// not enforced
+///
+/// Map
+///   - child[0] entry: Struct
+///     - child[0] key: K
+///     - child[1] value: V
+///
+/// Neither the "entry" field nor the "key" field may be nullable.
+///
+/// The metadata is structured so that Arrow systems without special handling
+/// for Map can make Map an alias for List. The "layout" attribute for the Map
+/// field must have the same contents as a List.
+pub enum MapOffset {}
+#[derive(Copy, Clone, Debug, PartialEq)]
+
+pub struct Map<'a> {
+    pub _tab: flatbuffers::Table<'a>,
+}
+
+impl<'a> flatbuffers::Follow<'a> for Map<'a> {
+    type Inner = Map<'a>;
+    #[inline]
+    fn follow(buf: &'a [u8], loc: usize) -> Self::Inner {
+        Self {
+            _tab: flatbuffers::Table { buf: buf, loc: loc },
+        }
+    }
+}
+
+impl<'a> Map<'a> {
+    #[inline]
+    pub fn init_from_table(table: flatbuffers::Table<'a>) -> Self {
+        Map { _tab: table }
+    }
+    #[allow(unused_mut)]
+    pub fn create<'bldr: 'args, 'args: 'mut_bldr, 'mut_bldr>(
+        _fbb: &'mut_bldr mut flatbuffers::FlatBufferBuilder<'bldr>,
+        args: &'args MapArgs,
+    ) -> flatbuffers::WIPOffset<Map<'bldr>> {
+        let mut builder = MapBuilder::new(_fbb);
+        builder.add_keysSorted(args.keysSorted);
+        builder.finish()
+    }
+
+    pub const VT_KEYSSORTED: flatbuffers::VOffsetT = 4;
+
+    /// Set to true if the keys within each value are sorted
+    #[inline]
+    pub fn keysSorted(&self) -> bool {
+        self._tab
+            .get::<bool>(Map::VT_KEYSSORTED, Some(false))
+            .unwrap()
+    }
+}
+
+pub struct MapArgs {
+    pub keysSorted: bool,
+}
+impl<'a> Default for MapArgs {
+    #[inline]
+    fn default() -> Self {
+        MapArgs { keysSorted: false }
+    }
+}
+pub struct MapBuilder<'a: 'b, 'b> {
+    fbb_: &'b mut flatbuffers::FlatBufferBuilder<'a>,
+    start_: flatbuffers::WIPOffset<flatbuffers::TableUnfinishedWIPOffset>,
+}
+impl<'a: 'b, 'b> MapBuilder<'a, 'b> {
+    #[inline]
+    pub fn add_keysSorted(&mut self, keysSorted: bool) {
+        self.fbb_
+            .push_slot::<bool>(Map::VT_KEYSSORTED, keysSorted, false);
+    }
+    #[inline]
+    pub fn new(_fbb: &'b mut flatbuffers::FlatBufferBuilder<'a>) -> MapBuilder<'a, 'b> {
+        let start = _fbb.start_table();
+        MapBuilder {
+            fbb_: _fbb,
+            start_: start,
+        }
+    }
+    #[inline]
+    pub fn finish(self) -> flatbuffers::WIPOffset<Map<'a>> {
+        let o = self.fbb_.end_table(self.start_);
+        flatbuffers::WIPOffset::new(o.value())
+    }
+}
+
+/// A union is a complex type with children in Field
+/// By default ids in the type vector refer to the offsets in the children
+/// optionally typeIds provides an indirection between the child offset and the type id
+/// for each child typeIds[offset] is the id used in the type vector
+pub enum UnionOffset {}
+#[derive(Copy, Clone, Debug, PartialEq)]
+
+pub struct Union<'a> {
+    pub _tab: flatbuffers::Table<'a>,
+}
+
+impl<'a> flatbuffers::Follow<'a> for Union<'a> {
+    type Inner = Union<'a>;
+    #[inline]
+    fn follow(buf: &'a [u8], loc: usize) -> Self::Inner {
+        Self {
+            _tab: flatbuffers::Table { buf: buf, loc: loc },
+        }
+    }
+}
+
+impl<'a> Union<'a> {
+    #[inline]
+    pub fn init_from_table(table: flatbuffers::Table<'a>) -> Self {
+        Union { _tab: table }
+    }
+    #[allow(unused_mut)]
+    pub fn create<'bldr: 'args, 'args: 'mut_bldr, 'mut_bldr>(
+        _fbb: &'mut_bldr mut flatbuffers::FlatBufferBuilder<'bldr>,
+        args: &'args UnionArgs<'args>,
+    ) -> flatbuffers::WIPOffset<Union<'bldr>> {
+        let mut builder = UnionBuilder::new(_fbb);
+        if let Some(x) = args.typeIds {
+            builder.add_typeIds(x);
+        }
+        builder.add_mode(args.mode);
+        builder.finish()
+    }
+
+    pub const VT_MODE: flatbuffers::VOffsetT = 4;
+    pub const VT_TYPEIDS: flatbuffers::VOffsetT = 6;
+
+    #[inline]
+    pub fn mode(&self) -> UnionMode {
+        self._tab
+            .get::<UnionMode>(Union::VT_MODE, Some(UnionMode::Sparse))
+            .unwrap()
+    }
+    #[inline]
+    pub fn typeIds(&self) -> Option<flatbuffers::Vector<'a, i32>> {
+        self._tab
+            .get::<flatbuffers::ForwardsUOffset<flatbuffers::Vector<'a, i32>>>(
+                Union::VT_TYPEIDS,
+                None,
+            )
+    }
+}
+
+pub struct UnionArgs<'a> {
+    pub mode: UnionMode,
+    pub typeIds: Option<flatbuffers::WIPOffset<flatbuffers::Vector<'a, i32>>>,
+}
+impl<'a> Default for UnionArgs<'a> {
+    #[inline]
+    fn default() -> Self {
+        UnionArgs {
+            mode: UnionMode::Sparse,
+            typeIds: None,
+        }
+    }
+}
+pub struct UnionBuilder<'a: 'b, 'b> {
+    fbb_: &'b mut flatbuffers::FlatBufferBuilder<'a>,
+    start_: flatbuffers::WIPOffset<flatbuffers::TableUnfinishedWIPOffset>,
+}
+impl<'a: 'b, 'b> UnionBuilder<'a, 'b> {
+    #[inline]
+    pub fn add_mode(&mut self, mode: UnionMode) {
+        self.fbb_
+            .push_slot::<UnionMode>(Union::VT_MODE, mode, UnionMode::Sparse);
+    }
+    #[inline]
+    pub fn add_typeIds(
+        &mut self,
+        typeIds: flatbuffers::WIPOffset<flatbuffers::Vector<'b, i32>>,
+    ) {
+        self.fbb_
+            .push_slot_always::<flatbuffers::WIPOffset<_>>(Union::VT_TYPEIDS, typeIds);
+    }
+    #[inline]
+    pub fn new(_fbb: &'b mut flatbuffers::FlatBufferBuilder<'a>) -> UnionBuilder<'a, 'b> {
+        let start = _fbb.start_table();
+        UnionBuilder {
+            fbb_: _fbb,
+            start_: start,
+        }
+    }
+    #[inline]
+    pub fn finish(self) -> flatbuffers::WIPOffset<Union<'a>> {
+        let o = self.fbb_.end_table(self.start_);
+        flatbuffers::WIPOffset::new(o.value())
+    }
+}
+
+pub enum IntOffset {}
+#[derive(Copy, Clone, Debug, PartialEq)]
+
+pub struct Int<'a> {
+    pub _tab: flatbuffers::Table<'a>,
+}
+
+impl<'a> flatbuffers::Follow<'a> for Int<'a> {
+    type Inner = Int<'a>;
+    #[inline]
+    fn follow(buf: &'a [u8], loc: usize) -> Self::Inner {
+        Self {
+            _tab: flatbuffers::Table { buf: buf, loc: loc },
+        }
+    }
+}
+
+impl<'a> Int<'a> {
+    #[inline]
+    pub fn init_from_table(table: flatbuffers::Table<'a>) -> Self {
+        Int { _tab: table }
+    }
+    #[allow(unused_mut)]
+    pub fn create<'bldr: 'args, 'args: 'mut_bldr, 'mut_bldr>(
+        _fbb: &'mut_bldr mut flatbuffers::FlatBufferBuilder<'bldr>,
+        args: &'args IntArgs,
+    ) -> flatbuffers::WIPOffset<Int<'bldr>> {
+        let mut builder = IntBuilder::new(_fbb);
+        builder.add_bitWidth(args.bitWidth);
+        builder.add_is_signed(args.is_signed);
+        builder.finish()
+    }
+
+    pub const VT_BITWIDTH: flatbuffers::VOffsetT = 4;
+    pub const VT_IS_SIGNED: flatbuffers::VOffsetT = 6;
+
+    #[inline]
+    pub fn bitWidth(&self) -> i32 {
+        self._tab.get::<i32>(Int::VT_BITWIDTH, Some(0)).unwrap()
+    }
+    #[inline]
+    pub fn is_signed(&self) -> bool {
+        self._tab
+            .get::<bool>(Int::VT_IS_SIGNED, Some(false))
+            .unwrap()
+    }
+}
+
+pub struct IntArgs {
+    pub bitWidth: i32,
+    pub is_signed: bool,
+}
+impl<'a> Default for IntArgs {
+    #[inline]
+    fn default() -> Self {
+        IntArgs {
+            bitWidth: 0,
+            is_signed: false,
+        }
+    }
+}
+pub struct IntBuilder<'a: 'b, 'b> {
+    fbb_: &'b mut flatbuffers::FlatBufferBuilder<'a>,
+    start_: flatbuffers::WIPOffset<flatbuffers::TableUnfinishedWIPOffset>,
+}
+impl<'a: 'b, 'b> IntBuilder<'a, 'b> {
+    #[inline]
+    pub fn add_bitWidth(&mut self, bitWidth: i32) {
+        self.fbb_.push_slot::<i32>(Int::VT_BITWIDTH, bitWidth, 0);
+    }
+    #[inline]
+    pub fn add_is_signed(&mut self, is_signed: bool) {
+        self.fbb_
+            .push_slot::<bool>(Int::VT_IS_SIGNED, is_signed, false);
+    }
+    #[inline]
+    pub fn new(_fbb: &'b mut flatbuffers::FlatBufferBuilder<'a>) -> IntBuilder<'a, 'b> {
+        let start = _fbb.start_table();
+        IntBuilder {
+            fbb_: _fbb,
+            start_: start,
+        }
+    }
+    #[inline]
+    pub fn finish(self) -> flatbuffers::WIPOffset<Int<'a>> {
+        let o = self.fbb_.end_table(self.start_);
+        flatbuffers::WIPOffset::new(o.value())
+    }
+}
+
+pub enum FloatingPointOffset {}
+#[derive(Copy, Clone, Debug, PartialEq)]
+
+pub struct FloatingPoint<'a> {
+    pub _tab: flatbuffers::Table<'a>,
+}
+
+impl<'a> flatbuffers::Follow<'a> for FloatingPoint<'a> {
+    type Inner = FloatingPoint<'a>;
+    #[inline]
+    fn follow(buf: &'a [u8], loc: usize) -> Self::Inner {
+        Self {
+            _tab: flatbuffers::Table { buf: buf, loc: loc },
+        }
+    }
+}
+
+impl<'a> FloatingPoint<'a> {
+    #[inline]
+    pub fn init_from_table(table: flatbuffers::Table<'a>) -> Self {
+        FloatingPoint { _tab: table }
+    }
+    #[allow(unused_mut)]
+    pub fn create<'bldr: 'args, 'args: 'mut_bldr, 'mut_bldr>(
+        _fbb: &'mut_bldr mut flatbuffers::FlatBufferBuilder<'bldr>,
+        args: &'args FloatingPointArgs,
+    ) -> flatbuffers::WIPOffset<FloatingPoint<'bldr>> {
+        let mut builder = FloatingPointBuilder::new(_fbb);
+        builder.add_precision(args.precision);
+        builder.finish()
+    }
+
+    pub const VT_PRECISION: flatbuffers::VOffsetT = 4;
+
+    #[inline]
+    pub fn precision(&self) -> Precision {
+        self._tab
+            .get::<Precision>(FloatingPoint::VT_PRECISION, Some(Precision::HALF))
+            .unwrap()
+    }
+}
+
+pub struct FloatingPointArgs {
+    pub precision: Precision,
+}
+impl<'a> Default for FloatingPointArgs {
+    #[inline]
+    fn default() -> Self {
+        FloatingPointArgs {
+            precision: Precision::HALF,
+        }
+    }
+}
+pub struct FloatingPointBuilder<'a: 'b, 'b> {
+    fbb_: &'b mut flatbuffers::FlatBufferBuilder<'a>,
+    start_: flatbuffers::WIPOffset<flatbuffers::TableUnfinishedWIPOffset>,
+}
+impl<'a: 'b, 'b> FloatingPointBuilder<'a, 'b> {
+    #[inline]
+    pub fn add_precision(&mut self, precision: Precision) {
+        self.fbb_.push_slot::<Precision>(
+            FloatingPoint::VT_PRECISION,
+            precision,
+            Precision::HALF,
+        );
+    }
+    #[inline]
+    pub fn new(
+        _fbb: &'b mut flatbuffers::FlatBufferBuilder<'a>,
+    ) -> FloatingPointBuilder<'a, 'b> {
+        let start = _fbb.start_table();
+        FloatingPointBuilder {
+            fbb_: _fbb,
+            start_: start,
+        }
+    }
+    #[inline]
+    pub fn finish(self) -> flatbuffers::WIPOffset<FloatingPoint<'a>> {
+        let o = self.fbb_.end_table(self.start_);
+        flatbuffers::WIPOffset::new(o.value())
+    }
+}
+
+/// Unicode with UTF-8 encoding
+pub enum Utf8Offset {}
+#[derive(Copy, Clone, Debug, PartialEq)]
+
+pub struct Utf8<'a> {
+    pub _tab: flatbuffers::Table<'a>,
+}
+
+impl<'a> flatbuffers::Follow<'a> for Utf8<'a> {
+    type Inner = Utf8<'a>;
+    #[inline]
+    fn follow(buf: &'a [u8], loc: usize) -> Self::Inner {
+        Self {
+            _tab: flatbuffers::Table { buf: buf, loc: loc },
+        }
+    }
+}
+
+impl<'a> Utf8<'a> {
+    #[inline]
+    pub fn init_from_table(table: flatbuffers::Table<'a>) -> Self {
+        Utf8 { _tab: table }
+    }
+    #[allow(unused_mut)]
+    pub fn create<'bldr: 'args, 'args: 'mut_bldr, 'mut_bldr>(
+        _fbb: &'mut_bldr mut flatbuffers::FlatBufferBuilder<'bldr>,
+        _args: &'args Utf8Args,
+    ) -> flatbuffers::WIPOffset<Utf8<'bldr>> {
+        let mut builder = Utf8Builder::new(_fbb);
+        builder.finish()
+    }
+}
+
+pub struct Utf8Args {}
+impl<'a> Default for Utf8Args {
+    #[inline]
+    fn default() -> Self {
+        Utf8Args {}
+    }
+}
+pub struct Utf8Builder<'a: 'b, 'b> {
+    fbb_: &'b mut flatbuffers::FlatBufferBuilder<'a>,
+    start_: flatbuffers::WIPOffset<flatbuffers::TableUnfinishedWIPOffset>,
+}
+impl<'a: 'b, 'b> Utf8Builder<'a, 'b> {
+    #[inline]
+    pub fn new(_fbb: &'b mut flatbuffers::FlatBufferBuilder<'a>) -> Utf8Builder<'a, 'b> {
+        let start = _fbb.start_table();
+        Utf8Builder {
+            fbb_: _fbb,
+            start_: start,
+        }
+    }
+    #[inline]
+    pub fn finish(self) -> flatbuffers::WIPOffset<Utf8<'a>> {
+        let o = self.fbb_.end_table(self.start_);
+        flatbuffers::WIPOffset::new(o.value())
+    }
+}
+
+pub enum BinaryOffset {}
+#[derive(Copy, Clone, Debug, PartialEq)]
+
+pub struct Binary<'a> {
+    pub _tab: flatbuffers::Table<'a>,
+}
+
+impl<'a> flatbuffers::Follow<'a> for Binary<'a> {
+    type Inner = Binary<'a>;
+    #[inline]
+    fn follow(buf: &'a [u8], loc: usize) -> Self::Inner {
+        Self {
+            _tab: flatbuffers::Table { buf: buf, loc: loc },
+        }
+    }
+}
+
+impl<'a> Binary<'a> {
+    #[inline]
+    pub fn init_from_table(table: flatbuffers::Table<'a>) -> Self {
+        Binary { _tab: table }
+    }
+    #[allow(unused_mut)]
+    pub fn create<'bldr: 'args, 'args: 'mut_bldr, 'mut_bldr>(
+        _fbb: &'mut_bldr mut flatbuffers::FlatBufferBuilder<'bldr>,
+        _args: &'args BinaryArgs,
+    ) -> flatbuffers::WIPOffset<Binary<'bldr>> {
+        let mut builder = BinaryBuilder::new(_fbb);
+        builder.finish()
+    }
+}
+
+pub struct BinaryArgs {}
+impl<'a> Default for BinaryArgs {
+    #[inline]
+    fn default() -> Self {
+        BinaryArgs {}
+    }
+}
+pub struct BinaryBuilder<'a: 'b, 'b> {
+    fbb_: &'b mut flatbuffers::FlatBufferBuilder<'a>,
+    start_: flatbuffers::WIPOffset<flatbuffers::TableUnfinishedWIPOffset>,
+}
+impl<'a: 'b, 'b> BinaryBuilder<'a, 'b> {
+    #[inline]
+    pub fn new(
+        _fbb: &'b mut flatbuffers::FlatBufferBuilder<'a>,
+    ) -> BinaryBuilder<'a, 'b> {
+        let start = _fbb.start_table();
+        BinaryBuilder {
+            fbb_: _fbb,
+            start_: start,
+        }
+    }
+    #[inline]
+    pub fn finish(self) -> flatbuffers::WIPOffset<Binary<'a>> {
+        let o = self.fbb_.end_table(self.start_);
+        flatbuffers::WIPOffset::new(o.value())
+    }
+}
+
+pub enum FixedSizeBinaryOffset {}
+#[derive(Copy, Clone, Debug, PartialEq)]
+
+pub struct FixedSizeBinary<'a> {
+    pub _tab: flatbuffers::Table<'a>,
+}
+
+impl<'a> flatbuffers::Follow<'a> for FixedSizeBinary<'a> {
+    type Inner = FixedSizeBinary<'a>;
+    #[inline]
+    fn follow(buf: &'a [u8], loc: usize) -> Self::Inner {
+        Self {
+            _tab: flatbuffers::Table { buf: buf, loc: loc },
+        }
+    }
+}
+
+impl<'a> FixedSizeBinary<'a> {
+    #[inline]
+    pub fn init_from_table(table: flatbuffers::Table<'a>) -> Self {
+        FixedSizeBinary { _tab: table }
+    }
+    #[allow(unused_mut)]
+    pub fn create<'bldr: 'args, 'args: 'mut_bldr, 'mut_bldr>(
+        _fbb: &'mut_bldr mut flatbuffers::FlatBufferBuilder<'bldr>,
+        args: &'args FixedSizeBinaryArgs,
+    ) -> flatbuffers::WIPOffset<FixedSizeBinary<'bldr>> {
+        let mut builder = FixedSizeBinaryBuilder::new(_fbb);
+        builder.add_byteWidth(args.byteWidth);
+        builder.finish()
+    }
+
+    pub const VT_BYTEWIDTH: flatbuffers::VOffsetT = 4;
+
+    /// Number of bytes per value
+    #[inline]
+    pub fn byteWidth(&self) -> i32 {
+        self._tab
+            .get::<i32>(FixedSizeBinary::VT_BYTEWIDTH, Some(0))
+            .unwrap()
+    }
+}
+
+pub struct FixedSizeBinaryArgs {
+    pub byteWidth: i32,
+}
+impl<'a> Default for FixedSizeBinaryArgs {
+    #[inline]
+    fn default() -> Self {
+        FixedSizeBinaryArgs { byteWidth: 0 }
+    }
+}
+pub struct FixedSizeBinaryBuilder<'a: 'b, 'b> {
+    fbb_: &'b mut flatbuffers::FlatBufferBuilder<'a>,
+    start_: flatbuffers::WIPOffset<flatbuffers::TableUnfinishedWIPOffset>,
+}
+impl<'a: 'b, 'b> FixedSizeBinaryBuilder<'a, 'b> {
+    #[inline]
+    pub fn add_byteWidth(&mut self, byteWidth: i32) {
+        self.fbb_
+            .push_slot::<i32>(FixedSizeBinary::VT_BYTEWIDTH, byteWidth, 0);
+    }
+    #[inline]
+    pub fn new(
+        _fbb: &'b mut flatbuffers::FlatBufferBuilder<'a>,
+    ) -> FixedSizeBinaryBuilder<'a, 'b> {
+        let start = _fbb.start_table();
+        FixedSizeBinaryBuilder {
+            fbb_: _fbb,
+            start_: start,
+        }
+    }
+    #[inline]
+    pub fn finish(self) -> flatbuffers::WIPOffset<FixedSizeBinary<'a>> {
+        let o = self.fbb_.end_table(self.start_);
+        flatbuffers::WIPOffset::new(o.value())
+    }
+}
+
+pub enum BoolOffset {}
+#[derive(Copy, Clone, Debug, PartialEq)]
+
+pub struct Bool<'a> {
+    pub _tab: flatbuffers::Table<'a>,
+}
+
+impl<'a> flatbuffers::Follow<'a> for Bool<'a> {
+    type Inner = Bool<'a>;
+    #[inline]
+    fn follow(buf: &'a [u8], loc: usize) -> Self::Inner {
+        Self {
+            _tab: flatbuffers::Table { buf: buf, loc: loc },
+        }
+    }
+}
+
+impl<'a> Bool<'a> {
+    #[inline]
+    pub fn init_from_table(table: flatbuffers::Table<'a>) -> Self {
+        Bool { _tab: table }
+    }
+    #[allow(unused_mut)]
+    pub fn create<'bldr: 'args, 'args: 'mut_bldr, 'mut_bldr>(
+        _fbb: &'mut_bldr mut flatbuffers::FlatBufferBuilder<'bldr>,
+        _args: &'args BoolArgs,
+    ) -> flatbuffers::WIPOffset<Bool<'bldr>> {
+        let mut builder = BoolBuilder::new(_fbb);
+        builder.finish()
+    }
+}
+
+pub struct BoolArgs {}
+impl<'a> Default for BoolArgs {
+    #[inline]
+    fn default() -> Self {
+        BoolArgs {}
+    }
+}
+pub struct BoolBuilder<'a: 'b, 'b> {
+    fbb_: &'b mut flatbuffers::FlatBufferBuilder<'a>,
+    start_: flatbuffers::WIPOffset<flatbuffers::TableUnfinishedWIPOffset>,
+}
+impl<'a: 'b, 'b> BoolBuilder<'a, 'b> {
+    #[inline]
+    pub fn new(_fbb: &'b mut flatbuffers::FlatBufferBuilder<'a>) -> BoolBuilder<'a, 'b> {
+        let start = _fbb.start_table();
+        BoolBuilder {
+            fbb_: _fbb,
+            start_: start,
+        }
+    }
+    #[inline]
+    pub fn finish(self) -> flatbuffers::WIPOffset<Bool<'a>> {
+        let o = self.fbb_.end_table(self.start_);
+        flatbuffers::WIPOffset::new(o.value())
+    }
+}
+
+pub enum DecimalOffset {}
+#[derive(Copy, Clone, Debug, PartialEq)]
+
+pub struct Decimal<'a> {
+    pub _tab: flatbuffers::Table<'a>,
+}
+
+impl<'a> flatbuffers::Follow<'a> for Decimal<'a> {
+    type Inner = Decimal<'a>;
+    #[inline]
+    fn follow(buf: &'a [u8], loc: usize) -> Self::Inner {
+        Self {
+            _tab: flatbuffers::Table { buf: buf, loc: loc },
+        }
+    }
+}
+
+impl<'a> Decimal<'a> {
+    #[inline]
+    pub fn init_from_table(table: flatbuffers::Table<'a>) -> Self {
+        Decimal { _tab: table }
+    }
+    #[allow(unused_mut)]
+    pub fn create<'bldr: 'args, 'args: 'mut_bldr, 'mut_bldr>(
+        _fbb: &'mut_bldr mut flatbuffers::FlatBufferBuilder<'bldr>,
+        args: &'args DecimalArgs,
+    ) -> flatbuffers::WIPOffset<Decimal<'bldr>> {
+        let mut builder = DecimalBuilder::new(_fbb);
+        builder.add_scale(args.scale);
+        builder.add_precision(args.precision);
+        builder.finish()
+    }
+
+    pub const VT_PRECISION: flatbuffers::VOffsetT = 4;
+    pub const VT_SCALE: flatbuffers::VOffsetT = 6;
+
+    /// Total number of decimal digits
+    #[inline]
+    pub fn precision(&self) -> i32 {
+        self._tab
+            .get::<i32>(Decimal::VT_PRECISION, Some(0))
+            .unwrap()
+    }
+    /// Number of digits after the decimal point "."
+    #[inline]
+    pub fn scale(&self) -> i32 {
+        self._tab.get::<i32>(Decimal::VT_SCALE, Some(0)).unwrap()
+    }
+}
+
+pub struct DecimalArgs {
+    pub precision: i32,
+    pub scale: i32,
+}
+impl<'a> Default for DecimalArgs {
+    #[inline]
+    fn default() -> Self {
+        DecimalArgs {
+            precision: 0,
+            scale: 0,
+        }
+    }
+}
+pub struct DecimalBuilder<'a: 'b, 'b> {
+    fbb_: &'b mut flatbuffers::FlatBufferBuilder<'a>,
+    start_: flatbuffers::WIPOffset<flatbuffers::TableUnfinishedWIPOffset>,
+}
+impl<'a: 'b, 'b> DecimalBuilder<'a, 'b> {
+    #[inline]
+    pub fn add_precision(&mut self, precision: i32) {
+        self.fbb_
+            .push_slot::<i32>(Decimal::VT_PRECISION, precision, 0);
+    }
+    #[inline]
+    pub fn add_scale(&mut self, scale: i32) {
+        self.fbb_.push_slot::<i32>(Decimal::VT_SCALE, scale, 0);
+    }
+    #[inline]
+    pub fn new(
+        _fbb: &'b mut flatbuffers::FlatBufferBuilder<'a>,
+    ) -> DecimalBuilder<'a, 'b> {
+        let start = _fbb.start_table();
+        DecimalBuilder {
+            fbb_: _fbb,
+            start_: start,
+        }
+    }
+    #[inline]
+    pub fn finish(self) -> flatbuffers::WIPOffset<Decimal<'a>> {
+        let o = self.fbb_.end_table(self.start_);
+        flatbuffers::WIPOffset::new(o.value())
+    }
+}
+
+/// Date is either a 32-bit or 64-bit type representing elapsed time since UNIX
+/// epoch (1970-01-01), stored in either of two units:
+///
+/// * Milliseconds (64 bits) indicating UNIX time elapsed since the epoch (no
+///   leap seconds), where the values are evenly divisible by 86400000
+/// * Days (32 bits) since the UNIX epoch
+pub enum DateOffset {}
+#[derive(Copy, Clone, Debug, PartialEq)]
+
+pub struct Date<'a> {
+    pub _tab: flatbuffers::Table<'a>,
+}
+
+impl<'a> flatbuffers::Follow<'a> for Date<'a> {
+    type Inner = Date<'a>;
+    #[inline]
+    fn follow(buf: &'a [u8], loc: usize) -> Self::Inner {
+        Self {
+            _tab: flatbuffers::Table { buf: buf, loc: loc },
+        }
+    }
+}
+
+impl<'a> Date<'a> {
+    #[inline]
+    pub fn init_from_table(table: flatbuffers::Table<'a>) -> Self {
+        Date { _tab: table }
+    }
+    #[allow(unused_mut)]
+    pub fn create<'bldr: 'args, 'args: 'mut_bldr, 'mut_bldr>(
+        _fbb: &'mut_bldr mut flatbuffers::FlatBufferBuilder<'bldr>,
+        args: &'args DateArgs,
+    ) -> flatbuffers::WIPOffset<Date<'bldr>> {
+        let mut builder = DateBuilder::new(_fbb);
+        builder.add_unit(args.unit);
+        builder.finish()
+    }
+
+    pub const VT_UNIT: flatbuffers::VOffsetT = 4;
+
+    #[inline]
+    pub fn unit(&self) -> DateUnit {
+        self._tab
+            .get::<DateUnit>(Date::VT_UNIT, Some(DateUnit::MILLISECOND))
+            .unwrap()
+    }
+}
+
+pub struct DateArgs {
+    pub unit: DateUnit,
+}
+impl<'a> Default for DateArgs {
+    #[inline]
+    fn default() -> Self {
+        DateArgs {
+            unit: DateUnit::MILLISECOND,
+        }
+    }
+}
+pub struct DateBuilder<'a: 'b, 'b> {
+    fbb_: &'b mut flatbuffers::FlatBufferBuilder<'a>,
+    start_: flatbuffers::WIPOffset<flatbuffers::TableUnfinishedWIPOffset>,
+}
+impl<'a: 'b, 'b> DateBuilder<'a, 'b> {
+    #[inline]
+    pub fn add_unit(&mut self, unit: DateUnit) {
+        self.fbb_
+            .push_slot::<DateUnit>(Date::VT_UNIT, unit, DateUnit::MILLISECOND);
+    }
+    #[inline]
+    pub fn new(_fbb: &'b mut flatbuffers::FlatBufferBuilder<'a>) -> DateBuilder<'a, 'b> {
+        let start = _fbb.start_table();
+        DateBuilder {
+            fbb_: _fbb,
+            start_: start,
+        }
+    }
+    #[inline]
+    pub fn finish(self) -> flatbuffers::WIPOffset<Date<'a>> {
+        let o = self.fbb_.end_table(self.start_);
+        flatbuffers::WIPOffset::new(o.value())
+    }
+}
+
+/// Time type. The physical storage type depends on the unit
+/// - SECOND and MILLISECOND: 32 bits
+/// - MICROSECOND and NANOSECOND: 64 bits
+pub enum TimeOffset {}
+#[derive(Copy, Clone, Debug, PartialEq)]
+
+pub struct Time<'a> {
+    pub _tab: flatbuffers::Table<'a>,
+}
+
+impl<'a> flatbuffers::Follow<'a> for Time<'a> {
+    type Inner = Time<'a>;
+    #[inline]
+    fn follow(buf: &'a [u8], loc: usize) -> Self::Inner {
+        Self {
+            _tab: flatbuffers::Table { buf: buf, loc: loc },
+        }
+    }
+}
+
+impl<'a> Time<'a> {
+    #[inline]
+    pub fn init_from_table(table: flatbuffers::Table<'a>) -> Self {
+        Time { _tab: table }
+    }
+    #[allow(unused_mut)]
+    pub fn create<'bldr: 'args, 'args: 'mut_bldr, 'mut_bldr>(
+        _fbb: &'mut_bldr mut flatbuffers::FlatBufferBuilder<'bldr>,
+        args: &'args TimeArgs,
+    ) -> flatbuffers::WIPOffset<Time<'bldr>> {
+        let mut builder = TimeBuilder::new(_fbb);
+        builder.add_bitWidth(args.bitWidth);
+        builder.add_unit(args.unit);
+        builder.finish()
+    }
+
+    pub const VT_UNIT: flatbuffers::VOffsetT = 4;
+    pub const VT_BITWIDTH: flatbuffers::VOffsetT = 6;
+
+    #[inline]
+    pub fn unit(&self) -> TimeUnit {
+        self._tab
+            .get::<TimeUnit>(Time::VT_UNIT, Some(TimeUnit::MILLISECOND))
+            .unwrap()
+    }
+    #[inline]
+    pub fn bitWidth(&self) -> i32 {
+        self._tab.get::<i32>(Time::VT_BITWIDTH, Some(32)).unwrap()
+    }
+}
+
+pub struct TimeArgs {
+    pub unit: TimeUnit,
+    pub bitWidth: i32,
+}
+impl<'a> Default for TimeArgs {
+    #[inline]
+    fn default() -> Self {
+        TimeArgs {
+            unit: TimeUnit::MILLISECOND,
+            bitWidth: 32,
+        }
+    }
+}
+pub struct TimeBuilder<'a: 'b, 'b> {
+    fbb_: &'b mut flatbuffers::FlatBufferBuilder<'a>,
+    start_: flatbuffers::WIPOffset<flatbuffers::TableUnfinishedWIPOffset>,
+}
+impl<'a: 'b, 'b> TimeBuilder<'a, 'b> {
+    #[inline]
+    pub fn add_unit(&mut self, unit: TimeUnit) {
+        self.fbb_
+            .push_slot::<TimeUnit>(Time::VT_UNIT, unit, TimeUnit::MILLISECOND);
+    }
+    #[inline]
+    pub fn add_bitWidth(&mut self, bitWidth: i32) {
+        self.fbb_.push_slot::<i32>(Time::VT_BITWIDTH, bitWidth, 32);
+    }
+    #[inline]
+    pub fn new(_fbb: &'b mut flatbuffers::FlatBufferBuilder<'a>) -> TimeBuilder<'a, 'b> {
+        let start = _fbb.start_table();
+        TimeBuilder {
+            fbb_: _fbb,
+            start_: start,
+        }
+    }
+    #[inline]
+    pub fn finish(self) -> flatbuffers::WIPOffset<Time<'a>> {
+        let o = self.fbb_.end_table(self.start_);
+        flatbuffers::WIPOffset::new(o.value())
+    }
+}
+
+/// Time elapsed from the Unix epoch, 00:00:00.000 on 1 January 1970, excluding
+/// leap seconds, as a 64-bit integer. Note that UNIX time does not include
+/// leap seconds.
+///
+/// The Timestamp metadata supports both "time zone naive" and "time zone
+/// aware" timestamps. Read about the timezone attribute for more detail
+pub enum TimestampOffset {}
+#[derive(Copy, Clone, Debug, PartialEq)]
+
+pub struct Timestamp<'a> {
+    pub _tab: flatbuffers::Table<'a>,
+}
+
+impl<'a> flatbuffers::Follow<'a> for Timestamp<'a> {
+    type Inner = Timestamp<'a>;
+    #[inline]
+    fn follow(buf: &'a [u8], loc: usize) -> Self::Inner {
+        Self {
+            _tab: flatbuffers::Table { buf: buf, loc: loc },
+        }
+    }
+}
+
+impl<'a> Timestamp<'a> {
+    #[inline]
+    pub fn init_from_table(table: flatbuffers::Table<'a>) -> Self {
+        Timestamp { _tab: table }
+    }
+    #[allow(unused_mut)]
+    pub fn create<'bldr: 'args, 'args: 'mut_bldr, 'mut_bldr>(
+        _fbb: &'mut_bldr mut flatbuffers::FlatBufferBuilder<'bldr>,
+        args: &'args TimestampArgs<'args>,
+    ) -> flatbuffers::WIPOffset<Timestamp<'bldr>> {
+        let mut builder = TimestampBuilder::new(_fbb);
+        if let Some(x) = args.timezone {
+            builder.add_timezone(x);
+        }
+        builder.add_unit(args.unit);
+        builder.finish()
+    }
+
+    pub const VT_UNIT: flatbuffers::VOffsetT = 4;
+    pub const VT_TIMEZONE: flatbuffers::VOffsetT = 6;
+
+    #[inline]
+    pub fn unit(&self) -> TimeUnit {
+        self._tab
+            .get::<TimeUnit>(Timestamp::VT_UNIT, Some(TimeUnit::SECOND))
+            .unwrap()
+    }
+    /// The time zone is a string indicating the name of a time zone, one of:
+    ///
+    /// * As used in the Olson time zone database (the "tz database" or
+    ///   "tzdata"), such as "America/New_York"
+    /// * An absolute time zone offset of the form +XX:XX or -XX:XX, such as +07:30
+    ///
+    /// Whether a timezone string is present indicates different semantics about
+    /// the data:
+    ///
+    /// * If the time zone is null or equal to an empty string, the data is "time
+    ///   zone naive" and shall be displayed *as is* to the user, not localized
+    ///   to the locale of the user. This data can be though of as UTC but
+    ///   without having "UTC" as the time zone, it is not considered to be
+    ///   localized to any time zone
+    ///
+    /// * If the time zone is set to a valid value, values can be displayed as
+    ///   "localized" to that time zone, even though the underlying 64-bit
+    ///   integers are identical to the same data stored in UTC. Converting
+    ///   between time zones is a metadata-only operation and does not change the
+    ///   underlying values
+    #[inline]
+    pub fn timezone(&self) -> Option<&'a str> {
+        self._tab
+            .get::<flatbuffers::ForwardsUOffset<&str>>(Timestamp::VT_TIMEZONE, None)
+    }
+}
+
+pub struct TimestampArgs<'a> {
+    pub unit: TimeUnit,
+    pub timezone: Option<flatbuffers::WIPOffset<&'a str>>,
+}
+impl<'a> Default for TimestampArgs<'a> {
+    #[inline]
+    fn default() -> Self {
+        TimestampArgs {
+            unit: TimeUnit::SECOND,
+            timezone: None,
+        }
+    }
+}
+pub struct TimestampBuilder<'a: 'b, 'b> {
+    fbb_: &'b mut flatbuffers::FlatBufferBuilder<'a>,
+    start_: flatbuffers::WIPOffset<flatbuffers::TableUnfinishedWIPOffset>,
+}
+impl<'a: 'b, 'b> TimestampBuilder<'a, 'b> {
+    #[inline]
+    pub fn add_unit(&mut self, unit: TimeUnit) {
+        self.fbb_
+            .push_slot::<TimeUnit>(Timestamp::VT_UNIT, unit, TimeUnit::SECOND);
+    }
+    #[inline]
+    pub fn add_timezone(&mut self, timezone: flatbuffers::WIPOffset<&'b str>) {
+        self.fbb_.push_slot_always::<flatbuffers::WIPOffset<_>>(
+            Timestamp::VT_TIMEZONE,
+            timezone,
+        );
+    }
+    #[inline]
+    pub fn new(
+        _fbb: &'b mut flatbuffers::FlatBufferBuilder<'a>,
+    ) -> TimestampBuilder<'a, 'b> {
+        let start = _fbb.start_table();
+        TimestampBuilder {
+            fbb_: _fbb,
+            start_: start,
+        }
+    }
+    #[inline]
+    pub fn finish(self) -> flatbuffers::WIPOffset<Timestamp<'a>> {
+        let o = self.fbb_.end_table(self.start_);
+        flatbuffers::WIPOffset::new(o.value())
+    }
+}
+
+pub enum IntervalOffset {}
+#[derive(Copy, Clone, Debug, PartialEq)]
+
+pub struct Interval<'a> {
+    pub _tab: flatbuffers::Table<'a>,
+}
+
+impl<'a> flatbuffers::Follow<'a> for Interval<'a> {
+    type Inner = Interval<'a>;
+    #[inline]
+    fn follow(buf: &'a [u8], loc: usize) -> Self::Inner {
+        Self {
+            _tab: flatbuffers::Table { buf: buf, loc: loc },
+        }
+    }
+}
+
+impl<'a> Interval<'a> {
+    #[inline]
+    pub fn init_from_table(table: flatbuffers::Table<'a>) -> Self {
+        Interval { _tab: table }
+    }
+    #[allow(unused_mut)]
+    pub fn create<'bldr: 'args, 'args: 'mut_bldr, 'mut_bldr>(
+        _fbb: &'mut_bldr mut flatbuffers::FlatBufferBuilder<'bldr>,
+        args: &'args IntervalArgs,
+    ) -> flatbuffers::WIPOffset<Interval<'bldr>> {
+        let mut builder = IntervalBuilder::new(_fbb);
+        builder.add_unit(args.unit);
+        builder.finish()
+    }
+
+    pub const VT_UNIT: flatbuffers::VOffsetT = 4;
+
+    #[inline]
+    pub fn unit(&self) -> IntervalUnit {
+        self._tab
+            .get::<IntervalUnit>(Interval::VT_UNIT, Some(IntervalUnit::YEAR_MONTH))
+            .unwrap()
+    }
+}
+
+pub struct IntervalArgs {
+    pub unit: IntervalUnit,
+}
+impl<'a> Default for IntervalArgs {
+    #[inline]
+    fn default() -> Self {
+        IntervalArgs {
+            unit: IntervalUnit::YEAR_MONTH,
+        }
+    }
+}
+pub struct IntervalBuilder<'a: 'b, 'b> {
+    fbb_: &'b mut flatbuffers::FlatBufferBuilder<'a>,
+    start_: flatbuffers::WIPOffset<flatbuffers::TableUnfinishedWIPOffset>,
+}
+impl<'a: 'b, 'b> IntervalBuilder<'a, 'b> {
+    #[inline]
+    pub fn add_unit(&mut self, unit: IntervalUnit) {
+        self.fbb_.push_slot::<IntervalUnit>(
+            Interval::VT_UNIT,
+            unit,
+            IntervalUnit::YEAR_MONTH,
+        );
+    }
+    #[inline]
+    pub fn new(
+        _fbb: &'b mut flatbuffers::FlatBufferBuilder<'a>,
+    ) -> IntervalBuilder<'a, 'b> {
+        let start = _fbb.start_table();
+        IntervalBuilder {
+            fbb_: _fbb,
+            start_: start,
+        }
+    }
+    #[inline]
+    pub fn finish(self) -> flatbuffers::WIPOffset<Interval<'a>> {
+        let o = self.fbb_.end_table(self.start_);
+        flatbuffers::WIPOffset::new(o.value())
+    }
+}
+
+/// ----------------------------------------------------------------------
+/// user defined key value pairs to add custom metadata to arrow
+/// key namespacing is the responsibility of the user
+pub enum KeyValueOffset {}
+#[derive(Copy, Clone, Debug, PartialEq)]
+
+pub struct KeyValue<'a> {
+    pub _tab: flatbuffers::Table<'a>,
+}
+
+impl<'a> flatbuffers::Follow<'a> for KeyValue<'a> {
+    type Inner = KeyValue<'a>;
+    #[inline]
+    fn follow(buf: &'a [u8], loc: usize) -> Self::Inner {
+        Self {
+            _tab: flatbuffers::Table { buf: buf, loc: loc },
+        }
+    }
+}
+
+impl<'a> KeyValue<'a> {
+    #[inline]
+    pub fn init_from_table(table: flatbuffers::Table<'a>) -> Self {
+        KeyValue { _tab: table }
+    }
+    #[allow(unused_mut)]
+    pub fn create<'bldr: 'args, 'args: 'mut_bldr, 'mut_bldr>(
+        _fbb: &'mut_bldr mut flatbuffers::FlatBufferBuilder<'bldr>,
+        args: &'args KeyValueArgs<'args>,
+    ) -> flatbuffers::WIPOffset<KeyValue<'bldr>> {
+        let mut builder = KeyValueBuilder::new(_fbb);
+        if let Some(x) = args.value {
+            builder.add_value(x);
+        }
+        if let Some(x) = args.key {
+            builder.add_key(x);
+        }
+        builder.finish()
+    }
+
+    pub const VT_KEY: flatbuffers::VOffsetT = 4;
+    pub const VT_VALUE: flatbuffers::VOffsetT = 6;
+
+    #[inline]
+    pub fn key(&self) -> Option<&'a str> {
+        self._tab
+            .get::<flatbuffers::ForwardsUOffset<&str>>(KeyValue::VT_KEY, None)
+    }
+    #[inline]
+    pub fn value(&self) -> Option<&'a str> {
+        self._tab
+            .get::<flatbuffers::ForwardsUOffset<&str>>(KeyValue::VT_VALUE, None)
+    }
+}
+
+pub struct KeyValueArgs<'a> {
+    pub key: Option<flatbuffers::WIPOffset<&'a str>>,
+    pub value: Option<flatbuffers::WIPOffset<&'a str>>,
+}
+impl<'a> Default for KeyValueArgs<'a> {
+    #[inline]
+    fn default() -> Self {
+        KeyValueArgs {
+            key: None,
+            value: None,
+        }
+    }
+}
+pub struct KeyValueBuilder<'a: 'b, 'b> {
+    fbb_: &'b mut flatbuffers::FlatBufferBuilder<'a>,
+    start_: flatbuffers::WIPOffset<flatbuffers::TableUnfinishedWIPOffset>,
+}
+impl<'a: 'b, 'b> KeyValueBuilder<'a, 'b> {
+    #[inline]
+    pub fn add_key(&mut self, key: flatbuffers::WIPOffset<&'b str>) {
+        self.fbb_
+            .push_slot_always::<flatbuffers::WIPOffset<_>>(KeyValue::VT_KEY, key);
+    }
+    #[inline]
+    pub fn add_value(&mut self, value: flatbuffers::WIPOffset<&'b str>) {
+        self.fbb_
+            .push_slot_always::<flatbuffers::WIPOffset<_>>(KeyValue::VT_VALUE, value);
+    }
+    #[inline]
+    pub fn new(
+        _fbb: &'b mut flatbuffers::FlatBufferBuilder<'a>,
+    ) -> KeyValueBuilder<'a, 'b> {
+        let start = _fbb.start_table();
+        KeyValueBuilder {
+            fbb_: _fbb,
+            start_: start,
+        }
+    }
+    #[inline]
+    pub fn finish(self) -> flatbuffers::WIPOffset<KeyValue<'a>> {
+        let o = self.fbb_.end_table(self.start_);
+        flatbuffers::WIPOffset::new(o.value())
+    }
+}
+
+/// ----------------------------------------------------------------------
+/// Dictionary encoding metadata
+pub enum DictionaryEncodingOffset {}
+#[derive(Copy, Clone, Debug, PartialEq)]
+
+pub struct DictionaryEncoding<'a> {
+    pub _tab: flatbuffers::Table<'a>,
+}
+
+impl<'a> flatbuffers::Follow<'a> for DictionaryEncoding<'a> {
+    type Inner = DictionaryEncoding<'a>;
+    #[inline]
+    fn follow(buf: &'a [u8], loc: usize) -> Self::Inner {
+        Self {
+            _tab: flatbuffers::Table { buf: buf, loc: loc },
+        }
+    }
+}
+
+impl<'a> DictionaryEncoding<'a> {
+    #[inline]
+    pub fn init_from_table(table: flatbuffers::Table<'a>) -> Self {
+        DictionaryEncoding { _tab: table }
+    }
+    #[allow(unused_mut)]
+    pub fn create<'bldr: 'args, 'args: 'mut_bldr, 'mut_bldr>(
+        _fbb: &'mut_bldr mut flatbuffers::FlatBufferBuilder<'bldr>,
+        args: &'args DictionaryEncodingArgs<'args>,
+    ) -> flatbuffers::WIPOffset<DictionaryEncoding<'bldr>> {
+        let mut builder = DictionaryEncodingBuilder::new(_fbb);
+        builder.add_id(args.id);
+        if let Some(x) = args.indexType {
+            builder.add_indexType(x);
+        }
+        builder.add_isOrdered(args.isOrdered);
+        builder.finish()
+    }
+
+    pub const VT_ID: flatbuffers::VOffsetT = 4;
+    pub const VT_INDEXTYPE: flatbuffers::VOffsetT = 6;
+    pub const VT_ISORDERED: flatbuffers::VOffsetT = 8;
+
+    /// The known dictionary id in the application where this data is used. In
+    /// the file or streaming formats, the dictionary ids are found in the
+    /// DictionaryBatch messages
+    #[inline]
+    pub fn id(&self) -> i64 {
+        self._tab
+            .get::<i64>(DictionaryEncoding::VT_ID, Some(0))
+            .unwrap()
+    }
+    /// The dictionary indices are constrained to be positive integers. If this
+    /// field is null, the indices must be signed int32
+    #[inline]
+    pub fn indexType(&self) -> Option<Int<'a>> {
+        self._tab.get::<flatbuffers::ForwardsUOffset<Int<'a>>>(
+            DictionaryEncoding::VT_INDEXTYPE,
+            None,
+        )
+    }
+    /// By default, dictionaries are not ordered, or the order does not have
+    /// semantic meaning. In some statistical, applications, dictionary-encoding
+    /// is used to represent ordered categorical data, and we provide a way to
+    /// preserve that metadata here
+    #[inline]
+    pub fn isOrdered(&self) -> bool {
+        self._tab
+            .get::<bool>(DictionaryEncoding::VT_ISORDERED, Some(false))
+            .unwrap()
+    }
+}
+
+pub struct DictionaryEncodingArgs<'a> {
+    pub id: i64,
+    pub indexType: Option<flatbuffers::WIPOffset<Int<'a>>>,
+    pub isOrdered: bool,
+}
+impl<'a> Default for DictionaryEncodingArgs<'a> {
+    #[inline]
+    fn default() -> Self {
+        DictionaryEncodingArgs {
+            id: 0,
+            indexType: None,
+            isOrdered: false,
+        }
+    }
+}
+pub struct DictionaryEncodingBuilder<'a: 'b, 'b> {
+    fbb_: &'b mut flatbuffers::FlatBufferBuilder<'a>,
+    start_: flatbuffers::WIPOffset<flatbuffers::TableUnfinishedWIPOffset>,
+}
+impl<'a: 'b, 'b> DictionaryEncodingBuilder<'a, 'b> {
+    #[inline]
+    pub fn add_id(&mut self, id: i64) {
+        self.fbb_.push_slot::<i64>(DictionaryEncoding::VT_ID, id, 0);
+    }
+    #[inline]
+    pub fn add_indexType(&mut self, indexType: flatbuffers::WIPOffset<Int<'b>>) {
+        self.fbb_.push_slot_always::<flatbuffers::WIPOffset<Int>>(
+            DictionaryEncoding::VT_INDEXTYPE,
+            indexType,
+        );
+    }
+    #[inline]
+    pub fn add_isOrdered(&mut self, isOrdered: bool) {
+        self.fbb_
+            .push_slot::<bool>(DictionaryEncoding::VT_ISORDERED, isOrdered, false);
+    }
+    #[inline]
+    pub fn new(
+        _fbb: &'b mut flatbuffers::FlatBufferBuilder<'a>,
+    ) -> DictionaryEncodingBuilder<'a, 'b> {
+        let start = _fbb.start_table();
+        DictionaryEncodingBuilder {
+            fbb_: _fbb,
+            start_: start,
+        }
+    }
+    #[inline]
+    pub fn finish(self) -> flatbuffers::WIPOffset<DictionaryEncoding<'a>> {
+        let o = self.fbb_.end_table(self.start_);
+        flatbuffers::WIPOffset::new(o.value())
+    }
+}
+
+/// ----------------------------------------------------------------------
+/// A field represents a named column in a record / row batch or child of a
+/// nested type.
+///
+/// - children is only for nested Arrow arrays
+/// - For primitive types, children will have length 0
+/// - nullable should default to true in general
+pub enum FieldOffset {}
+#[derive(Copy, Clone, Debug, PartialEq)]
+
+pub struct Field<'a> {
+    pub _tab: flatbuffers::Table<'a>,
+}
+
+impl<'a> flatbuffers::Follow<'a> for Field<'a> {
+    type Inner = Field<'a>;
+    #[inline]
+    fn follow(buf: &'a [u8], loc: usize) -> Self::Inner {
+        Self {
+            _tab: flatbuffers::Table { buf: buf, loc: loc },
+        }
+    }
+}
+
+impl<'a> Field<'a> {
+    #[inline]
+    pub fn init_from_table(table: flatbuffers::Table<'a>) -> Self {
+        Field { _tab: table }
+    }
+    #[allow(unused_mut)]
+    pub fn create<'bldr: 'args, 'args: 'mut_bldr, 'mut_bldr>(
+        _fbb: &'mut_bldr mut flatbuffers::FlatBufferBuilder<'bldr>,
+        args: &'args FieldArgs<'args>,
+    ) -> flatbuffers::WIPOffset<Field<'bldr>> {
+        let mut builder = FieldBuilder::new(_fbb);
+        if let Some(x) = args.custom_metadata {
+            builder.add_custom_metadata(x);
+        }
+        if let Some(x) = args.children {
+            builder.add_children(x);
+        }
+        if let Some(x) = args.dictionary {
+            builder.add_dictionary(x);
+        }
+        if let Some(x) = args.type_ {
+            builder.add_type_(x);
+        }
+        if let Some(x) = args.name {
+            builder.add_name(x);
+        }
+        builder.add_type_type(args.type_type);
+        builder.add_nullable(args.nullable);
+        builder.finish()
+    }
+
+    pub const VT_NAME: flatbuffers::VOffsetT = 4;
+    pub const VT_NULLABLE: flatbuffers::VOffsetT = 6;
+    pub const VT_TYPE_TYPE: flatbuffers::VOffsetT = 8;
+    pub const VT_TYPE_: flatbuffers::VOffsetT = 10;
+    pub const VT_DICTIONARY: flatbuffers::VOffsetT = 12;
+    pub const VT_CHILDREN: flatbuffers::VOffsetT = 14;
+    pub const VT_CUSTOM_METADATA: flatbuffers::VOffsetT = 16;
+
+    #[inline]
+    pub fn name(&self) -> Option<&'a str> {
+        self._tab
+            .get::<flatbuffers::ForwardsUOffset<&str>>(Field::VT_NAME, None)
+    }
+    #[inline]
+    pub fn nullable(&self) -> bool {
+        self._tab
+            .get::<bool>(Field::VT_NULLABLE, Some(false))
+            .unwrap()
+    }
+    #[inline]
+    pub fn type_type(&self) -> Type {
+        self._tab
+            .get::<Type>(Field::VT_TYPE_TYPE, Some(Type::NONE))
+            .unwrap()
+    }
+    #[inline]
+    pub fn type_(&self) -> Option<flatbuffers::Table<'a>> {
+        self._tab
+            .get::<flatbuffers::ForwardsUOffset<flatbuffers::Table<'a>>>(
+                Field::VT_TYPE_,
+                None,
+            )
+    }
+    #[inline]
+    pub fn dictionary(&self) -> Option<DictionaryEncoding<'a>> {
+        self._tab
+            .get::<flatbuffers::ForwardsUOffset<DictionaryEncoding<'a>>>(
+                Field::VT_DICTIONARY,
+                None,
+            )
+    }
+    #[inline]
+    pub fn children(
+        &self,
+    ) -> Option<flatbuffers::Vector<flatbuffers::ForwardsUOffset<Field<'a>>>> {
+        self._tab.get::<flatbuffers::ForwardsUOffset<
+            flatbuffers::Vector<flatbuffers::ForwardsUOffset<Field<'a>>>,
+        >>(Field::VT_CHILDREN, None)
+    }
+    #[inline]
+    pub fn custom_metadata(
+        &self,
+    ) -> Option<flatbuffers::Vector<flatbuffers::ForwardsUOffset<KeyValue<'a>>>> {
+        self._tab.get::<flatbuffers::ForwardsUOffset<
+            flatbuffers::Vector<flatbuffers::ForwardsUOffset<KeyValue<'a>>>,
+        >>(Field::VT_CUSTOM_METADATA, None)
+    }
+    #[inline]
+    #[allow(non_snake_case)]
+    pub fn type__as_null(&'a self) -> Option<Null> {
+        if self.type_type() == Type::Null {
+            self.type_().map(|u| Null::init_from_table(u))
+        } else {
+            None
+        }
+    }
+
+    #[inline]
+    #[allow(non_snake_case)]
+    pub fn type__as_int(&'a self) -> Option<Int> {
+        if self.type_type() == Type::Int {
+            self.type_().map(|u| Int::init_from_table(u))
+        } else {
+            None
+        }
+    }
+
+    #[inline]
+    #[allow(non_snake_case)]
+    pub fn type__as_floating_point(&'a self) -> Option<FloatingPoint> {
+        if self.type_type() == Type::FloatingPoint {
+            self.type_().map(|u| FloatingPoint::init_from_table(u))
+        } else {
+            None
+        }
+    }
+
+    #[inline]
+    #[allow(non_snake_case)]
+    pub fn type__as_binary(&'a self) -> Option<Binary> {
+        if self.type_type() == Type::Binary {
+            self.type_().map(|u| Binary::init_from_table(u))
+        } else {
+            None
+        }
+    }
+
+    #[inline]
+    #[allow(non_snake_case)]
+    pub fn type__as_utf_8(&'a self) -> Option<Utf8> {
+        if self.type_type() == Type::Utf8 {
+            self.type_().map(|u| Utf8::init_from_table(u))
+        } else {
+            None
+        }
+    }
+
+    #[inline]
+    #[allow(non_snake_case)]
+    pub fn type__as_bool(&'a self) -> Option<Bool> {
+        if self.type_type() == Type::Bool {
+            self.type_().map(|u| Bool::init_from_table(u))
+        } else {
+            None
+        }
+    }
+
+    #[inline]
+    #[allow(non_snake_case)]
+    pub fn type__as_decimal(&'a self) -> Option<Decimal> {
+        if self.type_type() == Type::Decimal {
+            self.type_().map(|u| Decimal::init_from_table(u))
+        } else {
+            None
+        }
+    }
+
+    #[inline]
+    #[allow(non_snake_case)]
+    pub fn type__as_date(&'a self) -> Option<Date> {
+        if self.type_type() == Type::Date {
+            self.type_().map(|u| Date::init_from_table(u))
+        } else {
+            None
+        }
+    }
+
+    #[inline]
+    #[allow(non_snake_case)]
+    pub fn type__as_time(&'a self) -> Option<Time> {
+        if self.type_type() == Type::Time {
+            self.type_().map(|u| Time::init_from_table(u))
+        } else {
+            None
+        }
+    }
+
+    #[inline]
+    #[allow(non_snake_case)]
+    pub fn type__as_timestamp(&'a self) -> Option<Timestamp> {
+        if self.type_type() == Type::Timestamp {
+            self.type_().map(|u| Timestamp::init_from_table(u))
+        } else {
+            None
+        }
+    }
+
+    #[inline]
+    #[allow(non_snake_case)]
+    pub fn type__as_interval(&'a self) -> Option<Interval> {
+        if self.type_type() == Type::Interval {
+            self.type_().map(|u| Interval::init_from_table(u))
+        } else {
+            None
+        }
+    }
+
+    #[inline]
+    #[allow(non_snake_case)]
+    pub fn type__as_list(&'a self) -> Option<List> {
+        if self.type_type() == Type::List {
+            self.type_().map(|u| List::init_from_table(u))
+        } else {
+            None
+        }
+    }
+
+    #[inline]
+    #[allow(non_snake_case)]
+    pub fn type__as_struct_(&'a self) -> Option<Struct_> {
+        if self.type_type() == Type::Struct_ {
+            self.type_().map(|u| Struct_::init_from_table(u))
+        } else {
+            None
+        }
+    }
+
+    #[inline]
+    #[allow(non_snake_case)]
+    pub fn type__as_union(&'a self) -> Option<Union> {
+        if self.type_type() == Type::Union {
+            self.type_().map(|u| Union::init_from_table(u))
+        } else {
+            None
+        }
+    }
+
+    #[inline]
+    #[allow(non_snake_case)]
+    pub fn type__as_fixed_size_binary(&'a self) -> Option<FixedSizeBinary> {
+        if self.type_type() == Type::FixedSizeBinary {
+            self.type_().map(|u| FixedSizeBinary::init_from_table(u))
+        } else {
+            None
+        }
+    }
+
+    #[inline]
+    #[allow(non_snake_case)]
+    pub fn type__as_fixed_size_list(&'a self) -> Option<FixedSizeList> {
+        if self.type_type() == Type::FixedSizeList {
+            self.type_().map(|u| FixedSizeList::init_from_table(u))
+        } else {
+            None
+        }
+    }
+
+    #[inline]
+    #[allow(non_snake_case)]
+    pub fn type__as_map(&'a self) -> Option<Map> {
+        if self.type_type() == Type::Map {
+            self.type_().map(|u| Map::init_from_table(u))
+        } else {
+            None
+        }
+    }
+}
+
+pub struct FieldArgs<'a> {
+    pub name: Option<flatbuffers::WIPOffset<&'a str>>,
+    pub nullable: bool,
+    pub type_type: Type,
+    pub type_: Option<flatbuffers::WIPOffset<flatbuffers::UnionWIPOffset>>,
+    pub dictionary: Option<flatbuffers::WIPOffset<DictionaryEncoding<'a>>>,
+    pub children: Option<
+        flatbuffers::WIPOffset<
+            flatbuffers::Vector<'a, flatbuffers::ForwardsUOffset<Field<'a>>>,
+        >,
+    >,
+    pub custom_metadata: Option<
+        flatbuffers::WIPOffset<
+            flatbuffers::Vector<'a, flatbuffers::ForwardsUOffset<KeyValue<'a>>>,
+        >,
+    >,
+}
+impl<'a> Default for FieldArgs<'a> {
+    #[inline]
+    fn default() -> Self {
+        FieldArgs {
+            name: None,
+            nullable: false,
+            type_type: Type::NONE,
+            type_: None,
+            dictionary: None,
+            children: None,
+            custom_metadata: None,
+        }
+    }
+}
+pub struct FieldBuilder<'a: 'b, 'b> {
+    fbb_: &'b mut flatbuffers::FlatBufferBuilder<'a>,
+    start_: flatbuffers::WIPOffset<flatbuffers::TableUnfinishedWIPOffset>,
+}
+impl<'a: 'b, 'b> FieldBuilder<'a, 'b> {
+    #[inline]
+    pub fn add_name(&mut self, name: flatbuffers::WIPOffset<&'b str>) {
+        self.fbb_
+            .push_slot_always::<flatbuffers::WIPOffset<_>>(Field::VT_NAME, name);
+    }
+    #[inline]
+    pub fn add_nullable(&mut self, nullable: bool) {
+        self.fbb_
+            .push_slot::<bool>(Field::VT_NULLABLE, nullable, false);
+    }
+    #[inline]
+    pub fn add_type_type(&mut self, type_type: Type) {
+        self.fbb_
+            .push_slot::<Type>(Field::VT_TYPE_TYPE, type_type, Type::NONE);
+    }
+    #[inline]
+    pub fn add_type_(
+        &mut self,
+        type_: flatbuffers::WIPOffset<flatbuffers::UnionWIPOffset>,
+    ) {
+        self.fbb_
+            .push_slot_always::<flatbuffers::WIPOffset<_>>(Field::VT_TYPE_, type_);
+    }
+    #[inline]
+    pub fn add_dictionary(
+        &mut self,
+        dictionary: flatbuffers::WIPOffset<DictionaryEncoding<'b>>,
+    ) {
+        self.fbb_
+            .push_slot_always::<flatbuffers::WIPOffset<DictionaryEncoding>>(
+                Field::VT_DICTIONARY,
+                dictionary,
+            );
+    }
+    #[inline]
+    pub fn add_children(
+        &mut self,
+        children: flatbuffers::WIPOffset<
+            flatbuffers::Vector<'b, flatbuffers::ForwardsUOffset<Field<'b>>>,
+        >,
+    ) {
+        self.fbb_
+            .push_slot_always::<flatbuffers::WIPOffset<_>>(Field::VT_CHILDREN, children);
+    }
+    #[inline]
+    pub fn add_custom_metadata(
+        &mut self,
+        custom_metadata: flatbuffers::WIPOffset<
+            flatbuffers::Vector<'b, flatbuffers::ForwardsUOffset<KeyValue<'b>>>,
+        >,
+    ) {
+        self.fbb_.push_slot_always::<flatbuffers::WIPOffset<_>>(
+            Field::VT_CUSTOM_METADATA,
+            custom_metadata,
+        );
+    }
+    #[inline]
+    pub fn new(_fbb: &'b mut flatbuffers::FlatBufferBuilder<'a>) -> FieldBuilder<'a, 'b> {
+        let start = _fbb.start_table();
+        FieldBuilder {
+            fbb_: _fbb,
+            start_: start,
+        }
+    }
+    #[inline]
+    pub fn finish(self) -> flatbuffers::WIPOffset<Field<'a>> {
+        let o = self.fbb_.end_table(self.start_);
+        flatbuffers::WIPOffset::new(o.value())
+    }
+}
+
+/// ----------------------------------------------------------------------
+/// A Schema describes the columns in a row batch
+pub enum SchemaOffset {}
+#[derive(Copy, Clone, Debug, PartialEq)]
+
+pub struct Schema<'a> {
+    pub _tab: flatbuffers::Table<'a>,
+}
+
+impl<'a> flatbuffers::Follow<'a> for Schema<'a> {
+    type Inner = Schema<'a>;
+    #[inline]
+    fn follow(buf: &'a [u8], loc: usize) -> Self::Inner {
+        Self {
+            _tab: flatbuffers::Table { buf: buf, loc: loc },
+        }
+    }
+}
+
+impl<'a> Schema<'a> {
+    #[inline]
+    pub fn init_from_table(table: flatbuffers::Table<'a>) -> Self {
+        Schema { _tab: table }
+    }
+    #[allow(unused_mut)]
+    pub fn create<'bldr: 'args, 'args: 'mut_bldr, 'mut_bldr>(
+        _fbb: &'mut_bldr mut flatbuffers::FlatBufferBuilder<'bldr>,
+        args: &'args SchemaArgs<'args>,
+    ) -> flatbuffers::WIPOffset<Schema<'bldr>> {
+        let mut builder = SchemaBuilder::new(_fbb);
+        if let Some(x) = args.custom_metadata {
+            builder.add_custom_metadata(x);
+        }
+        if let Some(x) = args.fields {
+            builder.add_fields(x);
+        }
+        builder.add_endianness(args.endianness);
+        builder.finish()
+    }
+
+    pub const VT_ENDIANNESS: flatbuffers::VOffsetT = 4;
+    pub const VT_FIELDS: flatbuffers::VOffsetT = 6;
+    pub const VT_CUSTOM_METADATA: flatbuffers::VOffsetT = 8;
+
+    /// endianness of the buffer
+    /// it is Little Endian by default
+    /// if endianness doesn't match the underlying system then the vectors need to be converted
+    #[inline]
+    pub fn endianness(&self) -> Endianness {
+        self._tab
+            .get::<Endianness>(Schema::VT_ENDIANNESS, Some(Endianness::Little))
+            .unwrap()
+    }
+    #[inline]
+    pub fn fields(
+        &self,
+    ) -> Option<flatbuffers::Vector<flatbuffers::ForwardsUOffset<Field<'a>>>> {
+        self._tab.get::<flatbuffers::ForwardsUOffset<
+            flatbuffers::Vector<flatbuffers::ForwardsUOffset<Field<'a>>>,
+        >>(Schema::VT_FIELDS, None)
+    }
+    #[inline]
+    pub fn custom_metadata(
+        &self,
+    ) -> Option<flatbuffers::Vector<flatbuffers::ForwardsUOffset<KeyValue<'a>>>> {
+        self._tab.get::<flatbuffers::ForwardsUOffset<
+            flatbuffers::Vector<flatbuffers::ForwardsUOffset<KeyValue<'a>>>,
+        >>(Schema::VT_CUSTOM_METADATA, None)
+    }
+}
+
+pub struct SchemaArgs<'a> {
+    pub endianness: Endianness,
+    pub fields: Option<
+        flatbuffers::WIPOffset<
+            flatbuffers::Vector<'a, flatbuffers::ForwardsUOffset<Field<'a>>>,
+        >,
+    >,
+    pub custom_metadata: Option<
+        flatbuffers::WIPOffset<
+            flatbuffers::Vector<'a, flatbuffers::ForwardsUOffset<KeyValue<'a>>>,
+        >,
+    >,
+}
+impl<'a> Default for SchemaArgs<'a> {
+    #[inline]
+    fn default() -> Self {
+        SchemaArgs {
+            endianness: Endianness::Little,
+            fields: None,
+            custom_metadata: None,
+        }
+    }
+}
+pub struct SchemaBuilder<'a: 'b, 'b> {
+    fbb_: &'b mut flatbuffers::FlatBufferBuilder<'a>,
+    start_: flatbuffers::WIPOffset<flatbuffers::TableUnfinishedWIPOffset>,
+}
+impl<'a: 'b, 'b> SchemaBuilder<'a, 'b> {
+    #[inline]
+    pub fn add_endianness(&mut self, endianness: Endianness) {
+        self.fbb_.push_slot::<Endianness>(
+            Schema::VT_ENDIANNESS,
+            endianness,
+            Endianness::Little,
+        );
+    }
+    #[inline]
+    pub fn add_fields(
+        &mut self,
+        fields: flatbuffers::WIPOffset<
+            flatbuffers::Vector<'b, flatbuffers::ForwardsUOffset<Field<'b>>>,
+        >,
+    ) {
+        self.fbb_
+            .push_slot_always::<flatbuffers::WIPOffset<_>>(Schema::VT_FIELDS, fields);
+    }
+    #[inline]
+    pub fn add_custom_metadata(
+        &mut self,
+        custom_metadata: flatbuffers::WIPOffset<
+            flatbuffers::Vector<'b, flatbuffers::ForwardsUOffset<KeyValue<'b>>>,
+        >,
+    ) {
+        self.fbb_.push_slot_always::<flatbuffers::WIPOffset<_>>(
+            Schema::VT_CUSTOM_METADATA,
+            custom_metadata,
+        );
+    }
+    #[inline]
+    pub fn new(
+        _fbb: &'b mut flatbuffers::FlatBufferBuilder<'a>,
+    ) -> SchemaBuilder<'a, 'b> {
+        let start = _fbb.start_table();
+        SchemaBuilder {
+            fbb_: _fbb,
+            start_: start,
+        }
+    }
+    #[inline]
+    pub fn finish(self) -> flatbuffers::WIPOffset<Schema<'a>> {
+        let o = self.fbb_.end_table(self.start_);
+        flatbuffers::WIPOffset::new(o.value())
+    }
+}
+
+#[inline]
+pub fn get_root_as_schema<'a>(buf: &'a [u8]) -> Schema<'a> {
+    flatbuffers::get_root::<Schema<'a>>(buf)
+}
+
+#[inline]
+pub fn get_size_prefixed_root_as_schema<'a>(buf: &'a [u8]) -> Schema<'a> {
+    flatbuffers::get_size_prefixed_root::<Schema<'a>>(buf)
+}
+
+#[inline]
+pub fn finish_schema_buffer<'a, 'b>(
+    fbb: &'b mut flatbuffers::FlatBufferBuilder<'a>,
+    root: flatbuffers::WIPOffset<Schema<'a>>,
+) {
+    fbb.finish(root, None);
+}
+
+#[inline]
+pub fn finish_size_prefixed_schema_buffer<'a, 'b>(
+    fbb: &'b mut flatbuffers::FlatBufferBuilder<'a>,
+    root: flatbuffers::WIPOffset<Schema<'a>>,
+) {
+    fbb.finish_size_prefixed(root, None);
+}
diff --git a/rust/arrow/src/ipc/gen/SparseTensor.rs b/rust/arrow/src/ipc/gen/SparseTensor.rs
new file mode 100644
index 0000000..8202abc
--- /dev/null
+++ b/rust/arrow/src/ipc/gen/SparseTensor.rs
@@ -0,0 +1,760 @@
+// 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.
+
+// automatically generated by the FlatBuffers compiler, do not modify
+
+
+#![allow(dead_code)]
+#![allow(unused_imports)]
+
+use crate::ipc::gen::Schema::*;
+use crate::ipc::gen::Tensor::*;
+
+#[allow(non_camel_case_types)]
+#[repr(u8)]
+#[derive(Clone, Copy, PartialEq, Debug)]
+pub enum SparseTensorIndex {
+    NONE = 0,
+    SparseTensorIndexCOO = 1,
+    SparseMatrixIndexCSR = 2,
+}
+
+const ENUM_MIN_SPARSE_TENSOR_INDEX: u8 = 0;
+const ENUM_MAX_SPARSE_TENSOR_INDEX: u8 = 2;
+
+impl<'a> flatbuffers::Follow<'a> for SparseTensorIndex {
+    type Inner = Self;
+    #[inline]
+    fn follow(buf: &'a [u8], loc: usize) -> Self::Inner {
+        flatbuffers::read_scalar_at::<Self>(buf, loc)
+    }
+}
+
+impl flatbuffers::EndianScalar for SparseTensorIndex {
+    #[inline]
+    fn to_little_endian(self) -> Self {
+        let n = u8::to_le(self as u8);
+        let p = &n as *const u8 as *const SparseTensorIndex;
+        unsafe { *p }
+    }
+    #[inline]
+    fn from_little_endian(self) -> Self {
+        let n = u8::from_le(self as u8);
+        let p = &n as *const u8 as *const SparseTensorIndex;
+        unsafe { *p }
+    }
+}
+
+impl flatbuffers::Push for SparseTensorIndex {
+    type Output = SparseTensorIndex;
+    #[inline]
+    fn push(&self, dst: &mut [u8], _rest: &[u8]) {
+        flatbuffers::emplace_scalar::<SparseTensorIndex>(dst, *self);
+    }
+}
+
+#[allow(non_camel_case_types)]
+const ENUM_VALUES_SPARSE_TENSOR_INDEX: [SparseTensorIndex; 3] = [
+    SparseTensorIndex::NONE,
+    SparseTensorIndex::SparseTensorIndexCOO,
+    SparseTensorIndex::SparseMatrixIndexCSR,
+];
+
+#[allow(non_camel_case_types)]
+const ENUM_NAMES_SPARSE_TENSOR_INDEX: [&'static str; 3] =
+    ["NONE", "SparseTensorIndexCOO", "SparseMatrixIndexCSR"];
+
+pub fn enum_name_sparse_tensor_index(e: SparseTensorIndex) -> &'static str {
+    let index: usize = e as usize;
+    ENUM_NAMES_SPARSE_TENSOR_INDEX[index]
+}
+
+pub struct SparseTensorIndexUnionTableOffset {}
+/// ----------------------------------------------------------------------
+/// EXPERIMENTAL: Data structures for sparse tensors
+/// Coodinate format of sparse tensor index.
+pub enum SparseTensorIndexCOOOffset {}
+#[derive(Copy, Clone, Debug, PartialEq)]
+
+pub struct SparseTensorIndexCOO<'a> {
+    pub _tab: flatbuffers::Table<'a>,
+}
+
+impl<'a> flatbuffers::Follow<'a> for SparseTensorIndexCOO<'a> {
+    type Inner = SparseTensorIndexCOO<'a>;
+    #[inline]
+    fn follow(buf: &'a [u8], loc: usize) -> Self::Inner {
+        Self {
+            _tab: flatbuffers::Table { buf: buf, loc: loc },
+        }
+    }
+}
+
+impl<'a> SparseTensorIndexCOO<'a> {
+    #[inline]
+    pub fn init_from_table(table: flatbuffers::Table<'a>) -> Self {
+        SparseTensorIndexCOO { _tab: table }
+    }
+    #[allow(unused_mut)]
+    pub fn create<'bldr: 'args, 'args: 'mut_bldr, 'mut_bldr>(
+        _fbb: &'mut_bldr mut flatbuffers::FlatBufferBuilder<'bldr>,
+        args: &'args SparseTensorIndexCOOArgs<'args>,
+    ) -> flatbuffers::WIPOffset<SparseTensorIndexCOO<'bldr>> {
+        let mut builder = SparseTensorIndexCOOBuilder::new(_fbb);
+        if let Some(x) = args.indicesBuffer {
+            builder.add_indicesBuffer(x);
+        }
+        builder.finish()
+    }
+
+    pub const VT_INDICESBUFFER: flatbuffers::VOffsetT = 4;
+
+    /// COO's index list are represented as a NxM matrix,
+    /// where N is the number of non-zero values,
+    /// and M is the number of dimensions of a sparse tensor.
+    /// indicesBuffer stores the location and size of this index matrix.
+    /// The type of index value is long, so the stride for the index matrix is unnecessary.
+    ///
+    /// For example, let X be a 2x3x4x5 tensor, and it has the following 6 non-zero values:
+    ///
+    ///   X[0, 1, 2, 0] := 1
+    ///   X[1, 1, 2, 3] := 2
+    ///   X[0, 2, 1, 0] := 3
+    ///   X[0, 1, 3, 0] := 4
+    ///   X[0, 1, 2, 1] := 5
+    ///   X[1, 2, 0, 4] := 6
+    ///
+    /// In COO format, the index matrix of X is the following 4x6 matrix:
+    ///
+    ///   [[0, 0, 0, 0, 1, 1],
+    ///    [1, 1, 1, 2, 1, 2],
+    ///    [2, 2, 3, 1, 2, 0],
+    ///    [0, 1, 0, 0, 3, 4]]
+    ///
+    /// Note that the indices are sorted in lexicographical order.
+    #[inline]
+    pub fn indicesBuffer(&self) -> Option<&'a Buffer> {
+        self._tab
+            .get::<Buffer>(SparseTensorIndexCOO::VT_INDICESBUFFER, None)
+    }
+}
+
+pub struct SparseTensorIndexCOOArgs<'a> {
+    pub indicesBuffer: Option<&'a Buffer>,
+}
+impl<'a> Default for SparseTensorIndexCOOArgs<'a> {
+    #[inline]
+    fn default() -> Self {
+        SparseTensorIndexCOOArgs {
+            indicesBuffer: None,
+        }
+    }
+}
+pub struct SparseTensorIndexCOOBuilder<'a: 'b, 'b> {
+    fbb_: &'b mut flatbuffers::FlatBufferBuilder<'a>,
+    start_: flatbuffers::WIPOffset<flatbuffers::TableUnfinishedWIPOffset>,
+}
+impl<'a: 'b, 'b> SparseTensorIndexCOOBuilder<'a, 'b> {
+    #[inline]
+    pub fn add_indicesBuffer(&mut self, indicesBuffer: &'b Buffer) {
+        self.fbb_.push_slot_always::<&Buffer>(
+            SparseTensorIndexCOO::VT_INDICESBUFFER,
+            indicesBuffer,
+        );
+    }
+    #[inline]
+    pub fn new(
+        _fbb: &'b mut flatbuffers::FlatBufferBuilder<'a>,
+    ) -> SparseTensorIndexCOOBuilder<'a, 'b> {
+        let start = _fbb.start_table();
+        SparseTensorIndexCOOBuilder {
+            fbb_: _fbb,
+            start_: start,
+        }
+    }
+    #[inline]
+    pub fn finish(self) -> flatbuffers::WIPOffset<SparseTensorIndexCOO<'a>> {
+        let o = self.fbb_.end_table(self.start_);
+        flatbuffers::WIPOffset::new(o.value())
+    }
+}
+
+/// Compressed Sparse Row format, that is matrix-specific.
+pub enum SparseMatrixIndexCSROffset {}
+#[derive(Copy, Clone, Debug, PartialEq)]
+
+pub struct SparseMatrixIndexCSR<'a> {
+    pub _tab: flatbuffers::Table<'a>,
+}
+
+impl<'a> flatbuffers::Follow<'a> for SparseMatrixIndexCSR<'a> {
+    type Inner = SparseMatrixIndexCSR<'a>;
+    #[inline]
+    fn follow(buf: &'a [u8], loc: usize) -> Self::Inner {
+        Self {
+            _tab: flatbuffers::Table { buf: buf, loc: loc },
+        }
+    }
+}
+
+impl<'a> SparseMatrixIndexCSR<'a> {
+    #[inline]
+    pub fn init_from_table(table: flatbuffers::Table<'a>) -> Self {
+        SparseMatrixIndexCSR { _tab: table }
+    }
+    #[allow(unused_mut)]
+    pub fn create<'bldr: 'args, 'args: 'mut_bldr, 'mut_bldr>(
+        _fbb: &'mut_bldr mut flatbuffers::FlatBufferBuilder<'bldr>,
+        args: &'args SparseMatrixIndexCSRArgs<'args>,
+    ) -> flatbuffers::WIPOffset<SparseMatrixIndexCSR<'bldr>> {
+        let mut builder = SparseMatrixIndexCSRBuilder::new(_fbb);
+        if let Some(x) = args.indicesBuffer {
+            builder.add_indicesBuffer(x);
+        }
+        if let Some(x) = args.indptrBuffer {
+            builder.add_indptrBuffer(x);
+        }
+        builder.finish()
+    }
+
+    pub const VT_INDPTRBUFFER: flatbuffers::VOffsetT = 4;
+    pub const VT_INDICESBUFFER: flatbuffers::VOffsetT = 6;
+
+    /// indptrBuffer stores the location and size of indptr array that
+    /// represents the range of the rows.
+    /// The i-th row spans from indptr[i] to indptr[i+1] in the data.
+    /// The length of this array is 1 + (the number of rows), and the type
+    /// of index value is long.
+    ///
+    /// For example, let X be the following 6x4 matrix:
+    ///
+    ///   X := [[0, 1, 2, 0],
+    ///         [0, 0, 3, 0],
+    ///         [0, 4, 0, 5],
+    ///         [0, 0, 0, 0],
+    ///         [6, 0, 7, 8],
+    ///         [0, 9, 0, 0]].
+    ///
+    /// The array of non-zero values in X is:
+    ///
+    ///   values(X) = [1, 2, 3, 4, 5, 6, 7, 8, 9].
+    ///
+    /// And the indptr of X is:
+    ///
+    ///   indptr(X) = [0, 2, 3, 5, 5, 8, 10].
+    #[inline]
+    pub fn indptrBuffer(&self) -> Option<&'a Buffer> {
+        self._tab
+            .get::<Buffer>(SparseMatrixIndexCSR::VT_INDPTRBUFFER, None)
+    }
+    /// indicesBuffer stores the location and size of the array that
+    /// contains the column indices of the corresponding non-zero values.
+    /// The type of index value is long.
+    ///
+    /// For example, the indices of the above X is:
+    ///
+    ///   indices(X) = [1, 2, 2, 1, 3, 0, 2, 3, 1].
+    ///
+    /// Note that the indices are sorted in lexicographical order for each row.
+    #[inline]
+    pub fn indicesBuffer(&self) -> Option<&'a Buffer> {
+        self._tab
+            .get::<Buffer>(SparseMatrixIndexCSR::VT_INDICESBUFFER, None)
+    }
+}
+
+pub struct SparseMatrixIndexCSRArgs<'a> {
+    pub indptrBuffer: Option<&'a Buffer>,
+    pub indicesBuffer: Option<&'a Buffer>,
+}
+impl<'a> Default for SparseMatrixIndexCSRArgs<'a> {
+    #[inline]
+    fn default() -> Self {
+        SparseMatrixIndexCSRArgs {
+            indptrBuffer: None,
+            indicesBuffer: None,
+        }
+    }
+}
+pub struct SparseMatrixIndexCSRBuilder<'a: 'b, 'b> {
+    fbb_: &'b mut flatbuffers::FlatBufferBuilder<'a>,
+    start_: flatbuffers::WIPOffset<flatbuffers::TableUnfinishedWIPOffset>,
+}
+impl<'a: 'b, 'b> SparseMatrixIndexCSRBuilder<'a, 'b> {
+    #[inline]
+    pub fn add_indptrBuffer(&mut self, indptrBuffer: &'b Buffer) {
+        self.fbb_.push_slot_always::<&Buffer>(
+            SparseMatrixIndexCSR::VT_INDPTRBUFFER,
+            indptrBuffer,
+        );
+    }
+    #[inline]
+    pub fn add_indicesBuffer(&mut self, indicesBuffer: &'b Buffer) {
+        self.fbb_.push_slot_always::<&Buffer>(
+            SparseMatrixIndexCSR::VT_INDICESBUFFER,
+            indicesBuffer,
+        );
+    }
+    #[inline]
+    pub fn new(
+        _fbb: &'b mut flatbuffers::FlatBufferBuilder<'a>,
+    ) -> SparseMatrixIndexCSRBuilder<'a, 'b> {
+        let start = _fbb.start_table();
+        SparseMatrixIndexCSRBuilder {
+            fbb_: _fbb,
+            start_: start,
+        }
+    }
+    #[inline]
+    pub fn finish(self) -> flatbuffers::WIPOffset<SparseMatrixIndexCSR<'a>> {
+        let o = self.fbb_.end_table(self.start_);
+        flatbuffers::WIPOffset::new(o.value())
+    }
+}
+
+pub enum SparseTensorOffset {}
+#[derive(Copy, Clone, Debug, PartialEq)]
+
+pub struct SparseTensor<'a> {
+    pub _tab: flatbuffers::Table<'a>,
+}
+
+impl<'a> flatbuffers::Follow<'a> for SparseTensor<'a> {
+    type Inner = SparseTensor<'a>;
+    #[inline]
+    fn follow(buf: &'a [u8], loc: usize) -> Self::Inner {
+        Self {
+            _tab: flatbuffers::Table { buf: buf, loc: loc },
+        }
+    }
+}
+
+impl<'a> SparseTensor<'a> {
+    #[inline]
+    pub fn init_from_table(table: flatbuffers::Table<'a>) -> Self {
+        SparseTensor { _tab: table }
+    }
+    #[allow(unused_mut)]
+    pub fn create<'bldr: 'args, 'args: 'mut_bldr, 'mut_bldr>(
+        _fbb: &'mut_bldr mut flatbuffers::FlatBufferBuilder<'bldr>,
+        args: &'args SparseTensorArgs<'args>,
+    ) -> flatbuffers::WIPOffset<SparseTensor<'bldr>> {
+        let mut builder = SparseTensorBuilder::new(_fbb);
+        builder.add_non_zero_length(args.non_zero_length);
+        if let Some(x) = args.data {
+            builder.add_data(x);
+        }
+        if let Some(x) = args.sparseIndex {
+            builder.add_sparseIndex(x);
+        }
+        if let Some(x) = args.shape {
+            builder.add_shape(x);
+        }
+        if let Some(x) = args.type_ {
+            builder.add_type_(x);
+        }
+        builder.add_sparseIndex_type(args.sparseIndex_type);
+        builder.add_type_type(args.type_type);
+        builder.finish()
+    }
+
+    pub const VT_TYPE_TYPE: flatbuffers::VOffsetT = 4;
+    pub const VT_TYPE_: flatbuffers::VOffsetT = 6;
+    pub const VT_SHAPE: flatbuffers::VOffsetT = 8;
+    pub const VT_NON_ZERO_LENGTH: flatbuffers::VOffsetT = 10;
+    pub const VT_SPARSEINDEX_TYPE: flatbuffers::VOffsetT = 12;
+    pub const VT_SPARSEINDEX: flatbuffers::VOffsetT = 14;
+    pub const VT_DATA: flatbuffers::VOffsetT = 16;
+
+    #[inline]
+    pub fn type_type(&self) -> Type {
+        self._tab
+            .get::<Type>(SparseTensor::VT_TYPE_TYPE, Some(Type::NONE))
+            .unwrap()
+    }
+    /// The type of data contained in a value cell.
+    /// Currently only fixed-width value types are supported,
+    /// no strings or nested types.
+    #[inline]
+    pub fn type_(&self) -> Option<flatbuffers::Table<'a>> {
+        self._tab
+            .get::<flatbuffers::ForwardsUOffset<flatbuffers::Table<'a>>>(
+                SparseTensor::VT_TYPE_,
+                None,
+            )
+    }
+    /// The dimensions of the tensor, optionally named.
+    #[inline]
+    pub fn shape(
+        &self,
+    ) -> Option<flatbuffers::Vector<flatbuffers::ForwardsUOffset<TensorDim<'a>>>> {
+        self._tab.get::<flatbuffers::ForwardsUOffset<
+            flatbuffers::Vector<flatbuffers::ForwardsUOffset<TensorDim<'a>>>,
+        >>(SparseTensor::VT_SHAPE, None)
+    }
+    /// The number of non-zero values in a sparse tensor.
+    #[inline]
+    pub fn non_zero_length(&self) -> i64 {
+        self._tab
+            .get::<i64>(SparseTensor::VT_NON_ZERO_LENGTH, Some(0))
+            .unwrap()
+    }
+    #[inline]
+    pub fn sparseIndex_type(&self) -> SparseTensorIndex {
+        self._tab
+            .get::<SparseTensorIndex>(
+                SparseTensor::VT_SPARSEINDEX_TYPE,
+                Some(SparseTensorIndex::NONE),
+            )
+            .unwrap()
+    }
+    /// Sparse tensor index
+    #[inline]
+    pub fn sparseIndex(&self) -> Option<flatbuffers::Table<'a>> {
+        self._tab
+            .get::<flatbuffers::ForwardsUOffset<flatbuffers::Table<'a>>>(
+                SparseTensor::VT_SPARSEINDEX,
+                None,
+            )
+    }
+    /// The location and size of the tensor's data
+    #[inline]
+    pub fn data(&self) -> Option<&'a Buffer> {
+        self._tab.get::<Buffer>(SparseTensor::VT_DATA, None)
+    }
+    #[inline]
+    #[allow(non_snake_case)]
+    pub fn type__as_null(&'a self) -> Option<Null> {
+        if self.type_type() == Type::Null {
+            self.type_().map(|u| Null::init_from_table(u))
+        } else {
+            None
+        }
+    }
+
+    #[inline]
+    #[allow(non_snake_case)]
+    pub fn type__as_int(&'a self) -> Option<Int> {
+        if self.type_type() == Type::Int {
+            self.type_().map(|u| Int::init_from_table(u))
+        } else {
+            None
+        }
+    }
+
+    #[inline]
+    #[allow(non_snake_case)]
+    pub fn type__as_floating_point(&'a self) -> Option<FloatingPoint> {
+        if self.type_type() == Type::FloatingPoint {
+            self.type_().map(|u| FloatingPoint::init_from_table(u))
+        } else {
+            None
+        }
+    }
+
+    #[inline]
+    #[allow(non_snake_case)]
+    pub fn type__as_binary(&'a self) -> Option<Binary> {
+        if self.type_type() == Type::Binary {
+            self.type_().map(|u| Binary::init_from_table(u))
+        } else {
+            None
+        }
+    }
+
+    #[inline]
+    #[allow(non_snake_case)]
+    pub fn type__as_utf_8(&'a self) -> Option<Utf8> {
+        if self.type_type() == Type::Utf8 {
+            self.type_().map(|u| Utf8::init_from_table(u))
+        } else {
+            None
+        }
+    }
+
+    #[inline]
+    #[allow(non_snake_case)]
+    pub fn type__as_bool(&'a self) -> Option<Bool> {
+        if self.type_type() == Type::Bool {
+            self.type_().map(|u| Bool::init_from_table(u))
+        } else {
+            None
+        }
+    }
+
+    #[inline]
+    #[allow(non_snake_case)]
+    pub fn type__as_decimal(&'a self) -> Option<Decimal> {
+        if self.type_type() == Type::Decimal {
+            self.type_().map(|u| Decimal::init_from_table(u))
+        } else {
+            None
+        }
+    }
+
+    #[inline]
+    #[allow(non_snake_case)]
+    pub fn type__as_date(&'a self) -> Option<Date> {
+        if self.type_type() == Type::Date {
+            self.type_().map(|u| Date::init_from_table(u))
+        } else {
+            None
+        }
+    }
+
+    #[inline]
+    #[allow(non_snake_case)]
+    pub fn type__as_time(&'a self) -> Option<Time> {
+        if self.type_type() == Type::Time {
+            self.type_().map(|u| Time::init_from_table(u))
+        } else {
+            None
+        }
+    }
+
+    #[inline]
+    #[allow(non_snake_case)]
+    pub fn type__as_timestamp(&'a self) -> Option<Timestamp> {
+        if self.type_type() == Type::Timestamp {
+            self.type_().map(|u| Timestamp::init_from_table(u))
+        } else {
+            None
+        }
+    }
+
+    #[inline]
+    #[allow(non_snake_case)]
+    pub fn type__as_interval(&'a self) -> Option<Interval> {
+        if self.type_type() == Type::Interval {
+            self.type_().map(|u| Interval::init_from_table(u))
+        } else {
+            None
+        }
+    }
+
+    #[inline]
+    #[allow(non_snake_case)]
+    pub fn type__as_list(&'a self) -> Option<List> {
+        if self.type_type() == Type::List {
+            self.type_().map(|u| List::init_from_table(u))
+        } else {
+            None
+        }
+    }
+
+    #[inline]
+    #[allow(non_snake_case)]
+    pub fn type__as_struct_(&'a self) -> Option<Struct_> {
+        if self.type_type() == Type::Struct_ {
+            self.type_().map(|u| Struct_::init_from_table(u))
+        } else {
+            None
+        }
+    }
+
+    #[inline]
+    #[allow(non_snake_case)]
+    pub fn type__as_union(&'a self) -> Option<Union> {
+        if self.type_type() == Type::Union {
+            self.type_().map(|u| Union::init_from_table(u))
+        } else {
+            None
+        }
+    }
+
+    #[inline]
+    #[allow(non_snake_case)]
+    pub fn type__as_fixed_size_binary(&'a self) -> Option<FixedSizeBinary> {
+        if self.type_type() == Type::FixedSizeBinary {
+            self.type_().map(|u| FixedSizeBinary::init_from_table(u))
+        } else {
+            None
+        }
+    }
+
+    #[inline]
+    #[allow(non_snake_case)]
+    pub fn type__as_fixed_size_list(&'a self) -> Option<FixedSizeList> {
+        if self.type_type() == Type::FixedSizeList {
+            self.type_().map(|u| FixedSizeList::init_from_table(u))
+        } else {
+            None
+        }
+    }
+
+    #[inline]
+    #[allow(non_snake_case)]
+    pub fn type__as_map(&'a self) -> Option<Map> {
+        if self.type_type() == Type::Map {
+            self.type_().map(|u| Map::init_from_table(u))
+        } else {
+            None
+        }
+    }
+
+    #[inline]
+    #[allow(non_snake_case)]
+    pub fn sparseIndex_as_sparse_tensor_index_coo(
+        &'a self,
+    ) -> Option<SparseTensorIndexCOO> {
+        if self.sparseIndex_type() == SparseTensorIndex::SparseTensorIndexCOO {
+            self.sparseIndex()
+                .map(|u| SparseTensorIndexCOO::init_from_table(u))
+        } else {
+            None
+        }
+    }
+
+    #[inline]
+    #[allow(non_snake_case)]
+    pub fn sparseIndex_as_sparse_matrix_index_csr(
+        &'a self,
+    ) -> Option<SparseMatrixIndexCSR> {
+        if self.sparseIndex_type() == SparseTensorIndex::SparseMatrixIndexCSR {
+            self.sparseIndex()
+                .map(|u| SparseMatrixIndexCSR::init_from_table(u))
+        } else {
+            None
+        }
+    }
+}
+
+pub struct SparseTensorArgs<'a> {
+    pub type_type: Type,
+    pub type_: Option<flatbuffers::WIPOffset<flatbuffers::UnionWIPOffset>>,
+    pub shape: Option<
+        flatbuffers::WIPOffset<
+            flatbuffers::Vector<'a, flatbuffers::ForwardsUOffset<TensorDim<'a>>>,
+        >,
+    >,
+    pub non_zero_length: i64,
+    pub sparseIndex_type: SparseTensorIndex,
+    pub sparseIndex: Option<flatbuffers::WIPOffset<flatbuffers::UnionWIPOffset>>,
+    pub data: Option<&'a Buffer>,
+}
+impl<'a> Default for SparseTensorArgs<'a> {
+    #[inline]
+    fn default() -> Self {
+        SparseTensorArgs {
+            type_type: Type::NONE,
+            type_: None,
+            shape: None,
+            non_zero_length: 0,
+            sparseIndex_type: SparseTensorIndex::NONE,
+            sparseIndex: None,
+            data: None,
+        }
+    }
+}
+pub struct SparseTensorBuilder<'a: 'b, 'b> {
+    fbb_: &'b mut flatbuffers::FlatBufferBuilder<'a>,
+    start_: flatbuffers::WIPOffset<flatbuffers::TableUnfinishedWIPOffset>,
+}
+impl<'a: 'b, 'b> SparseTensorBuilder<'a, 'b> {
+    #[inline]
+    pub fn add_type_type(&mut self, type_type: Type) {
+        self.fbb_
+            .push_slot::<Type>(SparseTensor::VT_TYPE_TYPE, type_type, Type::NONE);
+    }
+    #[inline]
+    pub fn add_type_(
+        &mut self,
+        type_: flatbuffers::WIPOffset<flatbuffers::UnionWIPOffset>,
+    ) {
+        self.fbb_
+            .push_slot_always::<flatbuffers::WIPOffset<_>>(SparseTensor::VT_TYPE_, type_);
+    }
+    #[inline]
+    pub fn add_shape(
+        &mut self,
+        shape: flatbuffers::WIPOffset<
+            flatbuffers::Vector<'b, flatbuffers::ForwardsUOffset<TensorDim<'b>>>,
+        >,
+    ) {
+        self.fbb_
+            .push_slot_always::<flatbuffers::WIPOffset<_>>(SparseTensor::VT_SHAPE, shape);
+    }
+    #[inline]
+    pub fn add_non_zero_length(&mut self, non_zero_length: i64) {
+        self.fbb_
+            .push_slot::<i64>(SparseTensor::VT_NON_ZERO_LENGTH, non_zero_length, 0);
+    }
+    #[inline]
+    pub fn add_sparseIndex_type(&mut self, sparseIndex_type: SparseTensorIndex) {
+        self.fbb_.push_slot::<SparseTensorIndex>(
+            SparseTensor::VT_SPARSEINDEX_TYPE,
+            sparseIndex_type,
+            SparseTensorIndex::NONE,
+        );
+    }
+    #[inline]
+    pub fn add_sparseIndex(
+        &mut self,
+        sparseIndex: flatbuffers::WIPOffset<flatbuffers::UnionWIPOffset>,
+    ) {
+        self.fbb_.push_slot_always::<flatbuffers::WIPOffset<_>>(
+            SparseTensor::VT_SPARSEINDEX,
+            sparseIndex,
+        );
+    }
+    #[inline]
+    pub fn add_data(&mut self, data: &'b Buffer) {
+        self.fbb_
+            .push_slot_always::<&Buffer>(SparseTensor::VT_DATA, data);
+    }
+    #[inline]
+    pub fn new(
+        _fbb: &'b mut flatbuffers::FlatBufferBuilder<'a>,
+    ) -> SparseTensorBuilder<'a, 'b> {
+        let start = _fbb.start_table();
+        SparseTensorBuilder {
+            fbb_: _fbb,
+            start_: start,
+        }
+    }
+    #[inline]
+    pub fn finish(self) -> flatbuffers::WIPOffset<SparseTensor<'a>> {
+        let o = self.fbb_.end_table(self.start_);
+        flatbuffers::WIPOffset::new(o.value())
+    }
+}
+
+#[inline]
+pub fn get_root_as_sparse_tensor<'a>(buf: &'a [u8]) -> SparseTensor<'a> {
+    flatbuffers::get_root::<SparseTensor<'a>>(buf)
+}
+
+#[inline]
+pub fn get_size_prefixed_root_as_sparse_tensor<'a>(buf: &'a [u8]) -> SparseTensor<'a> {
+    flatbuffers::get_size_prefixed_root::<SparseTensor<'a>>(buf)
+}
+
+#[inline]
+pub fn finish_sparse_tensor_buffer<'a, 'b>(
+    fbb: &'b mut flatbuffers::FlatBufferBuilder<'a>,
+    root: flatbuffers::WIPOffset<SparseTensor<'a>>,
+) {
+    fbb.finish(root, None);
+}
+
+#[inline]
+pub fn finish_size_prefixed_sparse_tensor_buffer<'a, 'b>(
+    fbb: &'b mut flatbuffers::FlatBufferBuilder<'a>,
+    root: flatbuffers::WIPOffset<SparseTensor<'a>>,
+) {
+    fbb.finish_size_prefixed(root, None);
+}
diff --git a/rust/arrow/src/ipc/gen/Tensor.rs b/rust/arrow/src/ipc/gen/Tensor.rs
new file mode 100644
index 0000000..ca0be1e
--- /dev/null
+++ b/rust/arrow/src/ipc/gen/Tensor.rs
@@ -0,0 +1,494 @@
+// 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.
+
+// automatically generated by the FlatBuffers compiler, do not modify
+
+
+#![allow(dead_code)]
+#![allow(unused_imports)]
+
+
+use crate::ipc::gen::Schema::*;
+
+use std::cmp::Ordering;
+use std::mem;
+
+use flatbuffers::EndianScalar;
+
+/// ----------------------------------------------------------------------
+/// Data structures for dense tensors
+/// Shape data for a single axis in a tensor
+pub enum TensorDimOffset {}
+#[derive(Copy, Clone, Debug, PartialEq)]
+
+pub struct TensorDim<'a> {
+    pub _tab: flatbuffers::Table<'a>,
+}
+
+impl<'a> flatbuffers::Follow<'a> for TensorDim<'a> {
+    type Inner = TensorDim<'a>;
+    #[inline]
+    fn follow(buf: &'a [u8], loc: usize) -> Self::Inner {
+        Self {
+            _tab: flatbuffers::Table { buf: buf, loc: loc },
+        }
+    }
+}
+
+impl<'a> TensorDim<'a> {
+    #[inline]
+    pub fn init_from_table(table: flatbuffers::Table<'a>) -> Self {
+        TensorDim { _tab: table }
+    }
+    #[allow(unused_mut)]
+    pub fn create<'bldr: 'args, 'args: 'mut_bldr, 'mut_bldr>(
+        _fbb: &'mut_bldr mut flatbuffers::FlatBufferBuilder<'bldr>,
+        args: &'args TensorDimArgs<'args>,
+    ) -> flatbuffers::WIPOffset<TensorDim<'bldr>> {
+        let mut builder = TensorDimBuilder::new(_fbb);
+        builder.add_size_(args.size_);
+        if let Some(x) = args.name {
+            builder.add_name(x);
+        }
+        builder.finish()
+    }
+
+    pub const VT_SIZE_: flatbuffers::VOffsetT = 4;
+    pub const VT_NAME: flatbuffers::VOffsetT = 6;
+
+    /// Length of dimension
+    #[inline]
+    pub fn size_(&self) -> i64 {
+        self._tab.get::<i64>(TensorDim::VT_SIZE_, Some(0)).unwrap()
+    }
+    /// Name of the dimension, optional
+    #[inline]
+    pub fn name(&self) -> Option<&'a str> {
+        self._tab
+            .get::<flatbuffers::ForwardsUOffset<&str>>(TensorDim::VT_NAME, None)
+    }
+}
+
+pub struct TensorDimArgs<'a> {
+    pub size_: i64,
+    pub name: Option<flatbuffers::WIPOffset<&'a str>>,
+}
+impl<'a> Default for TensorDimArgs<'a> {
+    #[inline]
+    fn default() -> Self {
+        TensorDimArgs {
+            size_: 0,
+            name: None,
+        }
+    }
+}
+pub struct TensorDimBuilder<'a: 'b, 'b> {
+    fbb_: &'b mut flatbuffers::FlatBufferBuilder<'a>,
+    start_: flatbuffers::WIPOffset<flatbuffers::TableUnfinishedWIPOffset>,
+}
+impl<'a: 'b, 'b> TensorDimBuilder<'a, 'b> {
+    #[inline]
+    pub fn add_size_(&mut self, size_: i64) {
+        self.fbb_.push_slot::<i64>(TensorDim::VT_SIZE_, size_, 0);
+    }
+    #[inline]
+    pub fn add_name(&mut self, name: flatbuffers::WIPOffset<&'b str>) {
+        self.fbb_
+            .push_slot_always::<flatbuffers::WIPOffset<_>>(TensorDim::VT_NAME, name);
+    }
+    #[inline]
+    pub fn new(
+        _fbb: &'b mut flatbuffers::FlatBufferBuilder<'a>,
+    ) -> TensorDimBuilder<'a, 'b> {
+        let start = _fbb.start_table();
+        TensorDimBuilder {
+            fbb_: _fbb,
+            start_: start,
+        }
+    }
+    #[inline]
+    pub fn finish(self) -> flatbuffers::WIPOffset<TensorDim<'a>> {
+        let o = self.fbb_.end_table(self.start_);
+        flatbuffers::WIPOffset::new(o.value())
+    }
+}
+
+pub enum TensorOffset {}
+#[derive(Copy, Clone, Debug, PartialEq)]
+
+pub struct Tensor<'a> {
+    pub _tab: flatbuffers::Table<'a>,
+}
+
+impl<'a> flatbuffers::Follow<'a> for Tensor<'a> {
+    type Inner = Tensor<'a>;
+    #[inline]
+    fn follow(buf: &'a [u8], loc: usize) -> Self::Inner {
+        Self {
+            _tab: flatbuffers::Table { buf: buf, loc: loc },
+        }
+    }
+}
+
+impl<'a> Tensor<'a> {
+    #[inline]
+    pub fn init_from_table(table: flatbuffers::Table<'a>) -> Self {
+        Tensor { _tab: table }
+    }
+    #[allow(unused_mut)]
+    pub fn create<'bldr: 'args, 'args: 'mut_bldr, 'mut_bldr>(
+        _fbb: &'mut_bldr mut flatbuffers::FlatBufferBuilder<'bldr>,
+        args: &'args TensorArgs<'args>,
+    ) -> flatbuffers::WIPOffset<Tensor<'bldr>> {
+        let mut builder = TensorBuilder::new(_fbb);
+        if let Some(x) = args.data {
+            builder.add_data(x);
+        }
+        if let Some(x) = args.strides {
+            builder.add_strides(x);
+        }
+        if let Some(x) = args.shape {
+            builder.add_shape(x);
+        }
+        if let Some(x) = args.type_ {
+            builder.add_type_(x);
+        }
+        builder.add_type_type(args.type_type);
+        builder.finish()
+    }
+
+    pub const VT_TYPE_TYPE: flatbuffers::VOffsetT = 4;
+    pub const VT_TYPE_: flatbuffers::VOffsetT = 6;
+    pub const VT_SHAPE: flatbuffers::VOffsetT = 8;
+    pub const VT_STRIDES: flatbuffers::VOffsetT = 10;
+    pub const VT_DATA: flatbuffers::VOffsetT = 12;
+
+    #[inline]
+    pub fn type_type(&self) -> Type {
+        self._tab
+            .get::<Type>(Tensor::VT_TYPE_TYPE, Some(Type::NONE))
+            .unwrap()
+    }
+    /// The type of data contained in a value cell. Currently only fixed-width
+    /// value types are supported, no strings or nested types
+    #[inline]
+    pub fn type_(&self) -> Option<flatbuffers::Table<'a>> {
+        self._tab
+            .get::<flatbuffers::ForwardsUOffset<flatbuffers::Table<'a>>>(
+                Tensor::VT_TYPE_,
+                None,
+            )
+    }
+    /// The dimensions of the tensor, optionally named
+    #[inline]
+    pub fn shape(
+        &self,
+    ) -> Option<flatbuffers::Vector<flatbuffers::ForwardsUOffset<TensorDim<'a>>>> {
+        self._tab.get::<flatbuffers::ForwardsUOffset<
+            flatbuffers::Vector<flatbuffers::ForwardsUOffset<TensorDim<'a>>>,
+        >>(Tensor::VT_SHAPE, None)
+    }
+    /// Non-negative byte offsets to advance one value cell along each dimension
+    #[inline]
+    pub fn strides(&self) -> Option<flatbuffers::Vector<'a, i64>> {
+        self._tab
+            .get::<flatbuffers::ForwardsUOffset<flatbuffers::Vector<'a, i64>>>(
+                Tensor::VT_STRIDES,
+                None,
+            )
+    }
+    /// The location and size of the tensor's data
+    #[inline]
+    pub fn data(&self) -> Option<&'a Buffer> {
+        self._tab.get::<Buffer>(Tensor::VT_DATA, None)
+    }
+    #[inline]
+    #[allow(non_snake_case)]
+    pub fn type__as_null(&'a self) -> Option<Null> {
+        if self.type_type() == Type::Null {
+            self.type_().map(|u| Null::init_from_table(u))
+        } else {
+            None
+        }
+    }
+
+    #[inline]
+    #[allow(non_snake_case)]
+    pub fn type__as_int(&'a self) -> Option<Int> {
+        if self.type_type() == Type::Int {
+            self.type_().map(|u| Int::init_from_table(u))
+        } else {
+            None
+        }
+    }
+
+    #[inline]
+    #[allow(non_snake_case)]
+    pub fn type__as_floating_point(&'a self) -> Option<FloatingPoint> {
+        if self.type_type() == Type::FloatingPoint {
+            self.type_().map(|u| FloatingPoint::init_from_table(u))
+        } else {
+            None
+        }
+    }
+
+    #[inline]
+    #[allow(non_snake_case)]
+    pub fn type__as_binary(&'a self) -> Option<Binary> {
+        if self.type_type() == Type::Binary {
+            self.type_().map(|u| Binary::init_from_table(u))
+        } else {
+            None
+        }
+    }
+
+    #[inline]
+    #[allow(non_snake_case)]
+    pub fn type__as_utf_8(&'a self) -> Option<Utf8> {
+        if self.type_type() == Type::Utf8 {
+            self.type_().map(|u| Utf8::init_from_table(u))
+        } else {
+            None
+        }
+    }
+
+    #[inline]
+    #[allow(non_snake_case)]
+    pub fn type__as_bool(&'a self) -> Option<Bool> {
+        if self.type_type() == Type::Bool {
+            self.type_().map(|u| Bool::init_from_table(u))
+        } else {
+            None
+        }
+    }
+
+    #[inline]
+    #[allow(non_snake_case)]
+    pub fn type__as_decimal(&'a self) -> Option<Decimal> {
+        if self.type_type() == Type::Decimal {
+            self.type_().map(|u| Decimal::init_from_table(u))
+        } else {
+            None
+        }
+    }
+
+    #[inline]
+    #[allow(non_snake_case)]
+    pub fn type__as_date(&'a self) -> Option<Date> {
+        if self.type_type() == Type::Date {
+            self.type_().map(|u| Date::init_from_table(u))
+        } else {
+            None
+        }
+    }
+
+    #[inline]
+    #[allow(non_snake_case)]
+    pub fn type__as_time(&'a self) -> Option<Time> {
+        if self.type_type() == Type::Time {
+            self.type_().map(|u| Time::init_from_table(u))
+        } else {
+            None
+        }
+    }
+
+    #[inline]
+    #[allow(non_snake_case)]
+    pub fn type__as_timestamp(&'a self) -> Option<Timestamp> {
+        if self.type_type() == Type::Timestamp {
+            self.type_().map(|u| Timestamp::init_from_table(u))
+        } else {
+            None
+        }
+    }
+
+    #[inline]
+    #[allow(non_snake_case)]
+    pub fn type__as_interval(&'a self) -> Option<Interval> {
+        if self.type_type() == Type::Interval {
+            self.type_().map(|u| Interval::init_from_table(u))
+        } else {
+            None
+        }
+    }
+
+    #[inline]
+    #[allow(non_snake_case)]
+    pub fn type__as_list(&'a self) -> Option<List> {
+        if self.type_type() == Type::List {
+            self.type_().map(|u| List::init_from_table(u))
+        } else {
+            None
+        }
+    }
+
+    #[inline]
+    #[allow(non_snake_case)]
+    pub fn type__as_struct_(&'a self) -> Option<Struct_> {
+        if self.type_type() == Type::Struct_ {
+            self.type_().map(|u| Struct_::init_from_table(u))
+        } else {
+            None
+        }
+    }
+
+    #[inline]
+    #[allow(non_snake_case)]
+    pub fn type__as_union(&'a self) -> Option<Union> {
+        if self.type_type() == Type::Union {
+            self.type_().map(|u| Union::init_from_table(u))
+        } else {
+            None
+        }
+    }
+
+    #[inline]
+    #[allow(non_snake_case)]
+    pub fn type__as_fixed_size_binary(&'a self) -> Option<FixedSizeBinary> {
+        if self.type_type() == Type::FixedSizeBinary {
+            self.type_().map(|u| FixedSizeBinary::init_from_table(u))
+        } else {
+            None
+        }
+    }
+
+    #[inline]
+    #[allow(non_snake_case)]
+    pub fn type__as_fixed_size_list(&'a self) -> Option<FixedSizeList> {
+        if self.type_type() == Type::FixedSizeList {
+            self.type_().map(|u| FixedSizeList::init_from_table(u))
+        } else {
+            None
+        }
+    }
+
+    #[inline]
+    #[allow(non_snake_case)]
+    pub fn type__as_map(&'a self) -> Option<Map> {
+        if self.type_type() == Type::Map {
+            self.type_().map(|u| Map::init_from_table(u))
+        } else {
+            None
+        }
+    }
+}
+
+pub struct TensorArgs<'a> {
+    pub type_type: Type,
+    pub type_: Option<flatbuffers::WIPOffset<flatbuffers::UnionWIPOffset>>,
+    pub shape: Option<
+        flatbuffers::WIPOffset<
+            flatbuffers::Vector<'a, flatbuffers::ForwardsUOffset<TensorDim<'a>>>,
+        >,
+    >,
+    pub strides: Option<flatbuffers::WIPOffset<flatbuffers::Vector<'a, i64>>>,
+    pub data: Option<&'a Buffer>,
+}
+impl<'a> Default for TensorArgs<'a> {
+    #[inline]
+    fn default() -> Self {
+        TensorArgs {
+            type_type: Type::NONE,
+            type_: None,
+            shape: None,
+            strides: None,
+            data: None,
+        }
+    }
+}
+pub struct TensorBuilder<'a: 'b, 'b> {
+    fbb_: &'b mut flatbuffers::FlatBufferBuilder<'a>,
+    start_: flatbuffers::WIPOffset<flatbuffers::TableUnfinishedWIPOffset>,
+}
+impl<'a: 'b, 'b> TensorBuilder<'a, 'b> {
+    #[inline]
+    pub fn add_type_type(&mut self, type_type: Type) {
+        self.fbb_
+            .push_slot::<Type>(Tensor::VT_TYPE_TYPE, type_type, Type::NONE);
+    }
+    #[inline]
+    pub fn add_type_(
+        &mut self,
+        type_: flatbuffers::WIPOffset<flatbuffers::UnionWIPOffset>,
+    ) {
+        self.fbb_
+            .push_slot_always::<flatbuffers::WIPOffset<_>>(Tensor::VT_TYPE_, type_);
+    }
+    #[inline]
+    pub fn add_shape(
+        &mut self,
+        shape: flatbuffers::WIPOffset<
+            flatbuffers::Vector<'b, flatbuffers::ForwardsUOffset<TensorDim<'b>>>,
+        >,
+    ) {
+        self.fbb_
+            .push_slot_always::<flatbuffers::WIPOffset<_>>(Tensor::VT_SHAPE, shape);
+    }
+    #[inline]
+    pub fn add_strides(
+        &mut self,
+        strides: flatbuffers::WIPOffset<flatbuffers::Vector<'b, i64>>,
+    ) {
+        self.fbb_
+            .push_slot_always::<flatbuffers::WIPOffset<_>>(Tensor::VT_STRIDES, strides);
+    }
+    #[inline]
+    pub fn add_data(&mut self, data: &'b Buffer) {
+        self.fbb_.push_slot_always::<&Buffer>(Tensor::VT_DATA, data);
+    }
+    #[inline]
+    pub fn new(
+        _fbb: &'b mut flatbuffers::FlatBufferBuilder<'a>,
+    ) -> TensorBuilder<'a, 'b> {
+        let start = _fbb.start_table();
+        TensorBuilder {
+            fbb_: _fbb,
+            start_: start,
+        }
+    }
+    #[inline]
+    pub fn finish(self) -> flatbuffers::WIPOffset<Tensor<'a>> {
+        let o = self.fbb_.end_table(self.start_);
+        flatbuffers::WIPOffset::new(o.value())
+    }
+}
+
+#[inline]
+pub fn get_root_as_tensor<'a>(buf: &'a [u8]) -> Tensor<'a> {
+    flatbuffers::get_root::<Tensor<'a>>(buf)
+}
+
+#[inline]
+pub fn get_size_prefixed_root_as_tensor<'a>(buf: &'a [u8]) -> Tensor<'a> {
+    flatbuffers::get_size_prefixed_root::<Tensor<'a>>(buf)
+}
+
+#[inline]
+pub fn finish_tensor_buffer<'a, 'b>(
+    fbb: &'b mut flatbuffers::FlatBufferBuilder<'a>,
+    root: flatbuffers::WIPOffset<Tensor<'a>>,
+) {
+    fbb.finish(root, None);
+}
+
+#[inline]
+pub fn finish_size_prefixed_tensor_buffer<'a, 'b>(
+    fbb: &'b mut flatbuffers::FlatBufferBuilder<'a>,
+    root: flatbuffers::WIPOffset<Tensor<'a>>,
+) {
+    fbb.finish_size_prefixed(root, None);
+}
diff --git a/rust/arrow/src/lib.rs b/rust/arrow/src/ipc/gen/mod.rs
similarity index 57%
copy from rust/arrow/src/lib.rs
copy to rust/arrow/src/ipc/gen/mod.rs
index 69e8530..39b05db 100644
--- a/rust/arrow/src/lib.rs
+++ b/rust/arrow/src/ipc/gen/mod.rs
@@ -15,27 +15,12 @@
 // specific language governing permissions and limitations
 // under the License.
 
-//! A native Rust implementation of [Apache Arrow](https://arrow.apache.org), a cross-language
-//! development platform for in-memory data.
-//!
-//! Currently the project is developed and tested against nightly Rust. To learn more
-//! about the status of Arrow in Rust, see `README.md`.
+//! Generated code
 
-#![feature(specialization)]
-#![allow(dead_code)]
-#![allow(non_camel_case_types)]
+#![allow(non_snake_case)]
 
-pub mod array;
-pub mod array_data;
-pub mod bitmap;
-pub mod buffer;
-pub mod builder;
-pub mod compute;
-pub mod csv;
-pub mod datatypes;
-pub mod error;
-pub mod json;
-pub mod memory;
-pub mod record_batch;
-pub mod tensor;
-pub mod util;
+pub mod File;
+pub mod Message;
+pub mod Schema;
+pub mod SparseTensor;
+pub mod Tensor;
diff --git a/rust/arrow/src/lib.rs b/rust/arrow/src/ipc/mod.rs
similarity index 57%
copy from rust/arrow/src/lib.rs
copy to rust/arrow/src/ipc/mod.rs
index 69e8530..c0220d7 100644
--- a/rust/arrow/src/lib.rs
+++ b/rust/arrow/src/ipc/mod.rs
@@ -15,27 +15,12 @@
 // specific language governing permissions and limitations
 // under the License.
 
-//! A native Rust implementation of [Apache Arrow](https://arrow.apache.org), a cross-language
-//! development platform for in-memory data.
-//!
-//! Currently the project is developed and tested against nightly Rust. To learn more
-//! about the status of Arrow in Rust, see `README.md`.
+pub mod convert;
 
-#![feature(specialization)]
-#![allow(dead_code)]
-#![allow(non_camel_case_types)]
+pub mod gen;
 
-pub mod array;
-pub mod array_data;
-pub mod bitmap;
-pub mod buffer;
-pub mod builder;
-pub mod compute;
-pub mod csv;
-pub mod datatypes;
-pub mod error;
-pub mod json;
-pub mod memory;
-pub mod record_batch;
-pub mod tensor;
-pub mod util;
+pub use self::gen::File::*;
+pub use self::gen::Message::*;
+pub use self::gen::Schema::*;
+pub use self::gen::SparseTensor::*;
+pub use self::gen::Tensor::*;
diff --git a/rust/arrow/src/lib.rs b/rust/arrow/src/lib.rs
index 69e8530..fb7bec7 100644
--- a/rust/arrow/src/lib.rs
+++ b/rust/arrow/src/lib.rs
@@ -34,6 +34,7 @@ pub mod compute;
 pub mod csv;
 pub mod datatypes;
 pub mod error;
+pub mod ipc;
 pub mod json;
 pub mod memory;
 pub mod record_batch;