You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by np...@apache.org on 2020/11/08 03:10:09 UTC

[arrow] branch master updated: ARROW-10228: [Julia] Contribute Julia implementation

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

npr 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 fb91d53  ARROW-10228: [Julia] Contribute Julia implementation
fb91d53 is described below

commit fb91d5327f19168f6d5be11d19db666ae7858e3a
Author: Jacob Quinn <qu...@gmail.com>
AuthorDate: Sat Nov 7 19:08:54 2020 -0800

    ARROW-10228: [Julia] Contribute Julia implementation
    
    Basically a reopening of #8393. After further discussion, this provides the cleanest contribution in terms of IP clearances and git history, and it should be fairly straightforward to manage the Julia-side of package release registering.
    
    Closes #8547 from quinnj/jq/ARROW-10228
    
    Authored-by: Jacob Quinn <qu...@gmail.com>
    Signed-off-by: Neal Richardson <ne...@gmail.com>
---
 dev/release/rat_exclude_files.txt                  |     9 +
 julia/Arrow/LICENSE.md                             |    15 +
 julia/Arrow/Project.toml                           |    34 +
 julia/Arrow/README.md                              |   145 +
 julia/Arrow/docs/.gitignore                        |     2 +
 julia/Arrow/docs/Manifest.toml                     |   204 +
 julia/Arrow/docs/Project.toml                      |     4 +
 julia/Arrow/docs/make.jl                           |    14 +
 julia/Arrow/docs/mkdocs.yml                        |    27 +
 julia/Arrow/docs/src/index.md                      |    10 +
 julia/Arrow/docs/src/manual.md                     |   110 +
 julia/Arrow/docs/src/reference.md                  |     6 +
 julia/Arrow/src/Arrow.jl                           |   107 +
 julia/Arrow/src/FlatBuffers/FlatBuffers.jl         |   153 +
 julia/Arrow/src/FlatBuffers/builder.jl             |   440 +
 julia/Arrow/src/FlatBuffers/table.jl               |   170 +
 julia/Arrow/src/arraytypes/arraytypes.jl           |   180 +
 julia/Arrow/src/arraytypes/bool.jl                 |   112 +
 julia/Arrow/src/arraytypes/compressed.jl           |    90 +
 julia/Arrow/src/arraytypes/dictencoding.jl         |   253 +
 julia/Arrow/src/arraytypes/fixedsizelist.jl        |   153 +
 julia/Arrow/src/arraytypes/list.jl                 |   209 +
 julia/Arrow/src/arraytypes/map.jl                  |   115 +
 julia/Arrow/src/arraytypes/primitive.jl            |   106 +
 julia/Arrow/src/arraytypes/struct.jl               |   125 +
 julia/Arrow/src/arraytypes/unions.jl               |   279 +
 julia/Arrow/src/arrowtypes.jl                      |   153 +
 julia/Arrow/src/eltypes.jl                         |   408 +
 julia/Arrow/src/metadata/File.jl                   |    90 +
 julia/Arrow/src/metadata/Flatbuf.jl                |    25 +
 julia/Arrow/src/metadata/Message.jl                |   202 +
 julia/Arrow/src/metadata/Schema.jl                 |   605 +
 julia/Arrow/src/table.jl                           |   556 +
 julia/Arrow/src/utils.jl                           |   197 +
 julia/Arrow/src/write.jl                           |   454 +
 julia/Arrow/test/arrowjson.jl                      |   611 +
 julia/Arrow/test/arrowjson/datetime.json           |   911 +
 julia/Arrow/test/arrowjson/decimal.json            | 32948 +++++++++++++++++++
 julia/Arrow/test/arrowjson/dictionary.json         |   422 +
 .../Arrow/test/arrowjson/dictionary_unsigned.json  |   323 +
 julia/Arrow/test/arrowjson/map.json                |   291 +
 julia/Arrow/test/arrowjson/nested.json             |   537 +
 julia/Arrow/test/arrowjson/primitive-empty.json    |   879 +
 julia/Arrow/test/arrowjson/primitive.json          |  1890 ++
 .../Arrow/test/arrowjson/primitive_no_batches.json |   287 +
 julia/Arrow/test/integrationtest.jl                |    49 +
 julia/Arrow/test/pyarrow_roundtrip.jl              |    74 +
 julia/Arrow/test/runtests.jl                       |   162 +
 julia/Arrow/test/testtables.jl                     |   242 +
 49 files changed, 45388 insertions(+)

diff --git a/dev/release/rat_exclude_files.txt b/dev/release/rat_exclude_files.txt
index 2c3f63d..2ba1bc9 100644
--- a/dev/release/rat_exclude_files.txt
+++ b/dev/release/rat_exclude_files.txt
@@ -258,3 +258,12 @@ ruby/red-arrow/.yardopts
 rust/arrow/test/data/*.csv
 rust/rust-toolchain
 rust/arrow-flight/src/arrow.flight.protocol.rs
+julia/Arrow/Project.toml
+julia/Arrow/README.md
+julia/Arrow/docs/Manifest.toml
+julia/Arrow/docs/Project.toml
+julia/Arrow/docs/make.jl
+julia/Arrow/docs/mkdocs.yml
+julia/Arrow/docs/src/index.md
+julia/Arrow/docs/src/manual.md
+julia/Arrow/docs/src/reference.md
\ No newline at end of file
diff --git a/julia/Arrow/LICENSE.md b/julia/Arrow/LICENSE.md
new file mode 100644
index 0000000..136e1c6
--- /dev/null
+++ b/julia/Arrow/LICENSE.md
@@ -0,0 +1,15 @@
+# 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.
\ No newline at end of file
diff --git a/julia/Arrow/Project.toml b/julia/Arrow/Project.toml
new file mode 100644
index 0000000..3d5de48
--- /dev/null
+++ b/julia/Arrow/Project.toml
@@ -0,0 +1,34 @@
+name = "Arrow"
+uuid = "69666777-d1a9-59fb-9406-91d4454c9d45"
+authors = ["quinnj <qu...@gmail.com>"]
+version = "0.3.0"
+
+[deps]
+CodecLz4 = "5ba52731-8f18-5e0d-9241-30f10d1ec561"
+CodecZstd = "6b39b394-51ab-5f42-8807-6242bab2b4c2"
+DataAPI = "9a962f9c-6df0-11e9-0e5d-c546b8b5ee8a"
+Dates = "ade2ca70-3891-5945-98fb-dc099432e06a"
+Mmap = "a63ad114-7e13-5084-954f-fe012c677804"
+PooledArrays = "2dfb63ee-cc39-5dd5-95bd-886bf059d720"
+SentinelArrays = "91c51154-3ec4-41a3-a24f-3f23e20d615c"
+Tables = "bd369af6-aec1-5ad0-b16a-f7cc5008161c"
+TimeZones = "f269a46b-ccf7-5d73-abea-4c690281aa53"
+
+[compat]
+CodecLz4 = "0.4"
+CodecZstd = "0.7"
+julia = "1.3"
+DataAPI = "1"
+PooledArrays = "0.5"
+Tables = "1.1"
+SentinelArrays = "1"
+
+[extras]
+JSON3 = "0f8b85d8-7281-11e9-16c2-39a750bddbf1"
+Random = "9a3f8284-a2c9-5f02-9a11-845980a1fd5c"
+StructTypes = "856f2bd8-1eba-4b0a-8007-ebc267875bd4"
+Test = "8dfed614-e22c-5e08-85e1-65c5234f0b40"
+
+
+[targets]
+test = ["Test", "Random", "JSON3", "StructTypes"]
diff --git a/julia/Arrow/README.md b/julia/Arrow/README.md
new file mode 100644
index 0000000..188e858
--- /dev/null
+++ b/julia/Arrow/README.md
@@ -0,0 +1,145 @@
+# Arrow
+
+[![Build Status](https://travis-ci.com/JuliaData/Arrow.jl.svg?branch=master)](https://travis-ci.com/JuliaData/Arrow.jl.svg?branch=master)
+[![codecov](https://codecov.io/gh/JuliaData/Arrow.jl/branch/master/graph/badge.svg)](https://codecov.io/gh/JuliaData/Arrow.jl)
+
+This is a pure Julia implementation of the [Apache Arrow](https://arrow.apache.org) data standard.  This package provides Julia `AbstractVector` objects for
+referencing data that conforms to the Arrow standard.  This allows users to seamlessly interface Arrow formatted data with a great deal of existing Julia code.
+
+Please see this [document](https://arrow.apache.org/docs/format/Columnar.html#physical-memory-layout) for a description of the Arrow memory layout.
+
+## Format Support
+
+This implementation supports the 1.0 version of the specification, including support for:
+  * All primitive data types
+  * All nested data types
+  * Dictionary encodings and messages
+  * Extension types
+  * Streaming, file, record batch, and replacement and isdelta dictionary messages
+
+It currently doesn't include support for:
+  * Tensors or sparse tensors
+  * Flight RPC
+  * C data interface
+
+Third-party data formats:
+  * csv and parquet support via the existing CSV.jl and Parquet.jl packages
+  * Other Tables.jl-compatible packages automatically supported (DataFrames.jl, JSONTables.jl, JuliaDB.jl, SQLite.jl, MySQL.jl, JDBC.jl, ODBC.jl, XLSX.jl, etc.)
+  * No current Julia packages support ORC or Avro data formats
+
+## Basic usage:
+
+### Installation
+
+```julia
+] add Arrow
+```
+
+### Reading
+
+#### `Arrow.Table`
+
+    Arrow.Table(io::IO; convert::Bool=true)
+    Arrow.Table(file::String; convert::Bool=true)
+    Arrow.Table(bytes::Vector{UInt8}, pos=1, len=nothing; convert::Bool=true)
+
+Read an arrow formatted table, from:
+ * `io`, bytes will be read all at once via `read(io)`
+ * `file`, bytes will be read via `Mmap.mmap(file)`
+ * `bytes`, a byte vector directly, optionally allowing specifying the starting byte position `pos` and `len`
+
+Returns a `Arrow.Table` object that allows column access via `table.col1`, `table[:col1]`, or `table[1]`.
+
+The Apache Arrow standard is foremost a "columnar" format and saves a variety of metadata with each column (such as column name, type, length, etc.).
+A data set which has tens of thousands of columns is probably not well suited for the arrow format and may cause dramatic file size increases when one saves to a `arrow` file.
+If it is possible to reshape the data such that there are not as many columns, `Arrow.Table` should not have as many problems. 
+A simple method Julia provides is to simply execute `transpose(data)` to switch the rows and columns of your data if that does not interfere with one's analysis.
+
+NOTE: the columns in an `Arrow.Table` are views into the original arrow memory, and hence are not easily
+modifiable (with e.g. `push!`, `append!`, etc.). To mutate arrow columns, call `copy(x)` to materialize
+the arrow data as a normal Julia array.
+
+`Arrow.Table` also satisfies the Tables.jl interface, and so can easily be materialized via any supporting
+sink function: e.g. `DataFrame(Arrow.Table(file))`, `SQLite.load!(db, "table", Arrow.Table(file))`, etc.
+
+Supports the `convert` keyword argument which controls whether certain arrow primitive types will be
+lazily converted to more friendly Julia defaults; by default, `convert=true`.
+
+##### Examples
+
+```julia
+using Arrow
+
+# read arrow table from file format
+tbl = Arrow.Table(file)
+
+# read arrow table from IO
+tbl = Arrow.Table(io)
+
+# read arrow table directly from bytes, like from an HTTP request
+resp = HTTP.get(url)
+tbl = Arrow.Table(resp.body)
+```
+
+#### `Arrow.Stream`
+
+    Arrow.Stream(io::IO; convert::Bool=true)
+    Arrow.Stream(file::String; convert::Bool=true)
+    Arrow.Stream(bytes::Vector{UInt8}, pos=1, len=nothing; convert::Bool=true)
+
+Start reading an arrow formatted table, from:
+ * `io`, bytes will be read all at once via `read(io)`
+ * `file`, bytes will be read via `Mmap.mmap(file)`
+ * `bytes`, a byte vector directly, optionally allowing specifying the starting byte position `pos` and `len`
+
+Reads the initial schema message from the arrow stream/file, then returns an `Arrow.Stream` object
+which will iterate over record batch messages, producing an `Arrow.Table` on each iteration.
+
+By iterating `Arrow.Table`, `Arrow.Stream` satisfies the `Tables.partitions` interface, and as such can
+be passed to Tables.jl-compatible sink functions.
+
+This allows iterating over extremely large "arrow tables" in chunks represented as record batches.
+
+Supports the `convert` keyword argument which controls whether certain arrow primitive types will be
+lazily converted to more friendly Julia defaults; by default, `convert=true`.
+
+### Writing
+
+#### `Arrow.write`
+
+    Arrow.write(io::IO, tbl)
+    Arrow.write(file::String, tbl)
+
+Write any Tables.jl-compatible `tbl` out as arrow formatted data.
+Providing an `io::IO` argument will cause the data to be written to it
+in the "streaming" format, unless `file=true` keyword argument is passed.
+Providing a `file::String` argument will result in the "file" format being written.
+
+Multiple record batches will be written based on the number of
+`Tables.partitions(tbl)` that are provided; by default, this is just
+one for a given table, but some table sources support automatic
+partitioning. Note you can turn multiple table objects into partitions
+by doing `Tables.partitioner([tbl1, tbl2, ...])`, but note that
+each table must have the exact same `Tables.Schema`.
+
+By default, `Arrow.write` will use multiple threads to write multiple
+record batches simultaneously (e.g. if julia is started with `julia -t 8`).
+
+Supported keyword arguments to `Arrow.write` include:
+  * `compress`: possible values include `:lz4`, `:zstd`, or your own initialized `LZ4FrameCompressor` or `ZstdCompressor` objects; will cause all buffers in each record batch to use the respective compression encoding
+  * `alignment::Int=8`: specify the number of bytes to align buffers to when written in messages; strongly recommended to only use alignment values of 8 or 64 for modern memory cache line optimization
+  * `dictencode::Bool=false`: whether all columns should use dictionary encoding when being written
+  * `dictencodenested::Bool=false`: whether nested data type columns should also dict encode nested arrays/buffers; many other implementations don't support this
+  * `denseunions::Bool=true`: whether Julia `Vector{<:Union}` arrays should be written using the dense union layout; passing `false` will result in the sparse union layout
+  * `largelists::Bool=false`: causes list column types to be written with Int64 offset arrays; mainly for testing purposes; by default, Int64 offsets will be used only if needed
+  * `file::Bool=false`: if a an `io` argument is being written to, passing `file=true` will cause the arrow file format to be written instead of just IPC streaming
+
+##### Examples
+
+```julia
+# write directly to any IO in streaming format
+Arrow.write(io, tbl)
+
+# write to a file in file format
+Arrow.write("data.arrow", tbl)
+```
diff --git a/julia/Arrow/docs/.gitignore b/julia/Arrow/docs/.gitignore
new file mode 100644
index 0000000..a303fff
--- /dev/null
+++ b/julia/Arrow/docs/.gitignore
@@ -0,0 +1,2 @@
+build/
+site/
diff --git a/julia/Arrow/docs/Manifest.toml b/julia/Arrow/docs/Manifest.toml
new file mode 100644
index 0000000..69420b1
--- /dev/null
+++ b/julia/Arrow/docs/Manifest.toml
@@ -0,0 +1,204 @@
+# This file is machine-generated - editing it directly is not advised
+
+[[ArgTools]]
+uuid = "0dad84c5-d112-42e6-8d28-ef12dabb789f"
+
+[[Arrow]]
+deps = ["CodecLz4", "CodecZstd", "DataAPI", "Dates", "Mmap", "PooledArrays", "SentinelArrays", "Tables"]
+git-tree-sha1 = "76641f71ac332cd4d3cf54b98234a0f597bd7a2f"
+uuid = "69666777-d1a9-59fb-9406-91d4454c9d45"
+version = "0.3.0"
+
+[[Artifacts]]
+uuid = "56f22d72-fd6d-98f1-02f0-08ddc0907c33"
+
+[[Base64]]
+uuid = "2a0f44e3-6c83-55bd-87e4-b1978d98bd5f"
+
+[[CodecLz4]]
+deps = ["Lz4_jll", "TranscodingStreams"]
+git-tree-sha1 = "59fe0cb37784288d6b9f1baebddbf75457395d40"
+uuid = "5ba52731-8f18-5e0d-9241-30f10d1ec561"
+version = "0.4.0"
+
+[[CodecZstd]]
+deps = ["TranscodingStreams", "Zstd_jll"]
+git-tree-sha1 = "d19cd9ae79ef31774151637492291d75194fc5fa"
+uuid = "6b39b394-51ab-5f42-8807-6242bab2b4c2"
+version = "0.7.0"
+
+[[DataAPI]]
+git-tree-sha1 = "176e23402d80e7743fc26c19c681bfb11246af32"
+uuid = "9a962f9c-6df0-11e9-0e5d-c546b8b5ee8a"
+version = "1.3.0"
+
+[[DataValueInterfaces]]
+git-tree-sha1 = "bfc1187b79289637fa0ef6d4436ebdfe6905cbd6"
+uuid = "e2d170a0-9d28-54be-80f0-106bbe20a464"
+version = "1.0.0"
+
+[[Dates]]
+deps = ["Printf"]
+uuid = "ade2ca70-3891-5945-98fb-dc099432e06a"
+
+[[DocStringExtensions]]
+deps = ["LibGit2", "Markdown", "Pkg", "Test"]
+git-tree-sha1 = "50ddf44c53698f5e784bbebb3f4b21c5807401b1"
+uuid = "ffbed154-4ef7-542d-bbb7-c09d3a79fcae"
+version = "0.8.3"
+
+[[Documenter]]
+deps = ["Base64", "Dates", "DocStringExtensions", "InteractiveUtils", "JSON", "LibGit2", "Logging", "Markdown", "REPL", "Test", "Unicode"]
+git-tree-sha1 = "fb1ff838470573adc15c71ba79f8d31328f035da"
+uuid = "e30172f5-a6a5-5a46-863b-614d45cd2de4"
+version = "0.25.2"
+
+[[DocumenterMarkdown]]
+deps = ["Documenter", "Test"]
+git-tree-sha1 = "c302ba512683c3db462ee4eff718ae6fedcbf380"
+uuid = "997ab1e6-3595-5248-9280-8efb232c3433"
+version = "0.2.0"
+
+[[Downloads]]
+deps = ["ArgTools", "LibCURL"]
+uuid = "f43a241f-c20a-4ad4-852c-f6b1247861c6"
+
+[[InteractiveUtils]]
+deps = ["Markdown"]
+uuid = "b77e0a4c-d291-57a0-90e8-8db25a27a240"
+
+[[IteratorInterfaceExtensions]]
+git-tree-sha1 = "a3f24677c21f5bbe9d2a714f95dcd58337fb2856"
+uuid = "82899510-4779-5014-852e-03e436cf321d"
+version = "1.0.0"
+
+[[JLLWrappers]]
+git-tree-sha1 = "c70593677bbf2c3ccab4f7500d0f4dacfff7b75c"
+uuid = "692b3bcd-3c85-4b1f-b108-f13ce0eb3210"
+version = "1.1.3"
+
+[[JSON]]
+deps = ["Dates", "Mmap", "Parsers", "Unicode"]
+git-tree-sha1 = "81690084b6198a2e1da36fcfda16eeca9f9f24e4"
+uuid = "682c06a0-de6a-54ab-a142-c8b1cf79cde6"
+version = "0.21.1"
+
+[[LibCURL]]
+deps = ["LibCURL_jll", "MozillaCACerts_jll"]
+uuid = "b27032c2-a3e7-50c8-80cd-2d36dbcbfd21"
+
+[[LibCURL_jll]]
+deps = ["Libdl"]
+uuid = "deac9b47-8bc7-5906-a0fe-35ac56dc84c0"
+
+[[LibGit2]]
+deps = ["Printf"]
+uuid = "76f85450-5226-5b5a-8eaa-529ad045b433"
+
+[[Libdl]]
+uuid = "8f399da3-3557-5675-b5ff-fb832c97cbdb"
+
+[[LinearAlgebra]]
+deps = ["Libdl"]
+uuid = "37e2e46d-f89d-539d-b4ee-838fcccc9c8e"
+
+[[Logging]]
+uuid = "56ddb016-857b-54e1-b83d-db4d58db5568"
+
+[[Lz4_jll]]
+deps = ["Libdl", "Pkg"]
+git-tree-sha1 = "51b1db0732bbdcfabb60e36095cc3ed9c0016932"
+uuid = "5ced341a-0733-55b8-9ab6-a4889d929147"
+version = "1.9.2+2"
+
+[[Markdown]]
+deps = ["Base64"]
+uuid = "d6f4376e-aef5-505a-96c1-9c027394607a"
+
+[[Mmap]]
+uuid = "a63ad114-7e13-5084-954f-fe012c677804"
+
+[[MozillaCACerts_jll]]
+uuid = "14a3606d-f60d-562e-9121-12d972cd8159"
+
+[[Parsers]]
+deps = ["Dates"]
+git-tree-sha1 = "6fa4202675c05ba0f8268a6ddf07606350eda3ce"
+uuid = "69de0a69-1ddd-5017-9359-2bf0b02dc9f0"
+version = "1.0.11"
+
+[[Pkg]]
+deps = ["Artifacts", "Dates", "Downloads", "LibGit2", "Libdl", "Logging", "Markdown", "Printf", "REPL", "Random", "SHA", "Serialization", "TOML", "UUIDs"]
+uuid = "44cfe95a-1eb2-52ea-b672-e2afdf69b78f"
+
+[[PooledArrays]]
+deps = ["DataAPI"]
+git-tree-sha1 = "b1333d4eced1826e15adbdf01a4ecaccca9d353c"
+uuid = "2dfb63ee-cc39-5dd5-95bd-886bf059d720"
+version = "0.5.3"
+
+[[Printf]]
+deps = ["Unicode"]
+uuid = "de0858da-6303-5e67-8744-51eddeeeb8d7"
+
+[[REPL]]
+deps = ["InteractiveUtils", "Markdown", "Sockets", "Unicode"]
+uuid = "3fa0cd96-eef1-5676-8a61-b3b8758bbffb"
+
+[[Random]]
+deps = ["Serialization"]
+uuid = "9a3f8284-a2c9-5f02-9a11-845980a1fd5c"
+
+[[SHA]]
+uuid = "ea8e919c-243c-51af-8825-aaa63cd721ce"
+
+[[SentinelArrays]]
+deps = ["Dates", "Random"]
+git-tree-sha1 = "6ccde405cf0759eba835eb613130723cb8f10ff9"
+uuid = "91c51154-3ec4-41a3-a24f-3f23e20d615c"
+version = "1.2.16"
+
+[[Serialization]]
+uuid = "9e88b42a-f829-5b0c-bbe9-9e923198166b"
+
+[[Sockets]]
+uuid = "6462fe0b-24de-5631-8697-dd941f90decc"
+
+[[TOML]]
+deps = ["Dates"]
+uuid = "fa267f1f-6049-4f14-aa54-33bafae1ed76"
+
+[[TableTraits]]
+deps = ["IteratorInterfaceExtensions"]
+git-tree-sha1 = "b1ad568ba658d8cbb3b892ed5380a6f3e781a81e"
+uuid = "3783bdb8-4a98-5b6b-af9a-565f29a5fe9c"
+version = "1.0.0"
+
+[[Tables]]
+deps = ["DataAPI", "DataValueInterfaces", "IteratorInterfaceExtensions", "LinearAlgebra", "TableTraits", "Test"]
+git-tree-sha1 = "24a584cf65e2cfabdadc21694fb69d2e74c82b44"
+uuid = "bd369af6-aec1-5ad0-b16a-f7cc5008161c"
+version = "1.1.0"
+
+[[Test]]
+deps = ["InteractiveUtils", "Logging", "Random", "Serialization"]
+uuid = "8dfed614-e22c-5e08-85e1-65c5234f0b40"
+
+[[TranscodingStreams]]
+deps = ["Random", "Test"]
+git-tree-sha1 = "7c53c35547de1c5b9d46a4797cf6d8253807108c"
+uuid = "3bb67fe8-82b1-5028-8e26-92a6c54297fa"
+version = "0.9.5"
+
+[[UUIDs]]
+deps = ["Random", "SHA"]
+uuid = "cf7118a7-6976-5b1a-9a39-7adc72f591a4"
+
+[[Unicode]]
+uuid = "4ec0a83e-493e-50e2-b9ac-8f72acf5a8f5"
+
+[[Zstd_jll]]
+deps = ["Artifacts", "JLLWrappers", "Libdl", "Pkg"]
+git-tree-sha1 = "6f1abcb0c44f184690912aa4b0ba861dd64f11b9"
+uuid = "3161d3a3-bdf6-5164-811a-617609db77b4"
+version = "1.4.5+2"
diff --git a/julia/Arrow/docs/Project.toml b/julia/Arrow/docs/Project.toml
new file mode 100644
index 0000000..fa6129c
--- /dev/null
+++ b/julia/Arrow/docs/Project.toml
@@ -0,0 +1,4 @@
+[deps]
+Arrow = "69666777-d1a9-59fb-9406-91d4454c9d45"
+Documenter = "e30172f5-a6a5-5a46-863b-614d45cd2de4"
+DocumenterMarkdown = "997ab1e6-3595-5248-9280-8efb232c3433"
diff --git a/julia/Arrow/docs/make.jl b/julia/Arrow/docs/make.jl
new file mode 100644
index 0000000..a8fbc9a
--- /dev/null
+++ b/julia/Arrow/docs/make.jl
@@ -0,0 +1,14 @@
+using Documenter, DocumenterMarkdown
+using Arrow
+
+makedocs(
+    format = Markdown(),
+    modules = [Arrow],
+    pages = [
+        "Home" => "index.md",
+        "User Manual" => "manual.md",
+        "API Reference" => "reference.md"
+    ]
+)
+
+deploydocs(repo = "github.com/JuliaData/Arrow.jl.git")
diff --git a/julia/Arrow/docs/mkdocs.yml b/julia/Arrow/docs/mkdocs.yml
new file mode 100644
index 0000000..6d57305
--- /dev/null
+++ b/julia/Arrow/docs/mkdocs.yml
@@ -0,0 +1,27 @@
+# See the mkdocs user guide for more information on these settings.
+#   http://www.mkdocs.org/user-guide/configuration/
+
+site_name:        Arrow.jl
+#repo_url:         https://github.com/USER_NAME/PACKAGE_NAME.jl
+#site_description: Description...
+#site_author:      USER_NAME
+
+theme: readthedocs
+
+extra_css:
+  - assets/Documenter.css
+
+extra_javascript:
+  - https://cdnjs.cloudflare.com/ajax/libs/mathjax/2.7.1/MathJax.js?config=TeX-AMS_HTML
+  - assets/mathjaxhelper.js
+
+markdown_extensions:
+  - extra
+  - tables
+  - fenced_code
+  - mdx_math
+
+docs_dir: 'build'
+
+pages:
+  - Home: index.md
diff --git a/julia/Arrow/docs/src/index.md b/julia/Arrow/docs/src/index.md
new file mode 100644
index 0000000..ee4d141
--- /dev/null
+++ b/julia/Arrow/docs/src/index.md
@@ -0,0 +1,10 @@
+# Arrow.jl
+
+```@contents
+Pages = ["manual.md", "reference.md"]
+Depth = 3
+```
+
+```@docs
+Arrow
+```
\ No newline at end of file
diff --git a/julia/Arrow/docs/src/manual.md b/julia/Arrow/docs/src/manual.md
new file mode 100644
index 0000000..a583b2a
--- /dev/null
+++ b/julia/Arrow/docs/src/manual.md
@@ -0,0 +1,110 @@
+# User Manual
+
+The goal of this documentation is to provide a brief introduction to the arrow data format, then provide a walk-through of the functionality provided in the Arrow.jl Julia package, with an aim to expose a little of the machinery "under the hood" to help explain how things work and how that influences real-world use-cases for the arrow data format.
+
+The best place to learn about the Apache arrow project is [the website itself](https://arrow.apache.org/), specifically the data format [specification](https://arrow.apache.org/docs/format/Columnar.html). Put briefly, the arrow project provides a formal speficiation for how columnar, "table" data can be laid out efficiently in memory to standardize and maximize the ability to share data across languages/platforms. In the current [apache/arrow GitHub repository](https://github.com/apache/ [...]
+
+The [Arrow.jl](https://github.com/JuliaData/Arrow.jl) Julia package is another implementation, allowing the ability to both read and write data in the arrow format. As a data format, arrow specifies an exact memory layout to be used for columnar table data, and as such, "reading" involves custom Julia objects ([`Arrow.Table`](@ref) and [`Arrow.Stream`](@ref)), which read the *metadata* of an "arrow memory blob", then *wrap* the array data contained therein, having learned the type and si [...]
+
+
+## Reading arrow data
+
+After installing the Arrow.jl Julia package (via `] add Arrow`), and if you have some arrow data, let's say a file named `data.arrow` generated from the [`pyarrow`](https://arrow.apache.org/docs/python/) library (a Python library for interfacing with arrow data), you can then read that arrow data into a Julia session by doing:
+
+```julia
+using Arrow
+
+table = Arrow.Table("data.arrow")
+```
+
+### `Arrow.Table`
+
+The type of `table` in this example will be an `Arrow.Table`. When "reading" the arrow data, `Arrow.Table` first ["mmapped"](https://en.wikipedia.org/wiki/Mmap) the `data.arrow` file, which is an important technique for dealing with data larger than available RAM on a system. By "mmapping" a file, the OS doesn't actually load the entire file contents into RAM at the same time, but file contents are "swapped" into RAM as different regions of a file are requested. Once "mmapped", `Arrow.Ta [...]
+
+* [`Arrow.Primitive`](@ref): the most common array type for simple, fixed-size elements like integers, floats, time types, and decimals
+* [`Arrow.List`](@ref): an array type where its own elements are also arrays of some kind, like string columns, where each element can be thought of as an array of characters
+* [`Arrow.FixedSizeList`](@ref): similar to the `List` type, but where each array element has a fixed number of elements itself; you can think of this like a `Vector{NTuple{N, T}}`, where `N` is the fixed-size width
+* [`Arrow.Map`](@ref): an array type where each element is like a Julia `Dict`; a list of key value pairs like a `Vector{Dict}`
+* [`Arrow.Struct`](@ref): an array type where each element is an instance of a custom struct, i.e. an ordered collection of named & typed fields, kind of like a `Vector{NamedTuple}`
+* [`Arrow.DenseUnion`](@ref): an array type where elements may be of several different types, stored compactly; can be thought of like `Vector{Union{A, B}}`
+* [`Arrow.SparseUnion`](@ref): another array type where elements may be of several different types, but stored as if made up of identically lengthed child arrays for each possible type (less memory efficient than `DenseUnion`)
+* [`Arrow.DictEncoded`](@ref): a special array type where values are "dictionary encoded", meaning the list of unique, possible values for an array are stored internally in an "encoding pool", whereas each stored element of the array is just an integer "code" to index into the encoding pool for the actual value.
+
+And while these custom array types do subtype `AbstractArray`, there is only limited support for `setindex!`. Remember, these arrays are "views" into the raw arrow bytes, so for array types other than `Arrow.Primitive`, it gets pretty tricky to allow manipulating those raw arrow bytes. Nevetheless, it's as simple as calling `copy(x)` where `x` is any `ArrowVector` type, and a normal Julia `Vector` type will be fully materialized (which would then allow mutating/manipulating values).
+
+So, what can you do with an `Arrow.Table` full of data? Quite a bit actually!
+
+Because `Arrow.Table` implements the [Tables.jl](https://juliadata.github.io/Tables.jl/stable/) interface, it opens up a world of integrations for using arrow data. A few examples include:
+
+* `df = DataFrame(Arrow.Table(file))`: Build a [`DataFrame`](https://juliadata.github.io/DataFrames.jl/stable/), using the arrow vectors themselves; this allows utilizing a host of DataFrames.jl functionality directly on arrow data; grouping, joining, selecting, etc.
+* `Tables.datavaluerows(Arrow.Table(file)) |> @map(...) |> @filter(...) |> DataFrame`: use [`Query.jl`'s](https://www.queryverse.org/Query.jl/stable/standalonequerycommands/) row-processing utilities to map, group, filter, mutate, etc. directly over arrow data.
+* `Arrow.Table(file) |> SQLite.load!(db, "arrow_table")`: load arrow data directly into an sqlite database/table, where sql queries can be executed on the data
+* `Arrow.Table(file) |> CSV.write("arrow.csv")`: write arrow data out to a csv file
+
+A full list of Julia packages leveraging the Tables.jl inteface can be found [here](https://github.com/JuliaData/Tables.jl/blob/master/INTEGRATIONS.md).
+
+Apart from letting other packages have all the fun, an `Arrow.Table` itself can be plenty useful. For example, with `tbl = Arrow.Table(file)`:
+* `tbl[1]`: retrieve the first column via indexing; the number of columns can be queried via `length(tbl)`
+* `tbl[:col1]` or `tbl.col1`: retrieve the column named `col1`, either via indexing with the column name given as a `Symbol`, or via "dot-access"
+* `for col in tbl`: iterate through columns in the table
+* `AbstractDict` methods like `haskey(tbl, :col1)`, `get(tbl, :col1, nothing)`, `keys(tbl)`, or `values(tbl)`
+
+### Arrow types
+
+In the arrow data format, specific logical types are supported, a list of which can be found [here](https://arrow.apache.org/docs/status.html#data-types). These include booleans, integers of various bit widths, floats, decimals, time types, and binary/string. While most of these map naturally to types builtin to Julia itself, there are a few cases where the definitions are slightly different, and in these cases, by default, they are converted to more "friendly" Julia types (this auto con [...]
+
+* `Date`, `Time`, `Timestamp`, and `Duration` all have natural Julia defintions in `Dates.Date`, `Dates.Time`, `TimeZones.ZonedDateTime`, and `Dates.Period` subtypes, respectively. 
+* `Char` and `Symbol` Julia types are mapped to arrow string types, with additional metadata of the original Julia type; this allows deserializing directly to `Char` and `Symbol` in Julia, while other language implementations will see these columns as just strings
+* `Decimal128` has no corresponding builtin Julia type, so it's deserialized using a compatible type definition in Arrow.jl itself: `Arrow.Decimal`
+
+Note that when `convert=false` is passed, data will be returned in Arrow.jl-defined types that exactly match the arrow definitions of those types; the authoritative source for how each type represents its data can be found in the arrow [`Schema.fbs`](https://github.com/apache/arrow/blob/master/format/Schema.fbs) file.
+
+### `Arrow.Stream`
+
+In addition to `Arrow.Table`, the Arrow.jl package also provides `Arrow.Stream` for processing arrow data. While `Arrow.Table` will iterate all record batches in an arrow file/stream, concatenating columns, `Arrow.Stream` provides a way to *iterate* through record batches, one at a time. Each iteration yields an `Arrow.Table` instance, with columns/data for a single record batch. This allows, if so desired, "batch processing" of arrow data, one record batch at a time, instead of creating [...]
+
+### Table and column metadata
+
+The arrow format allows attaching arbitrary metadata in the form of a `Dict{String, String}` to tables and individual columns. The Arrow.jl package supports retrieving serialized metadata by calling `Arrow.getmetadata(table)` or `Arrow.getmetadata(column)`.
+
+## Writing arrow data
+
+Ok, so that's a pretty good rundown of *reading* arrow data, but how do you *produce* arrow data? Enter `Arrow.write`.
+
+### `Arrow.write`
+
+With `Arrow.write`, you provide either an `io::IO` argument or `file::String` to write the arrow data to, as well as a Tables.jl-compatible source that contains the data to be written.
+
+What are some examples of Tables.jl-compatible sources? A few examples include:
+* `Arrow.write(io, df::DataFrame)`: A `DataFrame` is a collection of indexable columns
+* `Arrow.write(io, CSV.File(file))`: read data from a csv file and write out to arrow format
+* `Arrow.write(io, DBInterface.execute(db, sql_query))`: Execute an SQL query against a database via the [`DBInterface.jl`](https://github.com/JuliaDatabases/DBInterface.jl) interface, and write the query resultset out directly in the arrow format. Packages that implement DBInterface include [SQLite.jl](https://juliadatabases.github.io/SQLite.jl/stable/), [MySQL.jl](https://juliadatabases.github.io/MySQL.jl/dev/), and [ODBC.jl](http://juliadatabases.github.io/ODBC.jl/latest/). 
+* `df |> @map(...) |> Arrow.write(io)`: Write the results of a [Query.jl](https://www.queryverse.org/Query.jl/stable/) chain of operations directly out as arrow data
+* `jsontable(json) |> Arrow.write(io)`: Treat a json array of objects or object of arrays as a "table" and write it out as arrow data using the [JSONTables.jl](https://github.com/JuliaData/JSONTables.jl) package
+* `Arrow.write(io, (col1=data1, col2=data2, ...))`: a `NamedTuple` of `AbstractVector`s or an `AbstractVector` of `NamedTuple`s are both considered tables by default, so they can be quickly constructed for easy writing of arrow data if you already have columns of data
+
+And these are just a few examples of the numerous [integrations](https://github.com/JuliaData/Tables.jl/blob/master/INTEGRATIONS.md).
+
+In addition to just writing out a single "table" of data as a single arrow record batch, `Arrow.write` also supports writing out multiple record batches when the input supports the `Tables.partitions` functionality. One immediate, though perhaps not incredibly useful example, is `Arrow.Stream`. `Arrow.Stream` implements `Tables.partitions` in that it iterates "tables" (specifically `Arrow.Table`), and as such, `Arrow.write` will iterate an `Arrow.Stream`, and write out each `Arrow.Table` [...]
+
+In addition to inputs that support `Tables.partitions`, note that the Tables.jl itself provides the `Tables.partitioner` function, which allows providing your own separate instances of similarly-schema-ed tables as "partitions", like:
+
+```julia
+# treat 2 separate NamedTuples of vectors with same schema as 1 table, 2 partitions
+tbl_parts = Tables.partitioner([(col1=data1, col2=data2), (col1=data3, col2=data4)])
+Arrow.write(io, tbl_parts)
+
+# treat an array of csv files with same schema where each file is a partition
+# in this form, a function `CSV.File` is applied to each element of 2nd argument
+csv_parts = Tables.partitioner(CSV.File, csv_files)
+Arrow.write(io, csv_parts)
+```
+
+### Multithreaded writing
+
+By default, `Arrow.write` will use multiple threads to write multiple
+record batches simultaneously (e.g. if julia is started with `julia -t 8` or the `JULIA_NUM_THREADS` environment variable is set).
+
+### Compression
+
+Compression is supported when writing via the `compress` keyword argument. Possible values include `:lz4`, `:zstd`, or your own initialized `LZ4FrameCompressor` or `ZstdCompressor` objects; will cause all buffers in each record batch to use the respective compression encoding or compressor.
diff --git a/julia/Arrow/docs/src/reference.md b/julia/Arrow/docs/src/reference.md
new file mode 100644
index 0000000..9130082
--- /dev/null
+++ b/julia/Arrow/docs/src/reference.md
@@ -0,0 +1,6 @@
+# API Reference
+
+```@autodocs
+Modules = [Arrow]
+Order   = [:type, :function]
+```
\ No newline at end of file
diff --git a/julia/Arrow/src/Arrow.jl b/julia/Arrow/src/Arrow.jl
new file mode 100644
index 0000000..4eb5986
--- /dev/null
+++ b/julia/Arrow/src/Arrow.jl
@@ -0,0 +1,107 @@
+# 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.
+
+"""
+    Arrow.jl
+
+A pure Julia implementation of the [apache arrow](https://arrow.apache.org/) memory format specification.
+
+This implementation supports the 1.0 version of the specification, including support for:
+  * All primitive data types
+  * All nested data types
+  * Dictionary encodings, nested dictionary encodings, and messages
+  * Extension types
+  * Streaming, file, record batch, and replacement and isdelta dictionary messages
+  * Buffer compression/decompression via the standard LZ4 frame and Zstd formats
+
+It currently doesn't include support for:
+  * Tensors or sparse tensors
+  * Flight RPC
+  * C data interface
+
+Third-party data formats:
+  * csv and parquet support via the existing [CSV.jl](https://github.com/JuliaData/CSV.jl) and [Parquet.jl](https://github.com/JuliaIO/Parquet.jl) packages
+  * Other [Tables.jl](https://github.com/JuliaData/Tables.jl)-compatible packages automatically supported ([DataFrames.jl](https://github.com/JuliaData/DataFrames.jl), [JSONTables.jl](https://github.com/JuliaData/JSONTables.jl), [JuliaDB.jl](https://github.com/JuliaData/JuliaDB.jl), [SQLite.jl](https://github.com/JuliaDatabases/SQLite.jl), [MySQL.jl](https://github.com/JuliaDatabases/MySQL.jl), [JDBC.jl](https://github.com/JuliaDatabases/JDBC.jl), [ODBC.jl](https://github.com/JuliaDataba [...]
+  * No current Julia packages support ORC or Avro data formats
+
+See docs for official Arrow.jl API with the [User Manual](@ref) and reference docs for [`Arrow.Table`](@ref), [`Arrow.write`](@ref), and [`Arrow.Stream`](@ref).
+"""
+module Arrow
+
+using Mmap
+import Dates
+using DataAPI, Tables, SentinelArrays, PooledArrays, CodecLz4, CodecZstd, TimeZones
+
+using Base: @propagate_inbounds
+import Base: ==
+
+const DEBUG_LEVEL = Ref(0)
+
+function setdebug!(level::Int)
+    DEBUG_LEVEL[] = level
+    return
+end
+
+function withdebug(f, level)
+    lvl = DEBUG_LEVEL[]
+    try
+        setdebug!(level)
+        f()
+    finally
+        setdebug!(lvl)
+    end
+end
+
+macro debug(level, msg)
+    esc(quote
+        if DEBUG_LEVEL[] >= $level
+            println(string("DEBUG: ", $(QuoteNode(__source__.file)), ":", $(QuoteNode(__source__.line)), " ", $msg))
+        end
+    end)
+end
+
+const FILE_FORMAT_MAGIC_BYTES = b"ARROW1"
+const CONTINUATION_INDICATOR_BYTES = 0xffffffff
+
+# vendored flatbuffers code for now
+include("FlatBuffers/FlatBuffers.jl")
+using .FlatBuffers
+
+include("metadata/Flatbuf.jl")
+using .Flatbuf; const Meta = Flatbuf
+
+include("arrowtypes.jl")
+using .ArrowTypes
+include("utils.jl")
+include("arraytypes/arraytypes.jl")
+include("eltypes.jl")
+include("table.jl")
+include("write.jl")
+
+const LZ4_FRAME_COMPRESSOR = Ref{LZ4FrameCompressor}()
+const ZSTD_COMPRESSOR = Ref{ZstdCompressor}()
+
+function __init__()
+    zstd = ZstdCompressor(; level=3)
+    CodecZstd.TranscodingStreams.initialize(zstd)
+    ZSTD_COMPRESSOR[] = zstd
+    lz4 = LZ4FrameCompressor(; compressionlevel=4)
+    CodecLz4.TranscodingStreams.initialize(lz4)
+    LZ4_FRAME_COMPRESSOR[] = lz4
+    return
+end
+
+end  # module Arrow
diff --git a/julia/Arrow/src/FlatBuffers/FlatBuffers.jl b/julia/Arrow/src/FlatBuffers/FlatBuffers.jl
new file mode 100644
index 0000000..f4c7477
--- /dev/null
+++ b/julia/Arrow/src/FlatBuffers/FlatBuffers.jl
@@ -0,0 +1,153 @@
+# 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.
+
+module FlatBuffers
+
+const UOffsetT = UInt32
+const SOffsetT = Int32
+const VOffsetT = UInt16
+const VtableMetadataFields = 2
+
+basetype(::Enum) = UInt8
+
+function readbuffer(t::AbstractVector{UInt8}, pos::Integer, ::Type{Bool})
+    @inbounds b = t[pos + 1]
+    return b === 0x01
+end
+
+function readbuffer(t::AbstractVector{UInt8}, pos::Integer, ::Type{T}) where {T}
+    GC.@preserve t begin
+        ptr = convert(Ptr{T}, pointer(t, pos + 1))
+        x = unsafe_load(ptr)
+    end
+end
+
+include("builder.jl")
+include("table.jl")
+
+function Base.show(io::IO, x::TableOrStruct)
+    print(io, "$(typeof(x))")
+    if isempty(propertynames(x))
+        print(io, "()")
+    else
+        show(io, NamedTuple{propertynames(x)}(Tuple(getproperty(x, y) for y in propertynames(x))))
+    end
+end
+
+abstract type ScopedEnum{T<:Integer} <: Enum{T} end
+
+macro scopedenum(T, syms...)
+    if isempty(syms)
+        throw(ArgumentError("no arguments given for ScopedEnum $T"))
+    end
+    basetype = Int32
+    typename = T
+    if isa(T, Expr) && T.head === :(::) && length(T.args) == 2 && isa(T.args[1], Symbol)
+        typename = T.args[1]
+        basetype = Core.eval(__module__, T.args[2])
+        if !isa(basetype, DataType) || !(basetype <: Integer) || !isbitstype(basetype)
+            throw(ArgumentError("invalid base type for ScopedEnum $typename, $T=::$basetype; base type must be an integer primitive type"))
+        end
+    elseif !isa(T, Symbol)
+        throw(ArgumentError("invalid type expression for ScopedEnum $T"))
+    end
+    values = basetype[]
+    seen = Set{Symbol}()
+    namemap = Dict{basetype,Symbol}()
+    lo = hi = 0
+    i = zero(basetype)
+    hasexpr = false
+
+    if length(syms) == 1 && syms[1] isa Expr && syms[1].head === :block
+        syms = syms[1].args
+    end
+    for s in syms
+        s isa LineNumberNode && continue
+        if isa(s, Symbol)
+            if i == typemin(basetype) && !isempty(values)
+                throw(ArgumentError("overflow in value \"$s\" of ScopedEnum $typename"))
+            end
+        elseif isa(s, Expr) &&
+               (s.head === :(=) || s.head === :kw) &&
+               length(s.args) == 2 && isa(s.args[1], Symbol)
+            i = Core.eval(__module__, s.args[2]) # allow exprs, e.g. uint128"1"
+            if !isa(i, Integer)
+                throw(ArgumentError("invalid value for ScopedEnum $typename, $s; values must be integers"))
+            end
+            i = convert(basetype, i)
+            s = s.args[1]
+            hasexpr = true
+        else
+            throw(ArgumentError(string("invalid argument for ScopedEnum ", typename, ": ", s)))
+        end
+        if !Base.isidentifier(s)
+            throw(ArgumentError("invalid name for ScopedEnum $typename; \"$s\" is not a valid identifier"))
+        end
+        if hasexpr && haskey(namemap, i)
+            throw(ArgumentError("both $s and $(namemap[i]) have value $i in ScopedEnum $typename; values must be unique"))
+        end
+        namemap[i] = s
+        push!(values, i)
+        if s in seen
+            throw(ArgumentError("name \"$s\" in ScopedEnum $typename is not unique"))
+        end
+        push!(seen, s)
+        if length(values) == 1
+            lo = hi = i
+        else
+            lo = min(lo, i)
+            hi = max(hi, i)
+        end
+        i += oneunit(i)
+    end
+    defs = Expr(:block)
+    if isa(typename, Symbol)
+        for (i, sym) in namemap
+            push!(defs.args, :(const $(esc(sym)) = $(esc(typename))($i)))
+        end
+    end
+    mod = Symbol(typename, "Module")
+    syms = Tuple(Base.values(namemap))
+    blk = quote
+        module $(esc(mod))
+            export $(esc(typename))
+            # enum definition
+            primitive type $(esc(typename)) <: ScopedEnum{$(basetype)} $(sizeof(basetype) * 8) end
+            function $(esc(typename))(x::Integer)
+                $(Base.Enums.membershiptest(:x, values)) || Base.Enums.enum_argument_error($(Expr(:quote, typename)), x)
+                return Core.bitcast($(esc(typename)), convert($(basetype), x))
+            end
+            if isdefined(Base.Enums, :namemap)
+                Base.Enums.namemap(::Type{$(esc(typename))}) = $(esc(namemap))
+            end
+            Base.getproperty(::Type{$(esc(typename))}, sym::Symbol) = sym in $syms ? getfield($(esc(mod)), sym) : getfield($(esc(typename)), sym)
+            Base.typemin(x::Type{$(esc(typename))}) = $(esc(typename))($lo)
+            Base.typemax(x::Type{$(esc(typename))}) = $(esc(typename))($hi)
+            let insts = (Any[ $(esc(typename))(v) for v in $values ]...,)
+                Base.instances(::Type{$(esc(typename))}) = insts
+            end
+            FlatBuffers.basetype(::$(esc(typename))) = $(basetype)
+            FlatBuffers.basetype(::Type{$(esc(typename))}) = $(basetype)
+            $defs
+        end
+    end
+    push!(blk.args, :nothing)
+    blk.head = :toplevel
+    push!(blk.args, :(using .$mod))
+    return blk
+end
+
+end # module
diff --git a/julia/Arrow/src/FlatBuffers/builder.jl b/julia/Arrow/src/FlatBuffers/builder.jl
new file mode 100644
index 0000000..0c65c6f
--- /dev/null
+++ b/julia/Arrow/src/FlatBuffers/builder.jl
@@ -0,0 +1,440 @@
+# 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.
+
+const fileIdentifierLength = 4
+
+"""
+Scalar
+A Union of the Julia types `T <: Number` that are allowed in FlatBuffers schema
+"""
+const Scalar = Union{Bool,
+Int8, Int16, Int32, Int64,
+UInt8, UInt16, UInt32, UInt64,
+Float32, Float64, Enum}
+
+"""
+Builder is a state machine for creating FlatBuffer objects.
+Use a Builder to construct object(s) starting from leaf nodes.
+
+A Builder constructs byte buffers in a last-first manner for simplicity and
+performance.
+"""
+mutable struct Builder
+    bytes::Vector{UInt8}
+    minalign::Int
+    vtable::Vector{UOffsetT}
+    objectend::UOffsetT
+    vtables::Vector{UOffsetT}
+    head::UOffsetT
+    nested::Bool
+    finished::Bool
+    sharedstrings::Dict{String, UOffsetT}
+end
+
+bytes(b::Builder) = getfield(b, :bytes)
+
+Builder(size=0) = Builder(zeros(UInt8, size), 1, UOffsetT[], UOffsetT(0), UOffsetT[], UOffsetT(size), false, false, Dict{String, UOffsetT}())
+
+function reset!(b::Builder)
+    empty!(b.bytes)
+    empty!(b.vtable)
+    emtpy!(b.vtables)
+    empty!(b.sharedstrings)
+    b.minalign = 1
+    b.nested = false
+    b.finished = false
+    b.head = 0
+    return
+end
+
+Base.write(sink::Builder, o, x::Union{Bool,UInt8}) = sink.bytes[o+1] = UInt8(x)
+function Base.write(sink::Builder, off, x::T) where {T}
+    off += 1
+    for (i, ind) = enumerate(off:(off + sizeof(T) - 1))
+        sink.bytes[ind] = (x >> ((i-1) * 8)) % UInt8
+    end
+end
+Base.write(b::Builder, o, x::Float32) = write(b, o, reinterpret(UInt32, x))
+Base.write(b::Builder, o, x::Float64) = write(b, o, reinterpret(UInt64, x))
+Base.write(b::Builder, o, x::Enum) = write(b, o, basetype(x)(x))
+
+"""
+`finishedbytes` returns a pointer to the written data in the byte buffer.
+Panics if the builder is not in a finished state (which is caused by calling
+`finish!()`).
+"""
+function finishedbytes(b::Builder)
+    assertfinished(b)
+    return view(b.bytes, (b.head + 1):length(b.bytes))
+end
+
+function startobject!(b::Builder, numfields)
+    assertnotnested(b)
+    b.nested = true
+    resize!(b.vtable, numfields)
+    fill!(b.vtable, 0)
+    b.objectend = offset(b)
+    return
+end
+
+"""
+WriteVtable serializes the vtable for the current object, if applicable.
+
+Before writing out the vtable, this checks pre-existing vtables for equality
+to this one. If an equal vtable is found, point the object to the existing
+vtable and return.
+
+Because vtable values are sensitive to alignment of object data, not all
+logically-equal vtables will be deduplicated.
+
+A vtable has the following format:
+<VOffsetT: size of the vtable in bytes, including this value>
+<VOffsetT: size of the object in bytes, including the vtable offset>
+<VOffsetT: offset for a field> * N, where N is the number of fields in
+the schema for this type. Includes deprecated fields.
+Thus, a vtable is made of 2 + N elements, each SizeVOffsetT bytes wide.
+
+An object has the following format:
+<SOffsetT: offset to this object's vtable (may be negative)>
+<byte: data>+
+"""
+function writevtable!(b::Builder)
+    # Prepend a zero scalar to the object. Later in this function we'll
+    # write an offset here that points to the object's vtable:
+    prepend!(b, SOffsetT(0))
+
+    objectOffset = offset(b)
+    existingVtable = UOffsetT(0)
+
+    # Trim vtable of trailing zeroes.
+    i = findlast(!iszero, b.vtable)
+    if i !== nothing
+        resize!(b.vtable, i)
+    end
+    
+    # Search backwards through existing vtables, because similar vtables
+    # are likely to have been recently appended. See
+    # BenchmarkVtableDeduplication for a case in which this heuristic
+    # saves about 30% of the time used in writing objects with duplicate
+    # tables.
+    for i = length(b.vtables):-1:1
+        # Find the other vtable, which is associated with `i`:
+        vt2Offset = b.vtables[i]
+        vt2Start = length(b.bytes) - vt2Offset
+        vt2Len = readbuffer(b.bytes, vt2Start, VOffsetT)
+
+        metadata = VtableMetadataFields * sizeof(VOffsetT)
+        vt2End = vt2Start + vt2Len
+        vt2 = view(b.bytes, (vt2Start + metadata + 1):vt2End) #TODO: might need a +1 on the start of range here
+
+        # Compare the other vtable to the one under consideration.
+        # If they are equal, store the offset and break:
+        if vtableEqual(b.vtable, objectOffset, vt2)
+            existingVtable = vt2Offset
+            break
+        end
+    end
+
+    if existingVtable == 0
+        # Did not find a vtable, so write this one to the buffer.
+
+        # Write out the current vtable in reverse , because
+        # serialization occurs in last-first order:
+        for i = length(b.vtable):-1:1
+            off::UOffsetT = 0
+            if b.vtable[i] != 0
+                # Forward reference to field;
+                # use 32bit number to assert no overflow:
+                off = objectOffset - b.vtable[i]
+            end
+            prepend!(b, VOffsetT(off))
+        end
+
+        # The two metadata fields are written last.
+
+        # First, store the object bytesize:
+        objectSize = objectOffset - b.objectend
+        prepend!(b, VOffsetT(objectSize))
+
+        # Second, store the vtable bytesize:
+        vbytes = (length(b.vtable) + VtableMetadataFields) * sizeof(VOffsetT)
+        prepend!(b, VOffsetT(vbytes))
+
+        # Next, write the offset to the new vtable in the
+        # already-allocated SOffsetT at the beginning of this object:
+        objectStart = SOffsetT(length(b.bytes) - objectOffset)
+        write(b, objectStart, SOffsetT(offset(b) - objectOffset))
+
+        # Finally, store this vtable in memory for future
+        # deduplication:
+        push!(b.vtables, offset(b))
+    else
+        # Found a duplicate vtable.
+
+        objectStart = SOffsetT(length(b.bytes) - objectOffset)
+        b.head = objectStart
+
+        # Write the offset to the found vtable in the
+        # already-allocated SOffsetT at the beginning of this object:
+        write(b, b.head, SOffsetT(existingVtable) - SOffsetT(objectOffset))
+    end
+
+    empty!(b.vtable)
+    return objectOffset
+end
+
+"""
+`endobject` writes data necessary to finish object construction.
+"""
+function endobject!(b::Builder)
+    assertnested(b)
+    n = writevtable!(b)
+    b.nested = false
+    return n
+end
+
+offset(b::Builder) = UOffsetT(length(b.bytes) - b.head)
+
+pad!(b::Builder, n) = foreach(x->place!(b, 0x00), 1:n)
+
+"""
+`prep!` prepares to write an element of `size` after `additionalbytes`
+have been written, e.g. if you write a string, you need to align such
+the int length field is aligned to sizeof(Int32), and the string data follows it
+directly.
+If all you need to do is align, `additionalbytes` will be 0.
+"""
+function prep!(b::Builder, size, additionalbytes)
+    # Track the biggest thing we've ever aligned to.
+    if size > b.minalign
+        b.minalign = size
+    end
+    # Find the amount of alignment needed such that `size` is properly
+    # aligned after `additionalBytes`:
+    alignsize = xor(Int(-1), (length(b.bytes) - b.head) + additionalbytes) + 1
+    alignsize &= (size - 1)
+
+    # Reallocate the buffer if needed:
+    totalsize = alignsize + size + additionalbytes
+    if b.head <= totalsize
+        len = length(b.bytes)
+        prepend!(b.bytes, zeros(UInt8, totalsize))
+        b.head += length(b.bytes) - len
+    end
+    pad!(b, alignsize)
+    return
+end
+
+function Base.prepend!(b::Builder, x::T) where {T}
+    prep!(b, sizeof(T), 0)
+    place!(b, x)
+    return
+end
+
+function prependoffset!(b::Builder, off)
+    prep!(b, sizeof(Int32), 0) # Ensure alignment is already done.
+    if !(off <= offset(b))
+        throw(ArgumentError("unreachable: $off <= $(offset(b))"))
+    end
+    place!(b, SOffsetT(offset(b) - off + sizeof(SOffsetT)))
+    return
+end
+
+function prependoffsetslot!(b::Builder, o::Int, x::T, d) where {T}
+    if x != T(d)
+        prependoffset!(b, x)
+        slot!(b, o)
+    end
+    return
+end
+
+"""
+`startvector` initializes bookkeeping for writing a new vector.
+
+A vector has the following format:
+<UOffsetT: number of elements in this vector>
+<T: data>+, where T is the type of elements of this vector.
+"""
+function startvector!(b::Builder, elemSize, numElems, alignment)
+    assertnotnested(b)
+    b.nested = true
+    prep!(b, sizeof(UInt32), elemSize * numElems)
+    prep!(b, alignment, elemSize * numElems)
+    return offset(b)
+end
+
+"""
+`endvector` writes data necessary to finish vector construction.
+"""
+function endvector!(b::Builder, vectorNumElems)
+    assertnested(b)
+    place!(b, UOffsetT(vectorNumElems))
+    b.nested = false
+    return offset(b)
+end
+
+function createsharedstring!(b::Builder, s::AbstractString)
+    get!(b.sharedstrings, s) do
+        createstring!(b, s)
+    end
+end
+
+"""
+`createstring!` writes a null-terminated string as a vector.
+"""
+function createstring!(b::Builder, s::Union{AbstractString, AbstractVector{UInt8}})
+    assertnotnested(b)
+    b.nested = true
+    s = codeunits(s)
+    prep!(b, sizeof(UInt32), sizeof(s) + 1)
+    place!(b, UInt8(0))
+
+    l = sizeof(s)
+
+    b.head -= l
+    copyto!(b.bytes, b.head+1, s, 1, l)
+    return endvector!(b, sizeof(s))
+end
+
+createbytevector(b::Builder, v) = createstring!(b, v)
+
+function assertnested(b::Builder)
+    # If you get this assert, you're in an object while trying to write
+    # data that belongs outside of an object.
+    # To fix this, write non-inline data (like vectors) before creating
+    # objects.
+    if !b.nested
+        throw(ArgumentError("Incorrect creation order: must be inside object."))
+    end
+    return
+end
+
+function assertnotnested(b::Builder)
+    # If you hit this, you're trying to construct a Table/Vector/String
+    # during the construction of its parent table (between the MyTableBuilder
+    # and builder.Finish()).
+    # Move the creation of these view-objects to above the MyTableBuilder to
+    # not get this assert.
+    # Ignoring this assert may appear to work in simple cases, but the reason
+    # it is here is that storing objects in-line may cause vtable offsets
+    # to not fit anymore. It also leads to vtable duplication.
+    if b.nested
+        throw(ArgumentError("Incorrect creation order: object must not be nested."))
+    end
+    return
+end
+
+function assertfinished(b::Builder)
+    # If you get this assert, you're attempting to get access a buffer
+    # which hasn't been finished yet. Be sure to call builder.Finish()
+    # with your root table.
+    # If you really need to access an unfinished buffer, use the bytes
+    # buffer directly.
+    if !b.finished
+        throw(ArgumentError("Incorrect use of FinishedBytes(): must call 'Finish' first."))
+    end
+end
+
+"""
+`prependslot!` prepends a `T` onto the object at vtable slot `o`.
+If value `x` equals default `d`, then the slot will be set to zero and no
+other data will be written.
+"""
+function prependslot!(b::Builder, o::Int, x::T, d, sh=false) where {T <: Scalar}
+    if x != T(d)
+        prepend!(b, x)
+        slot!(b, o)
+    end
+    return
+end
+
+"""
+`prependstructslot!` prepends a struct onto the object at vtable slot `o`.
+Structs are stored inline, so nothing additional is being added.
+In generated code, `d` is always 0.
+"""
+function prependstructslot!(b::Builder, voffset, x, d)
+    if x != d
+        assertnested(b)
+        if x != offset(b)
+            throw(ArgumentError("inline data write outside of object"))
+        end
+        slot!(b, voffset)
+    end
+    return
+end
+
+"""
+`slot!` sets the vtable key `voffset` to the current location in the buffer.
+"""
+function slot!(b::Builder, slotnum)
+    b.vtable[slotnum + 1] = offset(b)
+end
+
+# FinishWithFileIdentifier finalizes a buffer, pointing to the given `rootTable`.
+# as well as applys a file identifier
+function finishwithfileidentifier(b::Builder, rootTable, fid)
+    if length(fid) != fileIdentifierLength
+        error("incorrect file identifier length")
+    end
+    # In order to add a file identifier to the flatbuffer message, we need
+    # to prepare an alignment and file identifier length
+    prep!(b, b.minalign, sizeof(Int32) + fileIdentifierLength)
+    for i = fileIdentifierLength:-1:1
+        # place the file identifier
+        place!(b, fid[i])
+    end
+    # finish
+    finish!(b, rootTable)
+end
+
+"""
+`finish!` finalizes a buffer, pointing to the given `rootTable`.
+"""
+function finish!(b::Builder, rootTable)
+    assertnotnested(b)
+    prep!(b, b.minalign, sizeof(UOffsetT))
+    prependoffset!(b, UOffsetT(rootTable))
+    b.finished = true
+    return
+end
+
+"vtableEqual compares an unwritten vtable to a written vtable."
+function vtableEqual(a::Vector{UOffsetT}, objectStart, b::AbstractVector{UInt8})
+    if length(a) * sizeof(VOffsetT) != length(b)
+        return false
+    end
+
+    for i = 0:(length(a)-1)
+        x = read(IOBuffer(view(b, (i * sizeof(VOffsetT) + 1):length(b))), VOffsetT)
+
+        # Skip vtable entries that indicate a default value.
+        x == 0 && a[i+1] == 0 && continue
+
+        y = objectStart - a[i+1]
+        x != y && return false
+    end
+    return true
+end
+
+"""
+`place!` prepends a `T` to the Builder, without checking for space.
+"""
+function place!(b::Builder, x::T) where {T}
+    b.head -= sizeof(T)
+    write(b, b.head, x)
+    return
+end
diff --git a/julia/Arrow/src/FlatBuffers/table.jl b/julia/Arrow/src/FlatBuffers/table.jl
new file mode 100644
index 0000000..cd87c4f
--- /dev/null
+++ b/julia/Arrow/src/FlatBuffers/table.jl
@@ -0,0 +1,170 @@
+# 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.
+
+"""
+Table
+
+The object containing the flatbuffer and positional information specific to the table.
+The `vtable` containing the offsets for specific members precedes `pos`.
+The actual values in the table follow `pos` offset and size of the vtable.
+
+- `bytes::Vector{UInt8}`: the flatbuffer itself
+- `pos::Integer`:  the base position in `bytes` of the table
+"""
+abstract type Table end
+abstract type Struct end
+
+const TableOrStruct = Union{Table, Struct}
+
+bytes(x::TableOrStruct) = getfield(x, :bytes)
+pos(x::TableOrStruct) = getfield(x, :pos)
+
+(::Type{T})(b::Builder) where {T <: TableOrStruct} = T(b.bytes[b.head+1:end], get(b, b.head, Int32))
+
+getrootas(::Type{T}, bytes::Vector{UInt8}, offset) where {T <: Table} = init(T, bytes, offset + readbuffer(bytes, offset, UOffsetT))
+init(::Type{T}, bytes::Vector{UInt8}, pos::Integer) where {T <: TableOrStruct} = T(bytes, pos)
+
+const TableOrBuilder = Union{Table, Struct, Builder}
+
+Base.get(t::TableOrBuilder, pos, ::Type{T}) where {T} = readbuffer(bytes(t), pos, T)
+Base.get(t::TableOrBuilder, pos, ::Type{T}) where {T <: Enum} = T(get(t, pos, basetype(T)))
+
+"""
+`offset` provides access into the Table's vtable.
+
+Deprecated fields are ignored by checking against the vtable's length.
+"""
+function offset(t::Table, vtableoffset)
+    vtable = pos(t) - get(t, pos(t), SOffsetT)
+    return vtableoffset < get(t, vtable, VOffsetT) ? get(t, vtable + vtableoffset, VOffsetT) : VOffsetT(0)
+end
+
+"`indirect` retrieves the relative offset stored at `offset`."
+indirect(t::Table, off) = off + get(t, off, UOffsetT)
+
+getvalue(t, o, ::Type{Nothing}) = nothing
+getvalue(t, o, ::Type{T}) where {T <: Scalar} = get(t, pos(t) + o, T)
+getvalue(t, o, ::Type{T}) where {T <: Enum} = T(get(t, pos(t) + o, enumtype(T)))
+
+function Base.String(t::Table, off)
+    off += get(t, off, UOffsetT)
+    start = off + sizeof(UOffsetT)
+    len = get(t, off, UOffsetT)
+    return unsafe_string(pointer(bytes(t), start + 1), len)
+end
+
+function bytevector(t::Table, off)
+    off += get(t, off, UOffsetT)
+    start = off + sizeof(UOffsetT)
+    len = get(t, off, UOffsetT)
+    return view(bytes(t), (start + 1):(start + len + 1))
+end
+
+"""
+`vectorlen` retrieves the length of the vector whose offset is stored at
+`off` in this object.
+"""
+function vectorlen(t::Table, off)
+    off += pos(t)
+    off += get(t, off, UOffsetT)
+    return Int(get(t, off, UOffsetT))
+end
+
+"""
+`vector` retrieves the start of data of the vector whose offset is stored
+at `off` in this object.
+"""
+function vector(t::Table, off)
+    off += pos(t)
+    x = off + get(t, off, UOffsetT)
+    # data starts after metadata containing the vector length
+    return x + sizeof(UOffsetT)
+end
+
+struct Array{T, S, TT} <: AbstractVector{T}
+    _tab::TT
+    pos::UOffsetT
+    data::Vector{S}
+end
+
+function Array{T}(t::Table, off) where {T}
+    a = vector(t, off)
+    S = T <: Table ? UOffsetT : T <: Struct ? NTuple{structsizeof(T), UInt8} : T
+    ptr = convert(Ptr{S}, pointer(bytes(t), pos(t) + a + 1))
+    data = unsafe_wrap(Base.Array, ptr, vectorlen(t, off))
+    return Array{T, S, typeof(t)}(t, a, data)
+end
+
+function structsizeof end
+
+Base.IndexStyle(::Type{<:Array}) = Base.IndexLinear()
+Base.size(x::Array) = size(x.data)
+Base.@propagate_inbounds function Base.getindex(A::Array{T, S}, i::Integer) where {T, S}
+    if T === S
+        return A.data[i]
+    elseif T <: Struct
+        return init(T, bytes(A._tab), A.pos + (i - 1) * structsizeof(T))
+    else # T isa Table
+        return init(T, bytes(A._tab), indirect(A._tab, A.pos + (i - 1) * 4))
+    end
+end
+
+Base.@propagate_inbounds function Base.setindex!(A::Array{T, S}, v, i::Integer) where {T, S}
+    if T === S
+        return setindex!(A.data, v, i)
+    else
+        error("setindex! not supported for reference/table types")
+    end
+end
+
+function union(t::Table, off)
+    off += pos(t)
+    return off + get(t, off, UOffsetT)
+end
+
+function union!(t::Table, t2::Table, off)
+    off += pos(t)
+    t2.pos = off + get(t, off, UOffsetT)
+    t2.bytes = bytes(t)
+    return
+end
+
+"""
+GetVOffsetTSlot retrieves the VOffsetT that the given vtable location
+points to. If the vtable value is zero, the default value `d`
+will be returned.
+"""
+function getoffsetslot(t::Table, slot, d)
+    off = offset(t, slot)
+    if off == 0
+        return d
+    end
+    return off
+end
+
+"""
+`getslot` retrieves the `T` that the given vtable location
+points to. If the vtable value is zero, the default value `d`
+will be returned.
+"""
+function getslot(t::Table, slot, d::T) where {T}
+    off = offset(t, slot)
+    if off == 0
+        return d
+    end
+
+    return get(t, pos(t) + off, T)
+end
diff --git a/julia/Arrow/src/arraytypes/arraytypes.jl b/julia/Arrow/src/arraytypes/arraytypes.jl
new file mode 100644
index 0000000..92ba3e4
--- /dev/null
+++ b/julia/Arrow/src/arraytypes/arraytypes.jl
@@ -0,0 +1,180 @@
+# 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.
+
+"""
+    Arrow.ArrowVector
+
+An abstract type that subtypes `AbstractVector`. Each specific arrow array type
+subtypes `ArrowVector`. See [`BoolVector`](@ref), [`Primitive`](@ref), [`List`](@ref),
+[`Map`](@ref), [`FixedSizeList`](@ref), [`Struct`](@ref), [`DenseUnion`](@ref),
+[`SparseUnion`](@ref), and [`DictEncoded`](@ref) for more details.
+"""
+abstract type ArrowVector{T} <: AbstractVector{T} end
+
+Base.IndexStyle(::Type{A}) where {A <: ArrowVector} = Base.IndexLinear()
+Base.similar(::Type{A}, dims::Dims) where {T, A <: ArrowVector{T}} = Vector{T}(undef, dims)
+validitybitmap(x::ArrowVector) = x.validity
+nullcount(x::ArrowVector) = validitybitmap(x).nc
+getmetadata(x::ArrowVector) = x.metadata
+
+function toarrowvector(x, i=1, de=Dict{Int64, Any}(), ded=DictEncoding[], meta=getmetadata(x); compression::Union{Nothing, LZ4FrameCompressor, ZstdCompressor}=nothing, kw...)
+    @debug 2 "converting top-level column to arrow format: col = $(typeof(x)), compression = $compression, kw = $(kw.data)"
+    @debug 3 x
+    A = arrowvector(x, i, 0, 0, de, ded, meta; compression=compression, kw...)
+    if compression isa LZ4FrameCompressor
+        A = compress(Meta.CompressionType.LZ4_FRAME, compression, A)
+    elseif compression isa ZstdCompressor
+        A = compress(Meta.CompressionType.ZSTD, compression, A)
+    end
+    @debug 2 "converted top-level column to arrow format: $(typeof(A))"
+    @debug 3 A
+    return A
+end
+
+function arrowvector(x, i, nl, fi, de, ded, meta; dictencoding::Bool=false, dictencode::Bool=false, kw...)
+    if !(x isa DictEncode) && !dictencoding && (dictencode || (x isa AbstractArray && DataAPI.refarray(x) !== x))
+        x = DictEncode(x, dictencodeid(i, nl, fi))
+    end
+    S = maybemissing(eltype(x))
+    return arrowvector(S, x, i, nl, fi, de, ded, meta; dictencode=dictencode, kw...)
+end
+
+# conversions to arrow types
+arrowvector(::Type{Dates.Date}, x, i, nl, fi, de, ded, meta; kw...) =
+    arrowvector(converter(DATE, x), i, nl, fi, de, ded, meta; kw...)
+arrowvector(::Type{Dates.Time}, x, i, nl, fi, de, ded, meta; kw...) =
+    arrowvector(converter(TIME, x), i, nl, fi, de, ded, meta; kw...)
+arrowvector(::Type{Dates.DateTime}, x, i, nl, fi, de, ded, meta; kw...) =
+    arrowvector(converter(DATETIME, x), i, nl, fi, de, ded, meta; kw...)
+arrowvector(::Type{ZonedDateTime}, x, i, nl, fi, de, ded, meta; kw...) =
+    arrowvector(converter(Timestamp{Meta.TimeUnit.MILLISECOND, Symbol(x[1].timezone)}, x), i, nl, fi, de, ded, meta; kw...)
+arrowvector(::Type{P}, x, i, nl, fi, de, ded, meta; kw...) where {P <: Dates.Period} =
+    arrowvector(converter(Duration{arrowperiodtype(P)}, x), i, nl, fi, de, ded, meta; kw...)
+
+# fallback that calls ArrowType
+function arrowvector(::Type{S}, x, i, nl, fi, de, ded, meta; kw...) where {S}
+    if ArrowTypes.istyperegistered(S)
+        meta = meta === nothing ? Dict{String, String}() : meta
+        arrowtype = ArrowTypes.getarrowtype!(meta, S)
+        return arrowvector(converter(arrowtype, x), i, nl, fi, de, ded, meta; kw...)
+    end
+    return arrowvector(ArrowType(S), x, i, nl, fi, de, ded, meta; kw...)
+end
+
+arrowvector(::NullType, x, i, nl, fi, de, ded, meta; kw...) = MissingVector(length(x))
+compress(Z::Meta.CompressionType, comp, v::MissingVector) =
+    Compressed{Z, MissingVector}(v, CompressedBuffer[], length(v), length(v), Compressed[])
+
+function makenodesbuffers!(col::MissingVector, fieldnodes, fieldbuffers, bufferoffset, alignment)
+    push!(fieldnodes, FieldNode(length(col), length(col)))
+    @debug 1 "made field node: nodeidx = $(length(fieldnodes)), col = $(typeof(col)), len = $(fieldnodes[end].length), nc = $(fieldnodes[end].null_count)"
+    return bufferoffset
+end
+
+function writebuffer(io, col::MissingVector, alignment)
+    return
+end
+
+"""
+    Arrow.ValidityBitmap
+
+A bit-packed array type where each bit corresponds to an element in an
+[`ArrowVector`](@ref), indicating whether that element is "valid" (bit == 1),
+or not (bit == 0). Used to indicate element missingness (whether it's null).
+
+If the null count of an array is zero, the `ValidityBitmap` will be "emtpy"
+and all elements are treated as "valid"/non-null.
+"""
+struct ValidityBitmap <: ArrowVector{Bool}
+    bytes::Vector{UInt8} # arrow memory blob
+    pos::Int # starting byte of validity bitmap
+    ℓ::Int # # of _elements_ (not bytes!) in bitmap (because bitpacking)
+    nc::Int # null count
+end
+
+Base.size(p::ValidityBitmap) = (p.ℓ,)
+nullcount(x::ValidityBitmap) = x.nc
+
+function ValidityBitmap(x)
+    T = eltype(x)
+    if !(T >: Missing)
+        return ValidityBitmap(UInt8[], 1, length(x), 0)
+    end
+    len = length(x)
+    blen = cld(len, 8)
+    bytes = Vector{UInt8}(undef, blen)
+    st = iterate(x)
+    i = 0
+    nc = 0
+    for k = 1:blen
+        b = 0x00
+        for j = 1:8
+            if (i + j) <= len
+                y, state = st
+                if y === missing
+                    nc += 1
+                    b = setbit(b, false, j)
+                else
+                    b = setbit(b, true, j)
+                end
+                st = iterate(x, state)
+            end
+        end
+        i += 8
+        @inbounds bytes[k] = b
+    end
+    return ValidityBitmap(nc == 0 ? UInt8[] : bytes, 1, nc == 0 ? 0 : len, nc)
+end
+
+@propagate_inbounds function Base.getindex(p::ValidityBitmap, i::Integer)
+    # no boundscheck because parent array should do it
+    # if a validity bitmap is empty, it either means:
+    #   1) the parent array null_count is 0, so all elements are valid
+    #   2) parent array is also empty, so "all" elements are valid
+    p.nc == 0 && return true
+    # translate element index to bitpacked byte index
+    a, b = fldmod1(i, 8)
+    @inbounds byte = p.bytes[p.pos + a - 1]
+    # check individual bit of byte
+    return getbit(byte, b)
+end
+
+@propagate_inbounds function Base.setindex!(p::ValidityBitmap, v, i::Integer)
+    x = convert(Bool, v)
+    p.ℓ == 0 && !x && throw(BoundsError(p, i))
+    a, b = fldmod1(i, 8)
+    @inbounds byte = p.bytes[p.pos + a - 1]
+    @inbounds p.bytes[p.pos + a - 1] = setbit(byte, x, b)
+    return v
+end
+
+function writebitmap(io, col::ArrowVector, alignment)
+    v = col.validity
+    @debug 1 "writing validity bitmap: nc = $(v.nc), n = $(cld(v.ℓ, 8))"
+    v.nc == 0 && return 0
+    n = Base.write(io, view(v.bytes, v.pos:(v.pos + cld(v.ℓ, 8) - 1)))
+    return n + writezeros(io, paddinglength(n, alignment))
+end
+
+include("compressed.jl")
+include("primitive.jl")
+include("bool.jl")
+include("list.jl")
+include("fixedsizelist.jl")
+include("map.jl")
+include("struct.jl")
+include("unions.jl")
+include("dictencoding.jl")
diff --git a/julia/Arrow/src/arraytypes/bool.jl b/julia/Arrow/src/arraytypes/bool.jl
new file mode 100644
index 0000000..0792903
--- /dev/null
+++ b/julia/Arrow/src/arraytypes/bool.jl
@@ -0,0 +1,112 @@
+# 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.
+
+"""
+    Arrow.BoolVector
+
+A bit-packed array type, similar to [`ValidityBitmap`](@ref), but which
+holds boolean values, `true` or `false`.
+"""
+struct BoolVector{T} <: ArrowVector{T}
+    arrow::Vector{UInt8} # need to hold a reference to arrow memory blob
+    pos::Int
+    validity::ValidityBitmap
+    ℓ::Int64
+    metadata::Union{Nothing, Dict{String, String}}
+end
+
+Base.size(p::BoolVector) = (p.ℓ,)
+
+@propagate_inbounds function Base.getindex(p::BoolVector{T}, i::Integer) where {T}
+    @boundscheck checkbounds(p, i)
+    if T >: Missing
+        @inbounds !p.validity[i] && return missing
+    end
+    a, b = fldmod1(i, 8)
+    @inbounds byte = p.arrow[p.pos + a - 1]
+    # check individual bit of byte
+    return getbit(byte, b)
+end
+
+@propagate_inbounds function Base.setindex!(p::BoolVector, v, i::Integer)
+    @boundscheck checkbounds(p, i)
+    x = convert(Bool, v)
+    a, b = fldmod1(i, 8)
+    @inbounds byte = p.arrow[p.pos + a - 1]
+    @inbounds p.arrow[p.pos + a - 1] = setbit(byte, x, b)
+    return v
+end
+
+arrowvector(::BoolType, x::BoolVector, i, nl, fi, de, ded, meta; kw...) = x
+
+function arrowvector(::BoolType, x, i, nl, fi, de, ded, meta; kw...)
+    validity = ValidityBitmap(x)
+    len = length(x)
+    blen = cld(len, 8)
+    bytes = Vector{UInt8}(undef, blen)
+    st = iterate(x)
+    i = 0
+    for k = 1:blen
+        b = 0x00
+        for j = 1:8
+            if (i + j) <= len
+                y, state = st
+                if y === missing || !y
+                    b = setbit(b, false, j)
+                else
+                    b = setbit(b, true, j)
+                end
+                st = iterate(x, state)
+            end
+        end
+        i += 8
+        @inbounds bytes[k] = b
+    end
+    return BoolVector{eltype(x)}(bytes, 1, validity, len, meta)
+end
+
+function compress(Z::Meta.CompressionType, comp, p::P) where {P <: BoolVector}
+    len = length(p)
+    nc = nullcount(p)
+    validity = compress(Z, comp, p.validity)
+    data = compress(Z, comp, view(p.arrow, p.pos:(p.pos + cld(p.ℓ, 8) - 1)))
+    return Compressed{Z, P}(p, [validity, data], len, nc, Compressed[])
+end
+
+function makenodesbuffers!(col::BoolVector, fieldnodes, fieldbuffers, bufferoffset, alignment)
+    len = length(col)
+    nc = nullcount(col)
+    push!(fieldnodes, FieldNode(len, nc))
+    @debug 1 "made field node: nodeidx = $(length(fieldnodes)), col = $(typeof(col)), len = $(fieldnodes[end].length), nc = $(fieldnodes[end].null_count)"
+    # validity bitmap
+    blen = nc == 0 ? 0 : bitpackedbytes(len, alignment)
+    push!(fieldbuffers, Buffer(bufferoffset, blen))
+    @debug 1 "made field buffer: bufferidx = $(length(fieldbuffers)), offset = $(fieldbuffers[end].offset), len = $(fieldbuffers[end].length), padded = $(padding(fieldbuffers[end].length, alignment))"
+    # adjust buffer offset, make primitive array buffer
+    bufferoffset += blen
+    blen = bitpackedbytes(len, alignment)
+    push!(fieldbuffers, Buffer(bufferoffset, blen))
+    @debug 1 "made field buffer: bufferidx = $(length(fieldbuffers)), offset = $(fieldbuffers[end].offset), len = $(fieldbuffers[end].length), padded = $(padding(fieldbuffers[end].length, alignment))"
+    return bufferoffset + blen
+end
+
+function writebuffer(io, col::BoolVector, alignment)
+    @debug 1 "writebuffer: col = $(typeof(col))"
+    @debug 2 col
+    writebitmap(io, col, alignment)
+    n = Base.write(io, view(col.arrow, col.pos:(col.pos + cld(col.ℓ, 8) - 1)))
+    return n + writezeros(io, paddinglength(n, alignment))
+end
diff --git a/julia/Arrow/src/arraytypes/compressed.jl b/julia/Arrow/src/arraytypes/compressed.jl
new file mode 100644
index 0000000..5f8e67e
--- /dev/null
+++ b/julia/Arrow/src/arraytypes/compressed.jl
@@ -0,0 +1,90 @@
+# 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.
+
+struct CompressedBuffer
+    data::Vector{UInt8}
+    uncompressedlength::Int64
+end
+
+"""
+    Arrow.Compressed
+
+Represents the compressed version of an [`ArrowVector`](@ref).
+Holds a reference to the original column. May have `Compressed`
+children for nested array types.
+"""
+struct Compressed{Z, A}
+    data::A
+    buffers::Vector{CompressedBuffer}
+    len::Int64
+    nullcount::Int64
+    children::Vector{Compressed}
+end
+
+Base.length(c::Compressed) = c.len
+Base.eltype(c::Compressed{Z, A}) where {Z, A} = eltype(A)
+getmetadata(x::Compressed) = getmetadata(x.data)
+compressiontype(c::Compressed{Z}) where {Z} = Z
+
+function compress(Z::Meta.CompressionType, comp, x::Array)
+    GC.@preserve x begin
+        y = unsafe_wrap(Array, convert(Ptr{UInt8}, pointer(x)), sizeof(x))
+        return CompressedBuffer(transcode(comp, y), length(y))
+    end
+end
+
+compress(Z::Meta.CompressionType, comp, x) = compress(Z, comp, convert(Array, x))
+
+compress(Z::Meta.CompressionType, comp, v::ValidityBitmap) =
+    v.nc == 0 ? CompressedBuffer(UInt8[], 0) : compress(Z, comp, view(v.bytes, v.pos:(v.pos + cld(v.ℓ, 8) - 1)))
+
+function makenodesbuffers!(col::Compressed, fieldnodes, fieldbuffers, bufferoffset, alignment)
+    push!(fieldnodes, FieldNode(col.len, col.nullcount))
+    @debug 1 "made field node: nodeidx = $(length(fieldnodes)), col = $(typeof(col)), len = $(fieldnodes[end].length), nc = $(fieldnodes[end].null_count)"
+    for buffer in col.buffers
+        blen = length(buffer.data) == 0 ? 0 : 8 + length(buffer.data)
+        push!(fieldbuffers, Buffer(bufferoffset, blen))
+        @debug 1 "made field buffer: bufferidx = $(length(fieldbuffers)), offset = $(fieldbuffers[end].offset), len = $(fieldbuffers[end].length), padded = $(padding(fieldbuffers[end].length, alignment))"
+        bufferoffset += padding(blen, alignment)
+    end
+    for child in col.children
+        bufferoffset = makenodesbuffers!(child, fieldnodes, fieldbuffers, bufferoffset, alignment)
+    end
+    return bufferoffset
+end
+
+function writearray(io, b::CompressedBuffer)
+    if length(b.data) > 0
+        n = Base.write(io, b.uncompressedlength)
+        @debug 1 "writing compressed buffer: uncompressedlength = $(b.uncompressedlength), n = $(length(b.data))"
+        @debug 2 b.data
+        return n + Base.write(io, b.data)
+    end
+    return 0
+end
+
+function writebuffer(io, col::Compressed, alignment)
+    @debug 1 "writebuffer: col = $(typeof(col))"
+    @debug 2 col
+    for buffer in col.buffers
+        n = writearray(io, buffer)
+        writezeros(io, paddinglength(n, alignment))
+    end
+    for child in col.children
+        writebuffer(io, child, alignment)
+    end
+    return
+end
diff --git a/julia/Arrow/src/arraytypes/dictencoding.jl b/julia/Arrow/src/arraytypes/dictencoding.jl
new file mode 100644
index 0000000..af12447
--- /dev/null
+++ b/julia/Arrow/src/arraytypes/dictencoding.jl
@@ -0,0 +1,253 @@
+# 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.
+
+"""
+    Arrow.DictEncoding
+
+Represents the "pool" of possible values for a [`DictEncoded`](@ref)
+array type. Whether the order of values is significant can be checked
+by looking at the `isOrdered` boolean field.
+"""
+mutable struct DictEncoding{T, A} <: ArrowVector{T}
+    id::Int64
+    data::A
+    isOrdered::Bool
+end
+
+Base.size(d::DictEncoding) = size(d.data)
+
+@propagate_inbounds function Base.getindex(d::DictEncoding{T}, i::Integer) where {T}
+    @boundscheck checkbounds(d, i)
+    return @inbounds ArrowTypes.arrowconvert(T, d.data[i])
+end
+
+# convenience wrapper to signal that an input column should be
+# dict encoded when written to the arrow format
+struct DictEncodeType{T} end
+getT(::Type{DictEncodeType{T}}) where {T} = T
+
+"""
+    Arrow.DictEncode(::AbstractVector, id::Integer=nothing)
+
+Signals that a column/array should be dictionary encoded when serialized
+to the arrow streaming/file format. An optional `id` number may be provided
+to signal that multiple columns should use the same pool when being
+dictionary encoded.
+"""
+struct DictEncode{T, A} <: AbstractVector{DictEncodeType{T}}
+    id::Int64
+    data::A
+end
+
+DictEncode(x::A, id=-1) where {A} = DictEncode{eltype(A), A}(id, x)
+Base.IndexStyle(::Type{<:DictEncode}) = Base.IndexLinear()
+Base.size(x::DictEncode) = (length(x.data),)
+Base.iterate(x::DictEncode, st...) = iterate(x.data, st...)
+Base.getindex(x::DictEncode, i::Int) = getindex(x.data, i)
+ArrowTypes.ArrowType(::Type{<:DictEncodeType}) = DictEncodedType()
+
+"""
+    Arrow.DictEncoded
+
+A dictionary encoded array type (similar to a `PooledArray`). Behaves just
+like a normal array in most respects; internally, possible values are stored
+in the `encoding::DictEncoding` field, while the `indices::Vector{<:Integer}`
+field holds the "codes" of each element for indexing into the encoding pool.
+Any column/array can be dict encoding when serializing to the arrow format
+either by passing the `dictencode=true` keyword argument to [`Arrow.write`](@ref)
+(which causes _all_ columns to be dict encoded), or wrapping individual columns/
+arrays in [`Arrow.DictEncode(x)`](@ref).
+"""
+struct DictEncoded{T, S, A} <: ArrowVector{T}
+    arrow::Vector{UInt8} # need to hold a reference to arrow memory blob
+    validity::ValidityBitmap
+    indices::Vector{S}
+    encoding::DictEncoding{T, A}
+    metadata::Union{Nothing, Dict{String, String}}
+end
+
+DictEncoded(b::Vector{UInt8}, v::ValidityBitmap, inds::Vector{S}, encoding::DictEncoding{T, A}, meta) where {S, T, A} =
+    DictEncoded{T, S, A}(b, v, inds, encoding, meta)
+
+Base.size(d::DictEncoded) = size(d.indices)
+
+isdictencoded(d::DictEncoded) = true
+isdictencoded(x) = false
+isdictencoded(c::Compressed{Z, A}) where {Z, A <: DictEncoded} = true
+
+signedtype(::Type{UInt8}) = Int8
+signedtype(::Type{UInt16}) = Int16
+signedtype(::Type{UInt32}) = Int32
+signedtype(::Type{UInt64}) = Int64
+
+indtype(d::DictEncoded{T, S, A}) where {T, S, A} = S
+indtype(c::Compressed{Z, A}) where {Z, A <: DictEncoded} = indtype(c.data)
+
+dictencodeid(colidx, nestedlevel, fieldid) = (Int64(nestedlevel) << 48) | (Int64(fieldid) << 32) | Int64(colidx)
+
+getid(d::DictEncoded) = d.encoding.id
+getid(c::Compressed{Z, A}) where {Z, A <: DictEncoded} = c.data.encoding.id
+
+arrowvector(::DictEncodedType, x::DictEncoded, i, nl, fi, de, ded, meta; kw...) = x
+
+function arrowvector(::DictEncodedType, x, i, nl, fi, de, ded, meta; dictencode::Bool=false, dictencodenested::Bool=false, kw...)
+    @assert x isa DictEncode
+    id = x.id == -1 ? dictencodeid(i, nl, fi) : x.id
+    x = x.data
+    len = length(x)
+    validity = ValidityBitmap(x)
+    if !haskey(de, id)
+        # dict encoding doesn't exist yet, so create for 1st time
+        if DataAPI.refarray(x) === x
+            # need to encode ourselves
+            x = PooledArray(x, encodingtype(length(x)))
+            inds = DataAPI.refarray(x)
+        else
+            inds = copy(DataAPI.refarray(x))
+        end
+        # adjust to "offset" instead of index
+        for i = 1:length(inds)
+            @inbounds inds[i] -= 1
+        end
+        pool = DataAPI.refpool(x)
+        # horrible hack? yes. better than taking CategoricalArrays dependency? also yes.
+        if typeof(pool).name.name == :CategoricalRefPool
+            pool = [get(pool[i]) for i = 1:length(pool)]
+        end
+        data = arrowvector(pool, i, nl, fi, de, ded, nothing; dictencode=dictencodenested, dictencodenested=dictencodenested, dictencoding=true, kw...)
+        encoding = DictEncoding{eltype(data), typeof(data)}(id, data, false)
+        de[id] = Lockable(encoding)
+    else
+        # encoding already exists
+          # compute inds based on it
+          # if value doesn't exist in encoding, push! it
+          # also add to deltas updates
+        encodinglockable = de[id]
+        @lock encodinglockable begin
+            encoding = encodinglockable.x
+            pool = Dict(a => (b - 1) for (b, a) in enumerate(encoding))
+            deltas = eltype(x)[]
+            len = length(x)
+            inds = Vector{encodingtype(len)}(undef, len)
+            categorical = typeof(x).name.name == :CategoricalArray
+            for (j, val) in enumerate(x)
+                if categorical
+                    val = get(val)
+                end
+                @inbounds inds[j] = get!(pool, val) do
+                    push!(deltas, val)
+                    length(pool)
+                end
+            end
+            if !isempty(deltas)
+                data = arrowvector(deltas, i, nl, fi, de, ded, nothing; dictencode=dictencodenested, dictencodenested=dictencodenested, dictencoding=true, kw...)
+                push!(ded, DictEncoding{eltype(data), typeof(data)}(id, data, false))
+                if typeof(encoding.data) <: ChainedVector
+                    append!(encoding.data, data)
+                else
+                    data2 = ChainedVector([encoding.data, data])
+                    encoding = DictEncoding{eltype(data2), typeof(data2)}(id, data2, false)
+                    de[id] = Lockable(encoding)
+                end
+            end
+        end
+    end
+    if meta !== nothing && data.metadata !== nothing
+        merge!(meta, data.metadata)
+    elseif data.metadata !== nothing
+        meta = data.metadata
+    end
+    return DictEncoded(UInt8[], validity, inds, encoding, meta)
+end
+
+@propagate_inbounds function Base.getindex(d::DictEncoded, i::Integer)
+    @boundscheck checkbounds(d, i)
+    @inbounds valid = d.validity[i]
+    !valid && return missing
+    @inbounds idx = d.indices[i]
+    return @inbounds d.encoding[idx + 1]
+end
+
+@propagate_inbounds function Base.setindex!(d::DictEncoded{T}, v, i::Integer) where {T}
+    @boundscheck checkbounds(d, i)
+    if v === missing
+        @inbounds d.validity[i] = false
+    else
+        ix = findfirst(d.encoding.data, v)
+        if ix === nothing
+            push!(d.encoding.data, v)
+            @inbounds d.indices[i] = length(d.encoding.data) - 1
+        else
+            @inbounds d.indices[i] = ix - 1
+        end
+    end
+    return v
+end
+
+function Base.copy(x::DictEncoded{T, S}) where {T, S}
+    pool = copy(x.encoding.data)
+    valid = x.validity
+    inds = x.indices
+    if T >: Missing
+        refs = Vector{S}(undef, length(inds))
+        @inbounds for i = 1:length(inds)
+            refs[i] = ifelse(valid[i], inds[i] + one(S), missing)
+        end
+    else
+        refs = copy(inds)
+        @inbounds for i = 1:length(inds)
+            refs[i] = refs[i] + one(S)
+        end
+    end
+    return PooledArray(PooledArrays.RefArray(refs), Dict{T, S}(val => i for (i, val) in enumerate(pool)), pool)
+end
+
+function compress(Z::Meta.CompressionType, comp, x::A) where {A <: DictEncoded}
+    len = length(x)
+    nc = nullcount(x)
+    validity = compress(Z, comp, x.validity)
+    inds = compress(Z, comp, x.indices)
+    return Compressed{Z, A}(x, [validity, inds], len, nc, Compressed[])
+end
+
+function makenodesbuffers!(col::DictEncoded{T, S}, fieldnodes, fieldbuffers, bufferoffset, alignment) where {T, S}
+    len = length(col)
+    nc = nullcount(col)
+    push!(fieldnodes, FieldNode(len, nc))
+    @debug 1 "made field node: nodeidx = $(length(fieldnodes)), col = $(typeof(col)), len = $(fieldnodes[end].length), nc = $(fieldnodes[end].null_count)"
+    # validity bitmap
+    blen = nc == 0 ? 0 : bitpackedbytes(len, alignment)
+    push!(fieldbuffers, Buffer(bufferoffset, blen))
+    @debug 1 "made field buffer: bufferidx = $(length(fieldbuffers)), offset = $(fieldbuffers[end].offset), len = $(fieldbuffers[end].length), padded = $(padding(fieldbuffers[end].length, alignment))"
+    bufferoffset += blen
+    # indices
+    blen = sizeof(S) * len
+    push!(fieldbuffers, Buffer(bufferoffset, blen))
+    @debug 1 "made field buffer: bufferidx = $(length(fieldbuffers)), offset = $(fieldbuffers[end].offset), len = $(fieldbuffers[end].length), padded = $(padding(fieldbuffers[end].length, alignment))"
+    bufferoffset += padding(blen, alignment)
+    return bufferoffset
+end
+
+function writebuffer(io, col::DictEncoded, alignment)
+    @debug 1 "writebuffer: col = $(typeof(col))"
+    @debug 2 col
+    writebitmap(io, col, alignment)
+    # write indices
+    n = writearray(io, col.indices)
+    @debug 1 "writing array: col = $(typeof(col.indices)), n = $n, padded = $(padding(n, alignment))"
+    writezeros(io, paddinglength(n, alignment))
+    return
+end
diff --git a/julia/Arrow/src/arraytypes/fixedsizelist.jl b/julia/Arrow/src/arraytypes/fixedsizelist.jl
new file mode 100644
index 0000000..9393049
--- /dev/null
+++ b/julia/Arrow/src/arraytypes/fixedsizelist.jl
@@ -0,0 +1,153 @@
+# 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.
+
+"""
+    Arrow.FixedSizeList
+
+An `ArrowVector` where each element is a "fixed size" list of some kind, like a `NTuple{N, T}`.
+"""
+struct FixedSizeList{T, A <: AbstractVector} <: ArrowVector{T}
+    arrow::Vector{UInt8} # need to hold a reference to arrow memory blob
+    validity::ValidityBitmap
+    data::A
+    ℓ::Int
+    metadata::Union{Nothing, Dict{String, String}}
+end
+
+Base.size(l::FixedSizeList) = (l.ℓ,)
+
+@propagate_inbounds function Base.getindex(l::FixedSizeList{T}, i::Integer) where {T}
+    @boundscheck checkbounds(l, i)
+    N = ArrowTypes.getsize(Base.nonmissingtype(T))
+    off = (i - 1) * N
+    if Base.nonmissingtype(T) !== T
+        return l.validity[i] ? ArrowTypes.arrowconvert(T, ntuple(j->l.data[off + j], N)) : missing
+    else
+        return ArrowTypes.arrowconvert(T, ntuple(j->l.data[off + j], N))
+    end
+end
+
+@propagate_inbounds function Base.setindex!(l::FixedSizeList{T}, v::T, i::Integer) where {T}
+    @boundscheck checkbounds(l, i)
+    if v === missing
+        @inbounds l.validity[i] = false
+    else
+        N = ArrowTypes.getsize(Base.nonmissingtype(T))
+        off = (i - 1) * N
+        foreach(1:N) do j
+            @inbounds l.data[off + j] = v[j]
+        end
+    end
+    return v
+end
+
+# lazy equal-spaced flattener
+struct ToFixedSizeList{T, N, A} <: AbstractVector{T}
+    data::A # A is AbstractVector of AbstractVector or AbstractString
+end
+
+function ToFixedSizeList(input)
+    NT = Base.nonmissingtype(eltype(input)) # typically NTuple{N, T}
+    return ToFixedSizeList{ArrowTypes.gettype(NT), ArrowTypes.getsize(NT), typeof(input)}(input)
+end
+
+Base.IndexStyle(::Type{<:ToFixedSizeList}) = Base.IndexLinear()
+Base.size(x::ToFixedSizeList{T, N}) where {T, N} = (N * length(x.data),)
+
+Base.@propagate_inbounds function Base.getindex(A::ToFixedSizeList{T, N}, i::Integer) where {T, N}
+    @boundscheck checkbounds(A, i)
+    a, b = fldmod1(i, N)
+    @inbounds x = A.data[a]
+    return @inbounds x === missing ? ArrowTypes.default(T) : x[b]
+end
+
+# efficient iteration
+@inline function Base.iterate(A::ToFixedSizeList{T, N}, (i, chunk, chunk_i, len)=(1, 1, 1, length(A))) where {T, N}
+    i > len && return nothing
+    @inbounds y = A.data[chunk]
+    @inbounds x = y === missing ? ArrowTypes.default(T) : y[chunk_i]
+    if chunk_i == N
+        chunk += 1
+        chunk_i = 1
+    else
+        chunk_i += 1
+    end
+    return x, (i + 1, chunk, chunk_i, len)
+end
+
+arrowvector(::FixedSizeListType, x::FixedSizeList, i, nl, fi, de, ded, meta; kw...) = x
+
+function arrowvector(::FixedSizeListType, x, i, nl, fi, de, ded, meta; kw...)
+    len = length(x)
+    validity = ValidityBitmap(x)
+    flat = ToFixedSizeList(x)
+    if eltype(flat) == UInt8
+        data = flat
+    else
+        data = arrowvector(flat, i, nl + 1, fi, de, ded, nothing; kw...)
+    end
+    return FixedSizeList{eltype(x), typeof(data)}(UInt8[], validity, data, len, meta)
+end
+
+function compress(Z::Meta.CompressionType, comp, x::FixedSizeList{T, A}) where {T, A}
+    len = length(x)
+    nc = nullcount(x)
+    validity = compress(Z, comp, x.validity)
+    buffers = [validity]
+    children = Compressed[]
+    if eltype(A) == UInt8
+        push!(buffers, compress(Z, comp, x.data))
+    else
+        push!(children, compress(Z, comp, x.data))
+    end
+    return Compressed{Z, typeof(x)}(x, buffers, len, nc, children)
+end
+
+function makenodesbuffers!(col::FixedSizeList{T, A}, fieldnodes, fieldbuffers, bufferoffset, alignment) where {T, A}
+    len = length(col)
+    nc = nullcount(col)
+    push!(fieldnodes, FieldNode(len, nc))
+    @debug 1 "made field node: nodeidx = $(length(fieldnodes)), col = $(typeof(col)), len = $(fieldnodes[end].length), nc = $(fieldnodes[end].null_count)"
+    # validity bitmap
+    blen = nc == 0 ? 0 : bitpackedbytes(len, alignment)
+    push!(fieldbuffers, Buffer(bufferoffset, blen))
+    @debug 1 "made field buffer: bufferidx = $(length(fieldbuffers)), offset = $(fieldbuffers[end].offset), len = $(fieldbuffers[end].length), padded = $(padding(fieldbuffers[end].length, alignment))"
+    bufferoffset += blen
+    if eltype(A) === UInt8
+        blen = ArrowTypes.getsize(Base.nonmissingtype(T)) * len
+        push!(fieldbuffers, Buffer(bufferoffset, blen))
+        @debug 1 "made field buffer: bufferidx = $(length(fieldbuffers)), offset = $(fieldbuffers[end].offset), len = $(fieldbuffers[end].length), padded = $(padding(fieldbuffers[end].length, alignment))"
+        bufferoffset += padding(blen, alignment)
+    else
+        bufferoffset = makenodesbuffers!(col.data, fieldnodes, fieldbuffers, bufferoffset, alignment)
+    end
+    return bufferoffset
+end
+
+function writebuffer(io, col::FixedSizeList{T, A}, alignment) where {T, A}
+    @debug 1 "writebuffer: col = $(typeof(col))"
+    @debug 2 col
+    writebitmap(io, col, alignment)
+    # write values array
+    if eltype(A) === UInt8
+        n = writearray(io, UInt8, col.data)
+        @debug 1 "writing array: col = $(typeof(col.data)), n = $n, padded = $(padding(n, alignment))"
+        writezeros(io, paddinglength(n, alignment))
+    else
+        writebuffer(io, col.data, alignment)
+    end
+    return
+end
diff --git a/julia/Arrow/src/arraytypes/list.jl b/julia/Arrow/src/arraytypes/list.jl
new file mode 100644
index 0000000..c975bb3
--- /dev/null
+++ b/julia/Arrow/src/arraytypes/list.jl
@@ -0,0 +1,209 @@
+# 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.
+
+struct Offsets{T <: Union{Int32, Int64}} <: ArrowVector{Tuple{T, T}}
+    arrow::Vector{UInt8} # need to hold a reference to arrow memory blob
+    offsets::Vector{T}
+end
+
+Base.size(o::Offsets) = (length(o.offsets) - 1,)
+
+@propagate_inbounds function Base.getindex(o::Offsets, i::Integer)
+    @boundscheck checkbounds(o, i)
+    @inbounds lo = o.offsets[i] + 1
+    @inbounds hi = o.offsets[i + 1]
+    return lo, hi
+end
+
+"""
+    Arrow.List
+
+An `ArrowVector` where each element is a variable sized list of some kind, like an `AbstractVector` or `AbstractString`.
+"""
+struct List{T, O, A} <: ArrowVector{T}
+    arrow::Vector{UInt8} # need to hold a reference to arrow memory blob
+    validity::ValidityBitmap
+    offsets::Offsets{O}
+    data::A
+    ℓ::Int
+    metadata::Union{Nothing, Dict{String, String}}
+end
+
+Base.size(l::List) = (l.ℓ,)
+
+@propagate_inbounds function Base.getindex(l::List{T}, i::Integer) where {T}
+    @boundscheck checkbounds(l, i)
+    @inbounds lo, hi = l.offsets[i]
+    if ArrowTypes.isstringtype(T)
+        if Base.nonmissingtype(T) !== T
+            return l.validity[i] ? ArrowTypes.arrowconvert(T, unsafe_string(pointer(l.data, lo), hi - lo + 1)) : missing
+        else
+            return ArrowTypes.arrowconvert(T, unsafe_string(pointer(l.data, lo), hi - lo + 1))
+        end
+    elseif Base.nonmissingtype(T) !== T
+        return l.validity[i] ? ArrowTypes.arrowconvert(T, view(l.data, lo:hi)) : missing
+    else
+        return ArrowTypes.arrowconvert(T, view(l.data, lo:hi))
+    end
+end
+
+# @propagate_inbounds function Base.setindex!(l::List{T}, v, i::Integer) where {T}
+
+# end
+
+# an AbstractVector version of Iterators.flatten
+# code based on SentinelArrays.ChainedVector
+struct ToList{T, stringtype, A, I} <: AbstractVector{T}
+    data::Vector{A} # A is AbstractVector or AbstractString
+    inds::Vector{I}
+end
+
+function ToList(input; largelists::Bool=false)
+    AT = eltype(input)
+    ST = Base.nonmissingtype(AT)
+    stringtype = ArrowTypes.isstringtype(ST)
+    T = stringtype ? UInt8 : eltype(ST)
+    len = stringtype ? ncodeunits : length
+    data = AT[]
+    I = largelists ? Int64 : Int32
+    inds = I[0]
+    sizehint!(data, length(input))
+    sizehint!(inds, length(input))
+    totalsize = I(0)
+    for x in input
+        if x === missing
+            push!(data, missing)
+        else
+            push!(data, x)
+            totalsize += len(x)
+            if I === Int32 && totalsize > 2147483647
+                I = Int64
+                inds = convert(Vector{Int64}, inds)
+            end
+        end
+        push!(inds, totalsize)
+    end
+    return ToList{T, stringtype, AT, I}(data, inds)
+end
+
+Base.IndexStyle(::Type{<:ToList}) = Base.IndexLinear()
+Base.size(x::ToList) = (length(x.inds) == 0 ? 0 : x.inds[end],)
+
+function Base.pointer(A::ToList{UInt8}, i::Integer)
+    chunk = searchsortedfirst(A.inds, i)
+    return pointer(A.data[chunk - 1])
+end
+
+@inline function index(A::ToList, i::Integer)
+    chunk = searchsortedfirst(A.inds, i)
+    return chunk - 1, i - (@inbounds A.inds[chunk - 1])
+end
+
+Base.@propagate_inbounds function Base.getindex(A::ToList{T, stringtype}, i::Integer) where {T, stringtype}
+    @boundscheck checkbounds(A, i)
+    chunk, ix = index(A, i)
+    @inbounds x = A.data[chunk]
+    return @inbounds stringtype ? codeunits(x)[ix] : x[ix]
+end
+
+Base.@propagate_inbounds function Base.setindex!(A::ToList{T, stringtype}, v, i::Integer) where {T, stringtype}
+    @boundscheck checkbounds(A, i)
+    chunk, ix = index(A, i)
+    @inbounds x = A.data[chunk]
+    if stringtype
+        codeunits(x)[ix] = v
+    else
+        x[ix] = v
+    end
+    return v
+end
+
+# efficient iteration
+@inline function Base.iterate(A::ToList{T, stringtype}) where {T, stringtype}
+    length(A) == 0 && return nothing
+    i = 1
+    chunk = 2
+    chunk_i = 1
+    chunk_len = A.inds[chunk]
+    while i > chunk_len
+        chunk += 1
+        chunk_len = A.inds[chunk]
+    end
+    val = A.data[chunk - 1]
+    x = stringtype ? codeunits(val)[1] : val[1]
+    # find next valid index
+    i += 1
+    if i > chunk_len
+        while true
+            chunk += 1
+            chunk > length(A.inds) && break
+            chunk_len = A.inds[chunk]
+            i <= chunk_len && break
+        end
+    else
+        chunk_i += 1
+    end
+    return x, (i, chunk, chunk_i, chunk_len, length(A))
+end
+
+@inline function Base.iterate(A::ToList{T, stringtype}, (i, chunk, chunk_i, chunk_len, len)) where {T, stringtype}
+    i > len && return nothing
+    @inbounds val = A.data[chunk - 1]
+    @inbounds x = stringtype ? codeunits(val)[chunk_i] : val[chunk_i]
+    i += 1
+    if i > chunk_len
+        chunk_i = 1
+        while true
+            chunk += 1
+            chunk > length(A.inds) && break
+            @inbounds chunk_len = A.inds[chunk]
+            i <= chunk_len && break
+        end
+    else
+        chunk_i += 1
+    end
+    return x, (i, chunk, chunk_i, chunk_len, len)
+end
+
+arrowvector(::ListType, x::List, i, nl, fi, de, ded, meta; kw...) = x
+
+function arrowvector(::ListType, x, i, nl, fi, de, ded, meta; largelists::Bool=false, kw...)
+    len = length(x)
+    validity = ValidityBitmap(x)
+    flat = ToList(x; largelists=largelists)
+    offsets = Offsets(UInt8[], flat.inds)
+    if eltype(flat) == UInt8 # binary or utf8string
+        data = flat
+    else
+        data = arrowvector(flat, i, nl + 1, fi, de, ded, nothing; lareglists=largelists, kw...)
+    end
+    return List{eltype(x), eltype(flat.inds), typeof(data)}(UInt8[], validity, offsets, data, len, meta)
+end
+
+function compress(Z::Meta.CompressionType, comp, x::List{T, O, A}) where {T, O, A}
+    len = length(x)
+    nc = nullcount(x)
+    validity = compress(Z, comp, x.validity)
+    offsets = compress(Z, comp, x.offsets.offsets)
+    buffers = [validity, offsets]
+    children = Compressed[]
+    if eltype(A) == UInt8
+        push!(buffers, compress(Z, comp, x.data))
+    else
+        push!(children, compress(Z, comp, x.data))
+    end
+    return Compressed{Z, typeof(x)}(x, buffers, len, nc, children)
+end
diff --git a/julia/Arrow/src/arraytypes/map.jl b/julia/Arrow/src/arraytypes/map.jl
new file mode 100644
index 0000000..8565172
--- /dev/null
+++ b/julia/Arrow/src/arraytypes/map.jl
@@ -0,0 +1,115 @@
+# 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.
+
+"""
+    Arrow.Map
+
+An `ArrowVector` where each element is a "map" of some kind, like a `Dict`.
+"""
+struct Map{T, O, A} <: ArrowVector{T}
+    validity::ValidityBitmap
+    offsets::Offsets{O}
+    data::A
+    ℓ::Int
+    metadata::Union{Nothing, Dict{String, String}}
+end
+
+Base.size(l::Map) = (l.ℓ,)
+
+@propagate_inbounds function Base.getindex(l::Map{T}, i::Integer) where {T}
+    @boundscheck checkbounds(l, i)
+    @inbounds lo, hi = l.offsets[i]
+    if Base.nonmissingtype(T) !== T
+        return l.validity[i] ? ArrowTypes.arrowconvert(T, Dict(x.key => x.value for x in view(l.data, lo:hi))) : missing
+    else
+        return ArrowTypes.arrowconvert(T, Dict(x.key => x.value for x in view(l.data, lo:hi)))
+    end
+end
+
+keyvalues(KT, ::Missing) = missing
+keyvalues(KT, x::AbstractDict) = [KT(k, v) for (k, v) in pairs(x)]
+
+arrowvector(::MapType, x::Map, i, nl, fi, de, ded, meta; kw...) = x
+
+function arrowvector(::MapType, x, i, nl, fi, de, ded, meta; largelists::Bool=false, kw...)
+    len = length(x)
+    validity = ValidityBitmap(x)
+    ET = eltype(x)
+    DT = Base.nonmissingtype(ET)
+    KT = KeyValue{keytype(DT), valtype(DT)}
+    VT = Vector{KT}
+    T = DT !== ET ? Union{Missing, VT} : VT
+    flat = ToList(T[keyvalues(KT, y) for y in x]; largelists=largelists)
+    offsets = Offsets(UInt8[], flat.inds)
+    data = arrowvector(flat, i, nl + 1, fi, de, ded, nothing; lareglists=largelists, kw...)
+    return Map{ET, eltype(flat.inds), typeof(data)}(validity, offsets, data, len, meta)
+end
+
+function compress(Z::Meta.CompressionType, comp, x::A) where {A <: Map}
+    len = length(x)
+    nc = nullcount(x)
+    validity = compress(Z, comp, x.validity)
+    offsets = compress(Z, comp, x.offsets.offsets)
+    buffers = [validity, offsets]
+    children = Compressed[]
+    push!(children, compress(Z, comp, x.data))
+    return Compressed{Z, A}(x, buffers, len, nc, children)
+end
+
+function makenodesbuffers!(col::Union{Map{T, O, A}, List{T, O, A}}, fieldnodes, fieldbuffers, bufferoffset, alignment) where {T, O, A}
+    len = length(col)
+    nc = nullcount(col)
+    push!(fieldnodes, FieldNode(len, nc))
+    @debug 1 "made field node: nodeidx = $(length(fieldnodes)), col = $(typeof(col)), len = $(fieldnodes[end].length), nc = $(fieldnodes[end].null_count)"
+    # validity bitmap
+    blen = nc == 0 ? 0 : bitpackedbytes(len, alignment)
+    push!(fieldbuffers, Buffer(bufferoffset, blen))
+    @debug 1 "made field buffer: bufferidx = $(length(fieldbuffers)), offset = $(fieldbuffers[end].offset), len = $(fieldbuffers[end].length), padded = $(padding(fieldbuffers[end].length, alignment))"
+    # adjust buffer offset, make array buffer
+    bufferoffset += blen
+    blen = sizeof(O) * (len + 1)
+    push!(fieldbuffers, Buffer(bufferoffset, blen))
+    @debug 1 "made field buffer: bufferidx = $(length(fieldbuffers)), offset = $(fieldbuffers[end].offset), len = $(fieldbuffers[end].length), padded = $(padding(fieldbuffers[end].length, alignment))"
+    bufferoffset += padding(blen, alignment)
+    if eltype(A) == UInt8
+        blen = length(col.data)
+        push!(fieldbuffers, Buffer(bufferoffset, blen))
+        @debug 1 "made field buffer: bufferidx = $(length(fieldbuffers)), offset = $(fieldbuffers[end].offset), len = $(fieldbuffers[end].length), padded = $(padding(fieldbuffers[end].length, alignment))"
+        bufferoffset += padding(blen, alignment)
+    else
+        bufferoffset = makenodesbuffers!(col.data, fieldnodes, fieldbuffers, bufferoffset, alignment)
+    end
+    return bufferoffset
+end
+
+function writebuffer(io, col::Union{Map{T, O, A}, List{T, O, A}}, alignment) where {T, O, A}
+    @debug 1 "writebuffer: col = $(typeof(col))"
+    @debug 2 col
+    writebitmap(io, col, alignment)
+    # write offsets
+    n = writearray(io, O, col.offsets.offsets)
+    @debug 1 "writing array: col = $(typeof(col.offsets.offsets)), n = $n, padded = $(padding(n, alignment))"
+    writezeros(io, paddinglength(n, alignment))
+    # write values array
+    if eltype(A) == UInt8
+        n = writearray(io, UInt8, col.data)
+        @debug 1 "writing array: col = $(typeof(col.data)), n = $n, padded = $(padding(n, alignment))"
+        writezeros(io, paddinglength(n, alignment))
+    else
+        writebuffer(io, col.data, alignment)
+    end
+    return
+end
diff --git a/julia/Arrow/src/arraytypes/primitive.jl b/julia/Arrow/src/arraytypes/primitive.jl
new file mode 100644
index 0000000..fc7c79a
--- /dev/null
+++ b/julia/Arrow/src/arraytypes/primitive.jl
@@ -0,0 +1,106 @@
+# 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.
+
+"""
+    Arrow.Primitive
+
+An `ArrowVector` where each element is a "fixed size" scalar of some kind, like an integer, float, decimal, or time type.
+"""
+struct Primitive{T, A} <: ArrowVector{T}
+    arrow::Vector{UInt8} # need to hold a reference to arrow memory blob
+    validity::ValidityBitmap
+    data::A
+    ℓ::Int64
+    metadata::Union{Nothing, Dict{String, String}}
+end
+
+Primitive(::Type{T}, b::Vector{UInt8}, v::ValidityBitmap, data::A, l, meta) where {T, A} =
+    Primitive{T, A}(b, v, data, l, meta)
+
+Base.size(p::Primitive) = (p.ℓ,)
+
+function Base.copy(p::Primitive)
+    if nullcount(p) == 0
+        return copy(p.data)
+    else
+        return convert(Array, p)
+    end
+end
+
+@propagate_inbounds function Base.getindex(p::Primitive{T}, i::Integer) where {T}
+    @boundscheck checkbounds(p, i)
+    if T >: Missing
+        return @inbounds (p.validity[i] ? ArrowTypes.arrowconvert(T, p.data[i]) : missing)
+    else
+        return @inbounds ArrowTypes.arrowconvert(T, p.data[i])
+    end
+end
+
+@propagate_inbounds function Base.setindex!(p::Primitive{T}, v, i::Integer) where {T}
+    @boundscheck checkbounds(p, i)
+    if T >: Missing
+        if v === missing
+            @inbounds p.validity[i] = false
+        else
+            @inbounds p.data[i] = convert(Base.nonmissingtype(T), v)
+        end
+    else
+        @inbounds p.data[i] = convert(Base.nonmissingtype(T), v)
+    end
+    return v
+end
+
+arrowvector(::PrimitiveType, x::Primitive, i, nl, fi, de, ded, meta; kw...) = x
+
+function arrowvector(::PrimitiveType, x, i, nl, fi, de, ded, meta; kw...)
+    validity = ValidityBitmap(x)
+    return Primitive(eltype(x), UInt8[], validity, x, length(x), meta)
+end
+
+function compress(Z::Meta.CompressionType, comp, p::P) where {P <: Primitive}
+    len = length(p)
+    nc = nullcount(p)
+    validity = compress(Z, comp, p.validity)
+    data = compress(Z, comp, p.data)
+    return Compressed{Z, P}(p, [validity, data], len, nc, Compressed[])
+end
+
+function makenodesbuffers!(col::Primitive{T}, fieldnodes, fieldbuffers, bufferoffset, alignment) where {T}
+    len = length(col)
+    nc = nullcount(col)
+    push!(fieldnodes, FieldNode(len, nc))
+    @debug 1 "made field node: nodeidx = $(length(fieldnodes)), col = $(typeof(col)), len = $(fieldnodes[end].length), nc = $(fieldnodes[end].null_count)"
+    # validity bitmap
+    blen = nc == 0 ? 0 : bitpackedbytes(len, alignment)
+    push!(fieldbuffers, Buffer(bufferoffset, blen))
+    @debug 1 "made field buffer: bufferidx = $(length(fieldbuffers)), offset = $(fieldbuffers[end].offset), len = $(fieldbuffers[end].length), padded = $(padding(fieldbuffers[end].length, alignment))"
+    # adjust buffer offset, make primitive array buffer
+    bufferoffset += blen
+    blen = len * sizeof(Base.nonmissingtype(T))
+    push!(fieldbuffers, Buffer(bufferoffset, blen))
+    @debug 1 "made field buffer: bufferidx = $(length(fieldbuffers)), offset = $(fieldbuffers[end].offset), len = $(fieldbuffers[end].length), padded = $(padding(fieldbuffers[end].length, alignment))"
+    return bufferoffset + padding(blen, alignment)
+end
+
+function writebuffer(io, col::Primitive{T}, alignment) where {T}
+    @debug 1 "writebuffer: col = $(typeof(col))"
+    @debug 2 col
+    writebitmap(io, col, alignment)
+    n = writearray(io, Base.nonmissingtype(T), col.data)
+    @debug 1 "writing array: col = $(typeof(col.data)), n = $n, padded = $(padding(n, alignment))"
+    writezeros(io, paddinglength(n, alignment))
+    return
+end
diff --git a/julia/Arrow/src/arraytypes/struct.jl b/julia/Arrow/src/arraytypes/struct.jl
new file mode 100644
index 0000000..96f6758
--- /dev/null
+++ b/julia/Arrow/src/arraytypes/struct.jl
@@ -0,0 +1,125 @@
+# 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.
+
+"""
+    Arrow.Struct
+
+An `ArrowVector` where each element is a "struct" of some kind with ordered, named fields, like a `NamedTuple{names, types}` or regular julia `struct`.
+"""
+struct Struct{T, S} <: ArrowVector{T}
+    validity::ValidityBitmap
+    data::S # Tuple of ArrowVector
+    ℓ::Int
+    metadata::Union{Nothing, Dict{String, String}}
+end
+
+Base.size(s::Struct) = (s.ℓ,)
+
+@propagate_inbounds function Base.getindex(s::Struct{T}, i::Integer) where {T}
+    @boundscheck checkbounds(s, i)
+    NT = Base.nonmissingtype(T)
+    if ArrowTypes.structtype(NT) === ArrowTypes.NAMEDTUPLE
+        if NT !== T
+            return s.validity[i] ? NT(ntuple(j->s.data[j][i], fieldcount(NT))) : missing
+        else
+            return NT(ntuple(j->s.data[j][i], fieldcount(NT)))
+        end
+    elseif ArrowTypes.structtype(NT) === ArrowTypes.STRUCT
+        if NT !== T
+            return s.validity[i] ? NT(ntuple(j->s.data[j][i], fieldcount(NT))...) : missing
+        else
+            return NT(ntuple(j->s.data[j][i], fieldcount(NT))...)
+        end
+    end
+end
+
+@propagate_inbounds function Base.setindex!(s::Struct{T}, v::T, i::Integer) where {T}
+    @boundscheck checkbounds(s, i)
+    if v === missing
+        @inbounds s.validity[i] = false
+    else
+        NT = Base.nonmissingtype(T)
+        N = fieldcount(NT)
+        foreach(1:N) do j
+            @inbounds s.data[j][i] = getfield(v, j)
+        end
+    end
+    return v
+end
+
+struct ToStruct{T, i, A} <: AbstractVector{T}
+    data::A # eltype is NamedTuple or some struct
+end
+
+ToStruct(x::A, j::Integer) where {A} = ToStruct{fieldtype(Base.nonmissingtype(eltype(A)), j), j, A}(x)
+
+Base.IndexStyle(::Type{<:ToStruct}) = Base.IndexLinear()
+Base.size(x::ToStruct) = (length(x.data),)
+
+Base.@propagate_inbounds function Base.getindex(A::ToStruct{T, j}, i::Integer) where {T, j}
+    @boundscheck checkbounds(A, i)
+    @inbounds x = A.data[i]
+    return x === missing ? ArrowTypes.default(T) : getfield(x, j)
+end
+
+arrowvector(::StructType, x::Struct, i, nl, fi, de, ded, meta; kw...) = x
+
+function arrowvector(::StructType, x, i, nl, fi, de, ded, meta; kw...)
+    len = length(x)
+    validity = ValidityBitmap(x)
+    T = Base.nonmissingtype(eltype(x))
+    data = Tuple(arrowvector(ToStruct(x, j), i, nl + 1, j, de, ded, nothing; kw...) for j = 1:fieldcount(T))
+    return Struct{eltype(x), typeof(data)}(validity, data, len, meta)
+end
+
+function compress(Z::Meta.CompressionType, comp, x::A) where {A <: Struct}
+    len = length(x)
+    nc = nullcount(x)
+    validity = compress(Z, comp, x.validity)
+    buffers = [validity]
+    children = Compressed[]
+    for y in x.data
+        push!(children, compress(Z, comp, y))
+    end
+    return Compressed{Z, A}(x, buffers, len, nc, children)
+end
+
+function makenodesbuffers!(col::Struct{T}, fieldnodes, fieldbuffers, bufferoffset, alignment) where {T}
+    len = length(col)
+    nc = nullcount(col)
+    push!(fieldnodes, FieldNode(len, nc))
+    @debug 1 "made field node: nodeidx = $(length(fieldnodes)), col = $(typeof(col)), len = $(fieldnodes[end].length), nc = $(fieldnodes[end].null_count)"
+    # validity bitmap
+    blen = nc == 0 ? 0 : bitpackedbytes(len, alignment)
+    push!(fieldbuffers, Buffer(bufferoffset, blen))
+    @debug 1 "made field buffer: bufferidx = $(length(fieldbuffers)), offset = $(fieldbuffers[end].offset), len = $(fieldbuffers[end].length), padded = $(padding(fieldbuffers[end].length, alignment))"
+    bufferoffset += blen
+    for child in col.data
+        bufferoffset = makenodesbuffers!(child, fieldnodes, fieldbuffers, bufferoffset, alignment)
+    end
+    return bufferoffset
+end
+
+function writebuffer(io, col::Struct, alignment)
+    @debug 1 "writebuffer: col = $(typeof(col))"
+    @debug 2 col
+    writebitmap(io, col, alignment)
+    # write values arrays
+    for child in col.data
+        writebuffer(io, child, alignment)
+    end
+    return
+end
diff --git a/julia/Arrow/src/arraytypes/unions.jl b/julia/Arrow/src/arraytypes/unions.jl
new file mode 100644
index 0000000..64d8630
--- /dev/null
+++ b/julia/Arrow/src/arraytypes/unions.jl
@@ -0,0 +1,279 @@
+# 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.
+
+# Union arrays
+# need a custom representation of Union types since arrow unions
+# are ordered, and possibly indirected via separate typeIds array
+# here, T is Meta.UnionMode.Dense or Meta.UnionMode.Sparse,
+# typeIds is a NTuple{N, Int32}, and U is a Tuple{...} of the
+# unioned types
+struct UnionT{T, typeIds, U}
+end
+
+unionmode(::Type{UnionT{T, typeIds, U}}) where {T, typeIds, U} = T
+typeids(::Type{UnionT{T, typeIds, U}}) where {T, typeIds, U} = typeIds
+Base.eltype(::Type{UnionT{T, typeIds, U}}) where {T, typeIds, U} = U
+
+ArrowTypes.ArrowType(::Type{<:UnionT}) = ArrowTypes.UnionType()
+
+# iterate a Julia Union{...} type, producing an array of unioned types
+function eachunion(U::Union, elems=nothing)
+    if elems === nothing
+        return eachunion(U.b, Type[U.a])
+    else
+        push!(elems, U.a)
+        return eachunion(U.b, elems)
+    end
+end
+
+function eachunion(T, elems)
+    push!(elems, T)
+    return elems
+end
+
+# produce typeIds, offsets, data tuple for DenseUnion
+isatypeid(x::T, ::Type{types}) where {T, types} = isatypeid(x, fieldtype(types, 1), types, 1)
+isatypeid(x::T, ::Type{S}, ::Type{types}, i) where {T, S, types} = x isa S ? i : isatypeid(x, fieldtype(types, i + 1), types, i + 1)
+
+"""
+    Arrow.DenseUnion
+
+An `ArrowVector` where the type of each element is one of a fixed set of types, meaning its eltype is like a julia `Union{type1, type2, ...}`.
+An `Arrow.DenseUnion`, in comparison to `Arrow.SparseUnion`, stores elements in a set of arrays, one array per possible type, and an "offsets"
+array, where each offset element is the index into one of the typed arrays. This allows a sort of "compression", where no extra space is
+used/allocated to store all the elements.
+"""
+struct DenseUnion{T, S} <: ArrowVector{T}
+    arrow::Vector{UInt8} # need to hold a reference to arrow memory blob
+    arrow2::Vector{UInt8} # if arrow blob is compressed, need a 2nd reference for uncompressed offsets bytes
+    typeIds::Vector{UInt8}
+    offsets::Vector{Int32}
+    data::S # Tuple of ArrowVector
+    metadata::Union{Nothing, Dict{String, String}}
+end
+
+Base.size(s::DenseUnion) = size(s.typeIds)
+nullcount(x::DenseUnion) = 0 # DenseUnion has no validity bitmap; only children do
+
+@propagate_inbounds function Base.getindex(s::DenseUnion{T}, i::Integer) where {T}
+    @boundscheck checkbounds(s, i)
+    @inbounds typeId = s.typeIds[i]
+    @inbounds off = s.offsets[i]
+    @inbounds x = s.data[typeId + 1][off + 1]
+    return x
+end
+
+@propagate_inbounds function Base.setindex!(s::DenseUnion{UnionT{T, typeIds, U}}, v, i::Integer) where {T, typeIds, U}
+    @boundscheck checkbounds(s, i)
+    @inbounds typeId = s.typeIds[i]
+    typeids = typeIds === nothing ? (0:(fieldcount(U) - 1)) : typeIds
+    vtypeId = Int8(typeids[isatypeid(v, U)])
+    if typeId == vtypeId
+        @inbounds off = s.offsets[i]
+        @inbounds s.data[typeId +1][off + 1] = v
+    else
+        throw(ArgumentError("type of item to set $(typeof(v)) must match existing item $(fieldtype(U, typeid))"))
+    end
+    return v
+end
+
+# convenience wrappers for signaling that an array shoudld be written
+# as with dense/sparse union arrow buffers
+struct DenseUnionVector{T, U} <: AbstractVector{UnionT{Meta.UnionMode.Dense, nothing, U}}
+    itr::T
+end
+
+DenseUnionVector(x::T) where {T} = DenseUnionVector{T, Tuple{eachunion(eltype(x))...}}(x)
+Base.IndexStyle(::Type{<:DenseUnionVector}) = Base.IndexLinear()
+Base.size(x::DenseUnionVector) = (length(x.itr),)
+Base.iterate(x::DenseUnionVector, st...) = iterate(x.itr, st...)
+Base.getindex(x::DenseUnionVector, i::Int) = getindex(x.itr, i)
+
+function todense(::Type{UnionT{T, typeIds, U}}, x) where {T, typeIds, U}
+    typeids = typeIds === nothing ? (0:(fieldcount(U) - 1)) : typeIds
+    len = length(x)
+    types = Vector{UInt8}(undef, len)
+    offsets = Vector{Int32}(undef, len)
+    data = Tuple(Vector{i == 1 ? Union{Missing, fieldtype(U, i)} : fieldtype(U, i)}(undef, 0) for i = 1:fieldcount(U))
+    for (i, y) in enumerate(x)
+        typeid = y === missing ? 0x00 : UInt8(typeids[isatypeid(y, U)])
+        @inbounds types[i] = typeid
+        @inbounds offsets[i] = length(data[typeid + 1])
+        push!(data[typeid + 1], y)
+    end
+    return types, offsets, data
+end
+
+struct SparseUnionVector{T, U} <: AbstractVector{UnionT{Meta.UnionMode.Sparse, nothing, U}}
+    itr::T
+end
+
+SparseUnionVector(x::T) where {T} = SparseUnionVector{T, Tuple{eachunion(eltype(x))...}}(x)
+Base.IndexStyle(::Type{<:SparseUnionVector}) = Base.IndexLinear()
+Base.size(x::SparseUnionVector) = (length(x.itr),)
+Base.iterate(x::SparseUnionVector, st...) = iterate(x.itr, st...)
+Base.getindex(x::SparseUnionVector, i::Int) = getindex(x.itr, i)
+
+# sparse union child array producer
+# for sparse unions, we split the parent array into
+# N children arrays, each having the same length as the parent
+# but with one child array per unioned type; each child
+# should include the elements from parent of its type
+# and other elements can be missing/default
+function sparsetypeids(::Type{UnionT{T, typeIds, U}}, x) where {T, typeIds, U}
+    typeids = typeIds === nothing ? (0:(fieldcount(U) - 1)) : typeIds
+    len = length(x)
+    types = Vector{UInt8}(undef, len)
+    for (i, y) in enumerate(x)
+        typeid = y === missing ? 0x00 : UInt8(typeids[isatypeid(y, U)])
+        @inbounds types[i] = typeid
+    end
+    return types
+end
+
+struct ToSparseUnion{T, A} <: AbstractVector{T}
+    data::A
+end
+
+ToSparseUnion(::Type{T}, data::A) where {T, A} = ToSparseUnion{T, A}(data)
+
+Base.IndexStyle(::Type{<:ToSparseUnion}) = Base.IndexLinear()
+Base.size(x::ToSparseUnion) = (length(x.data),)
+
+Base.@propagate_inbounds function Base.getindex(A::ToSparseUnion{T}, i::Integer) where {T}
+    @boundscheck checkbounds(A, i)
+    @inbounds x = A.data[i]
+    return @inbounds x isa T ? x : ArrowTypes.default(T)
+end
+
+function compress(Z::Meta.CompressionType, comp, x::A) where {A <: DenseUnion}
+    len = length(x)
+    nc = nullcount(x)
+    typeIds = compress(Z, comp, x.typeIds)
+    offsets = compress(Z, comp, x.offsets)
+    buffers = [typeIds, offsets]
+    children = Compressed[]
+    for y in x.data
+        push!(children, compress(Z, comp, y))
+    end
+    return Compressed{Z, A}(x, buffers, len, nc, children)
+end
+
+"""
+    Arrow.SparseUnion
+
+An `ArrowVector` where the type of each element is one of a fixed set of types, meaning its eltype is like a julia `Union{type1, type2, ...}`.
+An `Arrow.SparseUnion`, in comparison to `Arrow.DenseUnion`, stores elements in a set of arrays, one array per possible type, and each typed
+array has the same length as the full array. This ends up with "wasted" space, since only one slot among the typed arrays is valid per full
+array element, but can allow for certain optimizations when each typed array has the same length.
+"""
+struct SparseUnion{T, S} <: ArrowVector{T}
+    arrow::Vector{UInt8} # need to hold a reference to arrow memory blob
+    typeIds::Vector{UInt8}
+    data::S # Tuple of ArrowVector
+    metadata::Union{Nothing, Dict{String, String}}
+end
+
+Base.size(s::SparseUnion) = size(s.typeIds)
+nullcount(x::SparseUnion) = 0
+
+@propagate_inbounds function Base.getindex(s::SparseUnion{T}, i::Integer) where {T}
+    @boundscheck checkbounds(s, i)
+    @inbounds typeId = s.typeIds[i]
+    @inbounds x = s.data[typeId + 1][i]
+    return x
+end
+
+@propagate_inbounds function Base.setindex!(s::SparseUnion{UnionT{T, typeIds, U}}, v, i::Integer) where {T, typeIds, U}
+    @boundscheck checkbounds(s, i)
+    typeids = typeIds === nothing ? (0:(fieldcount(U) - 1)) : typeIds
+    vtypeId = Int8(typeids[isatypeid(v, U)])
+    @inbounds s.typeIds[i] = vtypeId
+    @inbounds s.data[vtypeId + 1][i] = v
+    return v
+end
+
+arrowvector(U::Union, x, i, nl, fi, de, ded, meta; denseunions::Bool=true, kw...) =
+    arrowvector(denseunions ? DenseUnionVector(x) : SparseUnionVector(x), i, nl, fi, de, ded, meta; denseunions=denseunions, kw...)
+
+arrowvector(::UnionType, x::Union{DenseUnion, SparseUnion}, i, nl, fi, de, ded, meta; kw...) = x
+
+function arrowvector(::UnionType, x, i, nl, fi, de, ded, meta; kw...)
+    UT = eltype(x)
+    if unionmode(UT) == Meta.UnionMode.Dense
+        x = x isa DenseUnionVector ? x.itr : x
+        typeids, offsets, data = todense(UT, x)
+        data2 = map(y -> arrowvector(y[2], i, nl + 1, y[1], de, ded, nothing; kw...), enumerate(data))
+        return DenseUnion{UT, typeof(data2)}(UInt8[], UInt8[], typeids, offsets, data2, meta)
+    else
+        x = x isa SparseUnionVector ? x.itr : x
+        typeids = sparsetypeids(UT, x)
+        data3 = Tuple(arrowvector(ToSparseUnion(fieldtype(eltype(UT), j), x), i, nl + 1, j, de, ded, nothing; kw...) for j = 1:fieldcount(eltype(UT)))
+        return SparseUnion{UT, typeof(data3)}(UInt8[], typeids, data3, meta)
+    end
+end
+
+function compress(Z::Meta.CompressionType, comp, x::A) where {A <: SparseUnion}
+    len = length(x)
+    nc = nullcount(x)
+    typeIds = compress(Z, comp, x.typeIds)
+    buffers = [typeIds]
+    children = Compressed[]
+    for y in x.data
+        push!(children, compress(Z, comp, y))
+    end
+    return Compressed{Z, A}(x, buffers, len, nc, children)
+end
+
+function makenodesbuffers!(col::Union{DenseUnion, SparseUnion}, fieldnodes, fieldbuffers, bufferoffset, alignment)
+    len = length(col)
+    nc = nullcount(col)
+    push!(fieldnodes, FieldNode(len, nc))
+    @debug 1 "made field node: nodeidx = $(length(fieldnodes)), col = $(typeof(col)), len = $(fieldnodes[end].length), nc = $(fieldnodes[end].null_count)"
+    # typeIds buffer
+    push!(fieldbuffers, Buffer(bufferoffset, len))
+    @debug 1 "made field buffer: bufferidx = $(length(fieldbuffers)), offset = $(fieldbuffers[end].offset), len = $(fieldbuffers[end].length), padded = $(padding(fieldbuffers[end].length, alignment))"
+    bufferoffset += padding(len, alignment)
+    if col isa DenseUnion
+        # offsets buffer
+        blen = sizeof(Int32) * len
+        push!(fieldbuffers, Buffer(bufferoffset, blen))
+        @debug 1 "made field buffer: bufferidx = $(length(fieldbuffers)), offset = $(fieldbuffers[end].offset), len = $(fieldbuffers[end].length), padded = $(padding(fieldbuffers[end].length, alignment))"
+        bufferoffset += padding(blen, alignment)
+    end
+    for child in col.data
+        bufferoffset = makenodesbuffers!(child, fieldnodes, fieldbuffers, bufferoffset, alignment)
+    end
+    return bufferoffset
+end
+
+function writebuffer(io, col::Union{DenseUnion, SparseUnion}, alignment)
+    @debug 1 "writebuffer: col = $(typeof(col))"
+    @debug 2 col
+    # typeIds buffer
+    n = writearray(io, UInt8, col.typeIds)
+    @debug 1 "writing array: col = $(typeof(col.typeIds)), n = $n, padded = $(padding(n, alignment))"
+    writezeros(io, paddinglength(n, alignment))
+    if col isa DenseUnion
+        n = writearray(io, Int32, col.offsets)
+        @debug 1 "writing array: col = $(typeof(col.offsets)), n = $n, padded = $(padding(n, alignment))"
+        writezeros(io, paddinglength(n, alignment))
+    end
+    for child in col.data
+        writebuffer(io, child, alignment)
+    end
+    return
+end
diff --git a/julia/Arrow/src/arrowtypes.jl b/julia/Arrow/src/arrowtypes.jl
new file mode 100644
index 0000000..2e9bf4c
--- /dev/null
+++ b/julia/Arrow/src/arrowtypes.jl
@@ -0,0 +1,153 @@
+# 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.
+
+"""
+The ArrowTypes module provides the [`ArrowTypes.Arrowtype`](@ref) interface trait that objects can define
+in order to signal how they should be serialized in the arrow format.
+"""
+module ArrowTypes
+
+export ArrowType, NullType, PrimitiveType, BoolType, ListType, FixedSizeListType, MapType, StructType, UnionType, DictEncodedType
+
+abstract type ArrowType end
+
+ArrowType(x::T) where {T} = ArrowType(T)
+ArrowType(::Type{T}) where {T} = isprimitivetype(T) ? PrimitiveType() : StructType()
+
+function arrowconvert end
+
+arrowconvert(T, x) = convert(T, x)
+arrowconvert(::Type{Union{T, Missing}}, x) where {T} = arrowconvert(T, x)
+arrowconvert(::Type{Union{T, Missing}}, ::Missing) where {T} = missing
+
+struct NullType <: ArrowType end
+
+ArrowType(::Type{Missing}) = NullType()
+
+struct PrimitiveType <: ArrowType end
+
+ArrowType(::Type{<:Integer}) = PrimitiveType()
+ArrowType(::Type{<:AbstractFloat}) = PrimitiveType()
+
+struct BoolType <: ArrowType end
+ArrowType(::Type{Bool}) = BoolType()
+
+struct ListType <: ArrowType end
+
+# isstringtype MUST BE UTF8 (other codeunit sizes not supported; arrow encoding for strings is specifically UTF8)
+isstringtype(T) = false
+isstringtype(::Type{Union{T, Missing}}) where {T} = isstringtype(T)
+
+ArrowType(::Type{<:AbstractString}) = ListType()
+isstringtype(::Type{<:AbstractString}) = true
+
+ArrowType(::Type{Symbol}) = ListType()
+isstringtype(::Type{Symbol}) = true
+arrowconvert(::Type{Symbol}, x::String) = Symbol(x)
+arrowconvert(::Type{String}, x::Symbol) = String(x)
+
+ArrowType(::Type{<:AbstractArray}) = ListType()
+
+struct FixedSizeListType <: ArrowType end
+
+ArrowType(::Type{NTuple{N, T}}) where {N, T} = FixedSizeListType()
+gettype(::Type{NTuple{N, T}}) where {N, T} = T
+getsize(::Type{NTuple{N, T}}) where {N, T} = N
+
+struct StructType <: ArrowType end
+
+ArrowType(::Type{<:NamedTuple}) = StructType()
+
+@enum STRUCT_TYPES NAMEDTUPLE STRUCT # KEYWORDARGS
+
+structtype(::Type{NamedTuple{N, T}}) where {N, T} = NAMEDTUPLE
+structtype(::Type{T}) where {T} = STRUCT
+
+# must implement keytype, valtype
+struct MapType <: ArrowType end
+
+ArrowType(::Type{<:AbstractDict}) = MapType()
+
+struct UnionType <: ArrowType end
+
+ArrowType(::Union) = UnionType()
+
+struct DictEncodedType <: ArrowType end
+
+"""
+There are a couple places when writing arrow buffers where
+we need to write a "dummy" value; it doesn't really matter
+what we write, but we need to write something of a specific
+type. So each supported writing type needs to define `default`.
+"""
+function default end
+
+default(T) = zero(T)
+default(::Type{Symbol}) = Symbol()
+default(::Type{Char}) = '\0'
+default(::Type{<:AbstractString}) = ""
+default(::Type{Union{T, Missing}}) where {T} = default(T)
+
+function default(::Type{A}) where {A <: AbstractVector{T}} where {T}
+    a = similar(A, 1)
+    a[1] = default(T)
+    return a
+end
+
+default(::Type{NTuple{N, T}}) where {N, T} = ntuple(i -> default(T), N)
+default(::Type{T}) where {T <: Tuple} = Tuple(default(fieldtype(T, i)) for i = 1:fieldcount(T))
+default(::Type{Dict{K, V}}) where {K, V} = Dict{K, V}()
+default(::Type{NamedTuple{names, types}}) where {names, types} = NamedTuple{names}(Tuple(default(fieldtype(types, i)) for i = 1:length(names)))
+
+const JULIA_TO_ARROW_TYPE_MAPPING = Dict{Type, Tuple{String, Type}}(
+    Char => ("JuliaLang.Char", UInt32),
+    Symbol => ("JuliaLang.Symbol", String),
+)
+
+istyperegistered(::Type{T}) where {T} = haskey(JULIA_TO_ARROW_TYPE_MAPPING, T)
+
+function getarrowtype!(meta, ::Type{T}) where {T}
+    arrowname, arrowtype = JULIA_TO_ARROW_TYPE_MAPPING[T]
+    meta["ARROW:extension:name"] = arrowname
+    meta["ARROW:extension:metadata"] = ""
+    return arrowtype
+end
+
+const ARROW_TO_JULIA_TYPE_MAPPING = Dict{String, Tuple{Type, Type}}(
+    "JuliaLang.Char" => (Char, UInt32),
+    "JuliaLang.Symbol" => (Symbol, String),
+)
+
+function extensiontype(meta)
+    if haskey(meta, "ARROW:extension:name")
+        typename = meta["ARROW:extension:name"]
+        if haskey(ARROW_TO_JULIA_TYPE_MAPPING, typename)
+            return ARROW_TO_JULIA_TYPE_MAPPING[typename][1]
+        else
+            @warn "unsupported ARROW:extension:name type: \"$typename\""
+        end
+    end
+    return nothing
+end
+
+function registertype!(juliatype::Type, arrowtype::Type, arrowname::String=string("JuliaLang.", string(juliatype)))
+    # TODO: validate that juliatype isn't already default arrow type
+    JULIA_TO_ARROW_TYPE_MAPPING[juliatype] = (arrowname, arrowtype)
+    ARROW_TO_JULIA_TYPE_MAPPING[arrowname] = (juliatype, arrowtype)
+    return
+end
+
+end # module ArrowTypes
\ No newline at end of file
diff --git a/julia/Arrow/src/eltypes.jl b/julia/Arrow/src/eltypes.jl
new file mode 100644
index 0000000..fbf26ab
--- /dev/null
+++ b/julia/Arrow/src/eltypes.jl
@@ -0,0 +1,408 @@
+# 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.
+
+"""
+Given a flatbuffers metadata type definition (a Field instance from Schema.fbs),
+translate to the appropriate Julia storage eltype
+"""
+function juliaeltype end
+
+finaljuliatype(T) = T
+finaljuliatype(::Type{Missing}) = Missing
+finaljuliatype(::Type{Union{T, Missing}}) where {T} = Union{Missing, finaljuliatype(T)}
+
+"""
+Given a FlatBuffers.Builder and a Julia column or column eltype,
+Write the field.type flatbuffer definition of the eltype
+"""
+function arrowtype end
+
+arrowtype(b, col::AbstractVector{T}) where {T} = arrowtype(b, maybemissing(T))
+arrowtype(b, col::DictEncoded) = arrowtype(b, col.encoding.data)
+arrowtype(b, col::Compressed) = arrowtype(b, col.data)
+
+function juliaeltype(f::Meta.Field, ::Nothing, convert::Bool)
+    T = juliaeltype(f, convert)
+    return convert ? finaljuliatype(T) : T
+end
+
+function juliaeltype(f::Meta.Field, meta::Dict{String, String}, convert::Bool)
+    TT = juliaeltype(f, convert)
+    !convert && return TT
+    T = finaljuliatype(TT)
+    TTT = ArrowTypes.extensiontype(meta)
+    return something(TTT, T)
+end
+
+function juliaeltype(f::Meta.Field, convert::Bool)
+    T = juliaeltype(f, f.type, convert)
+    return (f.nullable ? Union{T, Missing} : T)
+end
+
+juliaeltype(f::Meta.Field, ::Meta.Null, convert) = Missing
+
+function arrowtype(b, ::Type{Missing})
+    Meta.nullStart(b)
+    return Meta.Null, Meta.nullEnd(b), nothing
+end
+
+function juliaeltype(f::Meta.Field, int::Meta.Int, convert)
+    if int.is_signed
+        if int.bitWidth == 8
+            Int8
+        elseif int.bitWidth == 16
+            Int16
+        elseif int.bitWidth == 32
+            Int32
+        elseif int.bitWidth == 64
+            Int64
+        elseif int.bitWidth == 128
+            Int128
+        else
+            error("$int is not valid arrow type metadata")
+        end
+    else
+        if int.bitWidth == 8
+            UInt8
+        elseif int.bitWidth == 16
+            UInt16
+        elseif int.bitWidth == 32
+            UInt32
+        elseif int.bitWidth == 64
+            UInt64
+        elseif int.bitWidth == 128
+            UInt128
+        else
+            error("$int is not valid arrow type metadata")
+        end
+    end
+end
+
+function arrowtype(b, ::Type{T}) where {T <: Integer}
+    Meta.intStart(b)
+    Meta.intAddBitWidth(b, Int32(8 * sizeof(T)))
+    Meta.intAddIsSigned(b, T <: Signed)
+    return Meta.Int, Meta.intEnd(b), nothing
+end
+
+# primitive types
+function juliaeltype(f::Meta.Field, fp::Meta.FloatingPoint, convert)
+    if fp.precision == Meta.Precision.HALF
+        Float16
+    elseif fp.precision == Meta.Precision.SINGLE
+        Float32
+    elseif fp.precision == Meta.Precision.DOUBLE
+        Float64
+    end
+end
+
+function arrowtype(b, ::Type{T}) where {T <: AbstractFloat}
+    Meta.floatingPointStart(b)
+    Meta.floatingPointAddPrecision(b, T === Float16 ? Meta.Precision.HALF : T === Float32 ? Meta.Precision.SINGLE : Meta.Precision.DOUBLE)
+    return Meta.FloatingPoint, Meta.floatingPointEnd(b), nothing
+end
+
+juliaeltype(f::Meta.Field, b::Union{Meta.Utf8, Meta.LargeUtf8}, convert) = String
+
+datasizeof(x) = sizeof(x)
+datasizeof(x::AbstractVector) = sum(datasizeof, x)
+
+juliaeltype(f::Meta.Field, b::Union{Meta.Binary, Meta.LargeBinary}, convert) = Vector{UInt8}
+
+juliaeltype(f::Meta.Field, x::Meta.FixedSizeBinary, convert) = NTuple{Int(x.byteWidth), UInt8}
+
+# arggh!
+Base.write(io::IO, x::NTuple{N, T}) where {N, T} = sum(y -> Base.write(io, y), x)
+
+juliaeltype(f::Meta.Field, x::Meta.Bool, convert) = Bool
+
+function arrowtype(b, ::Type{Bool})
+    Meta.boolStart(b)
+    return Meta.Bool, Meta.boolEnd(b), nothing
+end
+
+struct Decimal{P, S}
+    value::Int128
+end
+
+Base.zero(::Type{Decimal{P, S}}) where {P, S} = Decimal{P, S}(Int128(0))
+==(a::Decimal{P, S}, b::Decimal{P, S}) where {P, S} = ==(a.value, b.value)
+Base.isequal(a::Decimal{P, S}, b::Decimal{P, S}) where {P, S} = isequal(a.value, b.value)
+
+function juliaeltype(f::Meta.Field, x::Meta.Decimal, convert)
+    return Decimal{x.precision, x.scale}
+end
+
+ArrowTypes.ArrowType(::Type{<:Decimal}) = PrimitiveType()
+
+function arrowtype(b, ::Type{Decimal{P, S}}) where {P, S}
+    Meta.decimalStart(b)
+    Meta.decimalAddPrecision(b, Int32(P))
+    Meta.decimalAddScale(b, Int32(S))
+    return Meta.Decimal, Meta.decimalEnd(b), nothing
+end
+
+Base.write(io::IO, x::Decimal) = Base.write(io, x.value)
+
+abstract type ArrowTimeType end
+Base.write(io::IO, x::ArrowTimeType) = Base.write(io, x.x)
+ArrowTypes.ArrowType(::Type{<:ArrowTimeType}) = PrimitiveType()
+
+struct Date{U, T} <: ArrowTimeType
+    x::T
+end
+
+Base.zero(::Type{Date{U, T}}) where {U, T} = Date{U, T}(T(0))
+storagetype(::Type{Date{U, T}}) where {U, T} = T
+bitwidth(x::Meta.DateUnit) = x == Meta.DateUnit.DAY ? Int32 : Int64
+Date{Meta.DateUnit.DAY}(days) = Date{Meta.DateUnit.DAY, Int32}(Int32(days))
+Date{Meta.DateUnit.MILLISECOND}(ms) = Date{Meta.DateUnit.MILLISECOND, Int64}(Int64(ms))
+const DATE = Date{Meta.DateUnit.DAY, Int32}
+const DATETIME = Date{Meta.DateUnit.MILLISECOND, Int64}
+
+juliaeltype(f::Meta.Field, x::Meta.Date, convert) = Date{x.unit, bitwidth(x.unit)}
+finaljuliatype(::Type{Date{Meta.DateUnit.DAY, Int32}}) = Dates.Date
+Base.convert(::Type{Dates.Date}, x::Date{Meta.DateUnit.DAY, Int32}) = Dates.Date(Dates.UTD(Int64(x.x + UNIX_EPOCH_DATE)))
+finaljuliatype(::Type{Date{Meta.DateUnit.MILLISECOND, Int64}}) = Dates.DateTime
+Base.convert(::Type{Dates.DateTime}, x::Date{Meta.DateUnit.MILLISECOND, Int64}) = Dates.DateTime(Dates.UTM(Int64(x.x + UNIX_EPOCH_DATETIME)))
+
+function arrowtype(b, ::Type{Date{U, T}}) where {U, T}
+    Meta.dateStart(b)
+    Meta.dateAddUnit(b, U)
+    return Meta.Date, Meta.dateEnd(b), nothing
+end
+
+const UNIX_EPOCH_DATE = Dates.value(Dates.Date(1970))
+Base.convert(::Type{Date{Meta.DateUnit.DAY, Int32}}, x::Dates.Date) = Date{Meta.DateUnit.DAY, Int32}(Int32(Dates.value(x) - UNIX_EPOCH_DATE))
+
+const UNIX_EPOCH_DATETIME = Dates.value(Dates.DateTime(1970))
+Base.convert(::Type{Date{Meta.DateUnit.MILLISECOND, Int64}}, x::Dates.DateTime) = Date{Meta.DateUnit.MILLISECOND, Int64}(Int64(Dates.value(x) - UNIX_EPOCH_DATETIME))
+
+struct Time{U, T} <: ArrowTimeType
+    x::T
+end
+
+Base.zero(::Type{Time{U, T}}) where {U, T} = Time{U, T}(T(0))
+const TIME = Time{Meta.TimeUnit.NANOSECOND, Int64}
+
+bitwidth(x::Meta.TimeUnit) = x == Meta.TimeUnit.SECOND || x == Meta.TimeUnit.MILLISECOND ? Int32 : Int64
+Time{U}(x) where {U <: Meta.TimeUnit} = Time{U, bitwidth(U)}(bitwidth(U)(x))
+storagetype(::Type{Time{U, T}}) where {U, T} = T
+juliaeltype(f::Meta.Field, x::Meta.Time, convert) = Time{x.unit, bitwidth(x.unit)}
+finaljuliatype(::Type{<:Time}) = Dates.Time
+periodtype(U::Meta.TimeUnit) = U === Meta.TimeUnit.SECOND ? Dates.Second :
+                               U === Meta.TimeUnit.MILLISECOND ? Dates.Millisecond :
+                               U === Meta.TimeUnit.MICROSECOND ? Dates.Microsecond : Dates.Nanosecond
+Base.convert(::Type{Dates.Time}, x::Time{U, T}) where {U, T} = Dates.Time(Dates.Nanosecond(Dates.tons(periodtype(U)(x.x))))
+
+function arrowtype(b, ::Type{Time{U, T}}) where {U, T}
+    Meta.timeStart(b)
+    Meta.timeAddUnit(b, U)
+    Meta.timeAddBitWidth(b, Int32(8 * sizeof(T)))
+    return Meta.Time, Meta.timeEnd(b), nothing
+end
+
+Base.convert(::Type{Time{Meta.TimeUnit.NANOSECOND, Int64}}, x::Dates.Time) = Time{Meta.TimeUnit.NANOSECOND, Int64}(Dates.value(x))
+
+struct Timestamp{U, TZ} <: ArrowTimeType
+    x::Int64
+end
+
+Base.zero(::Type{Timestamp{U, T}}) where {U, T} = Timestamp{U, T}(Int64(0))
+
+function juliaeltype(f::Meta.Field, x::Meta.Timestamp, convert)
+    return Timestamp{x.unit, x.timezone === nothing ? nothing : Symbol(x.timezone)}
+end
+
+finaljuliatype(::Type{<:Timestamp}) = ZonedDateTime
+Base.convert(::Type{ZonedDateTime}, x::Timestamp{U, TZ}) where {U, TZ} =
+    ZonedDateTime(Dates.DateTime(Dates.UTM(Int64(Dates.toms(periodtype(U)(x.x)) + UNIX_EPOCH_DATETIME))), TimeZone(String(TZ)))
+Base.convert(::Type{Timestamp{Meta.TimeUnit.MILLISECOND, TZ}}, x::ZonedDateTime) where {TZ} =
+    Timestamp{Meta.TimeUnit.MILLISECOND, TZ}(Int64(Dates.value(DateTime(x, Local)) - UNIX_EPOCH_DATETIME))
+
+function arrowtype(b, ::Type{Timestamp{U, TZ}}) where {U, TZ}
+    tz = TZ !== nothing ? FlatBuffers.createstring!(b, String(TZ)) : FlatBuffers.UOffsetT(0)
+    Meta.timestampStart(b)
+    Meta.timestampAddUnit(b, U)
+    Meta.timestampAddTimezone(b, tz)
+    return Meta.Timestamp, Meta.timestampEnd(b), nothing
+end
+
+struct Interval{U, T} <: ArrowTimeType
+    x::T
+end
+
+Base.zero(::Type{Interval{U, T}}) where {U, T} = Interval{U, T}(T(0))
+
+bitwidth(x::Meta.IntervalUnit) = x == Meta.IntervalUnit.YEAR_MONTH ? Int32 : Int64
+Interval{Meta.IntervalUnit.YEAR_MONTH}(x) = Interval{Meta.IntervalUnit.YEAR_MONTH, Int32}(Int32(x))
+Interval{Meta.IntervalUnit.DAY_TIME}(x) = Interval{Meta.IntervalUnit.DAY_TIME, Int64}(Int64(x))
+
+function juliaeltype(f::Meta.Field, x::Meta.Interval, convert)
+    return Interval{x.unit, bitwidth(x.unit)}
+end
+
+function arrowtype(b, ::Type{Interval{U, T}}) where {U, T}
+    Meta.intervalStart(b)
+    Meta.intervalAddUnit(b, U)
+    return Meta.Interval, Meta.intervalEnd(b), nothing
+end
+
+struct Duration{U} <: ArrowTimeType
+    x::Int64
+end
+
+Base.zero(::Type{Duration{U}}) where {U} = Duration{U}(Int64(0))
+
+function juliaeltype(f::Meta.Field, x::Meta.Duration, convert)
+    return Duration{x.unit}
+end
+
+finaljuliatype(::Type{Duration{U}}) where {U} = periodtype(U)
+Base.convert(::Type{P}, x::Duration{U}) where {P <: Dates.Period, U} = P(periodtype(U)(x.x))
+
+function arrowtype(b, ::Type{Duration{U}}) where {U}
+    Meta.durationStart(b)
+    Meta.durationAddUnit(b, U)
+    return Meta.Duration, Meta.durationEnd(b), nothing
+end
+
+arrowperiodtype(P) = Meta.TimeUnit.SECOND
+arrowperiodtype(::Type{Dates.Millisecond}) = Meta.TimeUnit.MILLISECOND
+arrowperiodtype(::Type{Dates.Microsecond}) = Meta.TimeUnit.MICROSECOND
+arrowperiodtype(::Type{Dates.Nanosecond}) = Meta.TimeUnit.NANOSECOND
+
+Base.convert(::Type{Duration{U}}, x::Dates.Period) where {U} = Duration{U}(Dates.value(periodtype(U)(x)))
+
+# nested types; call juliaeltype recursively on nested children
+function juliaeltype(f::Meta.Field, list::Union{Meta.List, Meta.LargeList}, convert)
+    return Vector{juliaeltype(f.children[1], buildmetadata(f.children[1]), convert)}
+end
+
+# arrowtype will call fieldoffset recursively for children
+function arrowtype(b, x::List{T, O, A}) where {T, O, A}
+    if eltype(A) == UInt8
+        if T <: AbstractString || T <: Union{AbstractString, Missing}
+            if O == Int32
+                Meta.utf8Start(b)
+                return Meta.Utf8, Meta.utf8End(b), nothing
+            else # if O == Int64
+                Meta.largUtf8Start(b)
+                return Meta.LargeUtf8, Meta.largUtf8End(b), nothing
+            end
+        else # if Vector{UInt8}
+            if O == Int32
+                Meta.binaryStart(b)
+                return Meta.Binary, Meta.binaryEnd(b), nothing
+            else # if O == Int64
+                Meta.largeBinaryStart(b)
+                return Meta.LargeBinary, Meta.largeBinaryEnd(b), nothing
+            end
+        end
+    else
+        children = [fieldoffset(b, "", x.data)]
+        if O == Int32
+            Meta.listStart(b)
+            return Meta.List, Meta.listEnd(b), children
+        else
+            Meta.largeListStart(b)
+            return Meta.LargeList, Meta.largeListEnd(b), children
+        end
+    end
+end
+
+function juliaeltype(f::Meta.Field, list::Meta.FixedSizeList, convert)
+    type = juliaeltype(f.children[1], buildmetadata(f.children[1]), convert)
+    return NTuple{Int(list.listSize), type}
+end
+
+function arrowtype(b, x::FixedSizeList{T, A}) where {T, A}
+    N = ArrowTypes.getsize(Base.nonmissingtype(T))
+    if eltype(A) == UInt8
+        Meta.fixedSizeBinaryStart(b)
+        Meta.fixedSizeBinaryAddByteWidth(b, Int32(N))
+        return Meta.FixedSizeBinary, Meta.fixedSizeBinaryEnd(b), nothing
+    else
+        children = [fieldoffset(b, "", x.data)]
+        Meta.fixedSizeListStart(b)
+        Meta.fixedSizeListAddListSize(b, Int32(N))
+        return Meta.FixedSizeList, Meta.fixedSizeListEnd(b), children
+    end
+end
+
+function juliaeltype(f::Meta.Field, map::Meta.Map, convert)
+    K = juliaeltype(f.children[1].children[1], buildmetadata(f.children[1].children[1]), convert)
+    V = juliaeltype(f.children[1].children[2], buildmetadata(f.children[1].children[2]), convert)
+    return Dict{K, V}
+end
+
+function arrowtype(b, x::Map)
+    children = [fieldoffset(b, "entries", x.data)]
+    Meta.mapStart(b)
+    return Meta.Map, Meta.mapEnd(b), children
+end
+
+struct KeyValue{K, V}
+    key::K
+    value::V
+end
+keyvalueK(::Type{KeyValue{K, V}}) where {K, V} = K
+keyvalueV(::Type{KeyValue{K, V}}) where {K, V} = V
+Base.length(kv::KeyValue) = 1
+Base.iterate(kv::KeyValue, st=1) = st === nothing ? nothing : (kv, nothing)
+ArrowTypes.default(::Type{KeyValue{K, V}}) where {K, V} = KeyValue(default(K), default(V))
+
+function arrowtype(b, ::Type{KeyValue{K, V}}) where {K, V}
+    children = [fieldoffset(b, "key", K), fieldoffset(b, "value", V)]
+    Meta.structStart(b)
+    return Meta.Struct, Meta.structEnd(b), children
+end
+
+function juliaeltype(f::Meta.Field, list::Meta.Struct, convert)
+    names = Tuple(Symbol(x.name) for x in f.children)
+    types = Tuple(juliaeltype(x, buildmetadata(x), convert) for x in f.children)
+    return NamedTuple{names, Tuple{types...}}
+end
+
+function arrowtype(b, x::Struct{T, S}) where {T, S}
+    names = fieldnames(Base.nonmissingtype(T))
+    children = [fieldoffset(b, names[i], x.data[i]) for i = 1:length(names)]
+    Meta.structStart(b)
+    return Meta.Struct, Meta.structEnd(b), children
+end
+
+# Unions
+function juliaeltype(f::Meta.Field, u::Meta.Union, convert)
+    return UnionT{u.mode, u.typeIds !== nothing ? Tuple(u.typeIds) : u.typeIds, Tuple{(juliaeltype(x, buildmetadata(x), convert) for x in f.children)...}}
+end
+
+arrowtype(b, x::Union{DenseUnion{TT, S}, SparseUnion{TT, S}}) where {TT, S} = arrowtype(b, TT, x)
+function arrowtype(b, ::Type{UnionT{T, typeIds, U}}, x::Union{DenseUnion{TT, S}, SparseUnion{TT, S}}) where {T, typeIds, U, TT, S}
+    if typeIds !== nothing
+        Meta.unionStartTypeIdsVector(b, length(typeIds))
+        for id in Iterators.reverse(typeIds)
+            FlatBuffers.prepend!(b, id)
+        end
+        TI = FlatBuffers.endvector!(b, length(typeIds))
+    end
+    children = [fieldoffset(b, "", x.data[i]) for i = 1:fieldcount(U)]
+    Meta.unionStart(b)
+    Meta.unionAddMode(b, T)
+    if typeIds !== nothing
+        Meta.unionAddTypeIds(b, TI)
+    end
+    return Meta.Union, Meta.unionEnd(b), children
+end
diff --git a/julia/Arrow/src/metadata/File.jl b/julia/Arrow/src/metadata/File.jl
new file mode 100644
index 0000000..8c1622c
--- /dev/null
+++ b/julia/Arrow/src/metadata/File.jl
@@ -0,0 +1,90 @@
+# 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.
+
+struct Footer <: FlatBuffers.Table
+    bytes::Vector{UInt8}
+    pos::Base.Int
+end
+
+Base.propertynames(x::Footer) = (:version, :schema, :dictionaries, :recordBatches, :custom_metadata)
+
+function Base.getproperty(x::Footer, field::Symbol)
+    if field === :version
+        o = FlatBuffers.offset(x, 4)
+        o != 0 && return FlatBuffers.get(x, o + FlatBuffers.pos(x), MetadataVersion)
+        return MetadataVersion.V1
+    elseif field === :schema
+        o = FlatBuffers.offset(x, 6)
+        if o != 0
+            y = FlatBuffers.indirect(x, o + FlatBuffers.pos(x))
+            return FlatBuffers.init(Schema, FlatBuffers.bytes(x), y)
+        end
+    elseif field === :dictionaries
+        o = FlatBuffers.offset(x, 8)
+        if o != 0
+            return FlatBuffers.Array{Block}(x, o)
+        end
+    elseif field === :recordBatches
+        o = FlatBuffers.offset(x, 10)
+        if o != 0
+            return FlatBuffers.Array{Block}(x, o)
+        end
+    elseif field === :custom_metadata
+        o = FlatBuffers.offset(x, 12)
+        if o != 0
+            return FlatBuffers.Array{KeyValue}(x, o)
+        end
+    end
+    return nothing
+end
+
+footerStart(b::FlatBuffers.Builder) = FlatBuffers.startobject!(b, 4)
+footerAddVersion(b::FlatBuffers.Builder, version::MetadataVersion) = FlatBuffers.prependslot!(b, 0, version, 0)
+footerAddSchema(b::FlatBuffers.Builder, schema::FlatBuffers.UOffsetT) = FlatBuffers.prependoffsetslot!(b, 1, schema, 0)
+footerAddDictionaries(b::FlatBuffers.Builder, dictionaries::FlatBuffers.UOffsetT) = FlatBuffers.prependoffsetslot!(b, 2, dictionaries, 0)
+footerStartDictionariesVector(b::FlatBuffers.Builder, numelems) = FlatBuffers.startvector!(b, 24, numelems, 8)
+footerAddRecordBatches(b::FlatBuffers.Builder, recordbatches::FlatBuffers.UOffsetT) = FlatBuffers.prependoffsetslot!(b, 3, recordbatches, 0)
+footerStartRecordBatchesVector(b::FlatBuffers.Builder, numelems) = FlatBuffers.startvector!(b, 24, numelems, 8)
+footerEnd(b::FlatBuffers.Builder) = FlatBuffers.endobject!(b)
+
+struct Block <: FlatBuffers.Struct
+    bytes::Vector{UInt8}
+    pos::Base.Int
+end
+
+FlatBuffers.structsizeof(::Base.Type{Block}) = 24
+
+Base.propertynames(x::Block) = (:offset, :metaDataLength, :bodyLength)
+
+function Base.getproperty(x::Block, field::Symbol)
+    if field === :offset
+        return FlatBuffers.get(x, FlatBuffers.pos(x), Int64)
+    elseif field === :metaDataLength
+        return FlatBuffers.get(x, FlatBuffers.pos(x) + 8, Int32)
+    elseif field === :bodyLength
+        return FlatBuffers.get(x, FlatBuffers.pos(x) + 16, Int64)
+    end
+    return nothing
+end
+
+function createBlock(b::FlatBuffers.Builder, offset::Int64, metadatalength::Int32, bodylength::Int64)
+    FlatBuffers.prep!(b, 8, 24)
+    prepend!(b, bodylength)
+    FlatBuffers.pad!(b, 4)
+    prepend!(b, metadatalength)
+    prepend!(b, offset)
+    return FlatBuffers.offset(b)
+end
\ No newline at end of file
diff --git a/julia/Arrow/src/metadata/Flatbuf.jl b/julia/Arrow/src/metadata/Flatbuf.jl
new file mode 100644
index 0000000..a69230d
--- /dev/null
+++ b/julia/Arrow/src/metadata/Flatbuf.jl
@@ -0,0 +1,25 @@
+# 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.
+
+module Flatbuf
+
+using ..FlatBuffers
+
+include("Schema.jl")
+include("File.jl")
+include("Message.jl")
+
+end # module
\ No newline at end of file
diff --git a/julia/Arrow/src/metadata/Message.jl b/julia/Arrow/src/metadata/Message.jl
new file mode 100644
index 0000000..4fe6253
--- /dev/null
+++ b/julia/Arrow/src/metadata/Message.jl
@@ -0,0 +1,202 @@
+# 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.
+
+struct FieldNode <: FlatBuffers.Struct
+    bytes::Vector{UInt8}
+    pos::Base.Int
+end
+
+FlatBuffers.structsizeof(::Base.Type{FieldNode}) = 16
+
+Base.propertynames(x::FieldNode) = (:length, :null_count)
+
+function Base.getproperty(x::FieldNode, field::Symbol)
+    if field === :length
+        return FlatBuffers.get(x, FlatBuffers.pos(x), Int64)
+    elseif field === :null_count
+        return FlatBuffers.get(x, FlatBuffers.pos(x) + 8, Int64)
+    end
+    return nothing
+end
+
+function createFieldNode(b::FlatBuffers.Builder, length::Int64, nullCount::Int64)
+    FlatBuffers.prep!(b, 8, 16)
+    prepend!(b, nullCount)
+    prepend!(b, length)
+    return FlatBuffers.offset(b)
+end
+
+FlatBuffers.@scopedenum CompressionType::Int8 LZ4_FRAME ZSTD
+
+FlatBuffers.@scopedenum BodyCompressionMethod::Int8 BUFFER
+
+struct BodyCompression <: FlatBuffers.Table
+    bytes::Vector{UInt8}
+    pos::Base.Int
+end
+
+Base.propertynames(x::BodyCompression) = (:codec, :method)
+
+function Base.getproperty(x::BodyCompression, field::Symbol)
+    if field === :codec
+        o = FlatBuffers.offset(x, 4)
+        o != 0 && return FlatBuffers.get(x, o + FlatBuffers.pos(x), CompressionType)
+        return CompressionType.LZ4_FRAME
+    elseif field === :method
+        o = FlatBuffers.offset(x, 6)
+        o != 0 && return FlatBuffers.get(x, o + FlatBuffers.pos(x), BodyCompressionMethod)
+        return BodyCompressionMethod.BUFFER
+    end
+    return nothing
+end
+
+bodyCompressionStart(b::FlatBuffers.Builder) = FlatBuffers.startobject!(b, 2)
+bodyCompressionAddCodec(b::FlatBuffers.Builder, codec::CompressionType) = FlatBuffers.prependslot!(b, 0, codec, 0)
+bodyCompressionAddMethod(b::FlatBuffers.Builder, method::BodyCompressionMethod) = FlatBuffers.prependslot!(b, 1, method, 0)
+bodyCompressionEnd(b::FlatBuffers.Builder) = FlatBuffers.endobject!(b)
+
+struct RecordBatch <: FlatBuffers.Table
+    bytes::Vector{UInt8}
+    pos::Base.Int
+end
+
+Base.propertynames(x::RecordBatch) = (:length, :nodes, :buffers, :compression)
+
+function Base.getproperty(x::RecordBatch, field::Symbol)
+    if field === :length
+        o = FlatBuffers.offset(x, 4)
+        o != 0 && return FlatBuffers.get(x, o + FlatBuffers.pos(x), Int64)
+    elseif field === :nodes
+        o = FlatBuffers.offset(x, 6)
+        if o != 0
+            return FlatBuffers.Array{FieldNode}(x, o)
+        end
+    elseif field === :buffers
+        o = FlatBuffers.offset(x, 8)
+        if o != 0
+            return FlatBuffers.Array{Buffer}(x, o)
+        end
+    elseif field === :compression
+        o = FlatBuffers.offset(x, 10)
+        if o != 0
+            y = FlatBuffers.indirect(x, o + FlatBuffers.pos(x))
+            return FlatBuffers.init(BodyCompression, FlatBuffers.bytes(x), y)
+        end
+    end
+    return nothing
+end
+
+recordBatchStart(b::FlatBuffers.Builder) = FlatBuffers.startobject!(b, 4)
+recordBatchAddLength(b::FlatBuffers.Builder, length::Int64) = FlatBuffers.prependslot!(b, 0, length, 0)
+recordBatchAddNodes(b::FlatBuffers.Builder, nodes::FlatBuffers.UOffsetT) = FlatBuffers.prependoffsetslot!(b, 1, nodes, 0)
+recordBatchStartNodesVector(b::FlatBuffers.Builder, numelems) = FlatBuffers.startvector!(b, 16, numelems, 8)
+recordBatchAddBuffers(b::FlatBuffers.Builder, buffers::FlatBuffers.UOffsetT) = FlatBuffers.prependoffsetslot!(b, 2, buffers, 0)
+recordBatchStartBuffersVector(b::FlatBuffers.Builder, numelems) = FlatBuffers.startvector!(b, 16, numelems, 8)
+recordBatchAddCompression(b::FlatBuffers.Builder, c::FlatBuffers.UOffsetT) = FlatBuffers.prependoffsetslot!(b, 3, c, 0)
+recordBatchEnd(b::FlatBuffers.Builder) = FlatBuffers.endobject!(b)
+
+struct DictionaryBatch <: FlatBuffers.Table
+    bytes::Vector{UInt8}
+    pos::Base.Int
+end
+
+Base.propertynames(x::DictionaryBatch) = (:id, :data, :isDelta)
+
+function Base.getproperty(x::DictionaryBatch, field::Symbol)
+    if field === :id
+        o = FlatBuffers.offset(x, 4)
+        o != 0 && return FlatBuffers.get(x, o + FlatBuffers.pos(x), Int64)
+        return Int64(0)
+    elseif field === :data
+        o = FlatBuffers.offset(x, 6)
+        if o != 0
+            y = FlatBuffers.indirect(x, o + FlatBuffers.pos(x))
+            return FlatBuffers.init(RecordBatch, FlatBuffers.bytes(x), y)
+        end
+    elseif field === :isDelta
+        o = FlatBuffers.offset(x, 8)
+        o != 0 && return FlatBuffers.get(x, o + FlatBuffers.pos(x), Base.Bool)
+        return false
+    end
+    return nothing
+end
+
+dictionaryBatchStart(b::FlatBuffers.Builder) = FlatBuffers.startobject!(b, 3)
+dictionaryBatchAddId(b::FlatBuffers.Builder, id::Int64) = FlatBuffers.prependslot!(b, 0, id, 0)
+dictionaryBatchAddData(b::FlatBuffers.Builder, data::FlatBuffers.UOffsetT) = FlatBuffers.prependoffsetslot!(b, 1, data, 0)
+dictionaryBatchAddIsDelta(b::FlatBuffers.Builder, isdelta::Base.Bool) = FlatBuffers.prependslot!(b, 2, isdelta, false)
+dictionaryBatchEnd(b::FlatBuffers.Builder) = FlatBuffers.endobject!(b)
+
+function MessageHeader(b::UInt8)
+    b == 1 && return Schema
+    b == 2 && return DictionaryBatch
+    b == 3 && return RecordBatch
+    # b == 4 && return Tensor
+    # b == 5 && return SparseTensor
+    return nothing
+end
+
+function MessageHeader(::Base.Type{T})::Int16 where {T}
+    T == Schema && return 1
+    T == DictionaryBatch && return 2
+    T == RecordBatch && return 3
+    # T == Tensor && return 4
+    # T == SparseTensor && return 5
+    return 0
+end
+
+struct Message <: FlatBuffers.Table
+    bytes::Vector{UInt8}
+    pos::Base.Int
+end
+
+Base.propertynames(x::Message) = (:version, :header, :bodyLength, :custom_metadata)
+
+function Base.getproperty(x::Message, field::Symbol)
+    if field === :version
+        o = FlatBuffers.offset(x, 4)
+        o != 0 && return FlatBuffers.get(x, o + FlatBuffers.pos(x), MetadataVersion)
+    elseif field === :header
+        o = FlatBuffers.offset(x, 6)
+        if o != 0
+            T = MessageHeader(FlatBuffers.get(x, o + FlatBuffers.pos(x), UInt8))
+            o = FlatBuffers.offset(x, 8)
+            pos = FlatBuffers.union(x, o)
+            if o != 0
+                return FlatBuffers.init(T, FlatBuffers.bytes(x), pos)
+            end
+        end
+    elseif field === :bodyLength
+        o = FlatBuffers.offset(x, 10)
+        o != 0 && return FlatBuffers.get(x, o + FlatBuffers.pos(x), Int64)
+        return Int64(0)
+    elseif field === :custom_metadata
+        o = FlatBuffers.offset(x, 12)
+        if o != 0
+            return FlatBuffers.Array{KeyValue}(x, o)
+        end
+    end
+    return nothing
+end
+
+messageStart(b::FlatBuffers.Builder) = FlatBuffers.startobject!(b, 5)
+messageAddVersion(b::FlatBuffers.Builder, version::MetadataVersion) = FlatBuffers.prependslot!(b, 0, version, 0)
+messageAddHeaderType(b::FlatBuffers.Builder, ::Core.Type{T}) where {T} = FlatBuffers.prependslot!(b, 1, MessageHeader(T), 0)
+messageAddHeader(b::FlatBuffers.Builder, header::FlatBuffers.UOffsetT) = FlatBuffers.prependoffsetslot!(b, 2, header, 0)
+messageAddBodyLength(b::FlatBuffers.Builder, bodyLength::Int64) = FlatBuffers.prependslot!(b, 3, bodyLength, 0)
+messageAddCustomMetadata(b::FlatBuffers.Builder, meta::FlatBuffers.UOffsetT) = FlatBuffers.prependoffsetslot!(b, 4, meta, 0)
+messageStartCustomMetadataVector(b::FlatBuffers.Builder, numelems) = FlatBuffers.startvector!(b, 4, numelems, 4)
+messageEnd(b::FlatBuffers.Builder) = FlatBuffers.endobject!(b)
\ No newline at end of file
diff --git a/julia/Arrow/src/metadata/Schema.jl b/julia/Arrow/src/metadata/Schema.jl
new file mode 100644
index 0000000..1ddf748
--- /dev/null
+++ b/julia/Arrow/src/metadata/Schema.jl
@@ -0,0 +1,605 @@
+# 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.
+
+FlatBuffers.@scopedenum MetadataVersion::Int16 V1 V2 V3 V4 V5
+
+struct Null <: FlatBuffers.Table
+    bytes::Vector{UInt8}
+    pos::Base.Int
+end
+
+Base.propertynames(x::Null) = ()
+
+nullStart(b::FlatBuffers.Builder) = FlatBuffers.startobject!(b, 0)
+nullEnd(b::FlatBuffers.Builder) = FlatBuffers.endobject!(b)
+
+struct Struct <: FlatBuffers.Table
+    bytes::Vector{UInt8}
+    pos::Base.Int
+end
+
+Base.propertynames(x::Struct) = ()
+
+structStart(b::FlatBuffers.Builder) = FlatBuffers.startobject!(b, 0)
+structEnd(b::FlatBuffers.Builder) = FlatBuffers.endobject!(b)
+
+struct List <: FlatBuffers.Table
+    bytes::Vector{UInt8}
+    pos::Base.Int
+end
+
+Base.propertynames(x::List) = ()
+
+listStart(b::FlatBuffers.Builder) = FlatBuffers.startobject!(b, 0)
+listEnd(b::FlatBuffers.Builder) = FlatBuffers.endobject!(b)
+
+struct LargeList <: FlatBuffers.Table
+    bytes::Vector{UInt8}
+    pos::Base.Int
+end
+
+Base.propertynames(x::LargeList) = ()
+
+largeListStart(b::FlatBuffers.Builder) = FlatBuffers.startobject!(b, 0)
+largeListEnd(b::FlatBuffers.Builder) = FlatBuffers.endobject!(b)
+
+struct FixedSizeList <: FlatBuffers.Table
+    bytes::Vector{UInt8}
+    pos::Base.Int
+end
+
+Base.propertynames(x::FixedSizeList) = (:listSize,)
+
+function Base.getproperty(x::FixedSizeList, field::Symbol)
+    if field === :listSize
+        o = FlatBuffers.offset(x, 4)
+        o != 0 && return FlatBuffers.get(x, o + FlatBuffers.pos(x), Int32)
+        return Int32(0)
+    end
+    return nothing
+end
+
+fixedSizeListStart(b::FlatBuffers.Builder) = FlatBuffers.startobject!(b, 1)
+fixedSizeListAddListSize(b::FlatBuffers.Builder, listSize::Int32) = FlatBuffers.prependslot!(b, 0, listSize, 0)
+fixedSizeListEnd(b::FlatBuffers.Builder) = FlatBuffers.endobject!(b)
+
+struct Map <: FlatBuffers.Table
+    bytes::Vector{UInt8}
+    pos::Base.Int
+end
+
+Base.propertynames(x::Map) = (:keysSorted,)
+
+function Base.getproperty(x::Map, field::Symbol)
+    if field === :keysSorted
+        o = FlatBuffers.offset(x, 4)
+        o != 0 && return FlatBuffers.get(x, o + FlatBuffers.pos(x), Base.Bool)
+    end
+    return nothing
+end
+
+mapStart(b::FlatBuffers.Builder) = FlatBuffers.startobject!(b, 1)
+mapAddKeysSorted(b::FlatBuffers.Builder, keyssorted::Base.Bool) = FlatBuffers.prependslot!(b, 0, keyssorted, 0)
+mapEnd(b::FlatBuffers.Builder) = FlatBuffers.endobject!(b)
+
+FlatBuffers.@scopedenum UnionMode::Int16 Sparse Dense
+
+struct Union <: FlatBuffers.Table
+    bytes::Vector{UInt8}
+    pos::Base.Int
+end
+
+Base.propertynames(x::Union) = (:mode, :typeIds)
+
+function Base.getproperty(x::Union, field::Symbol)
+    if field === :mode
+        o = FlatBuffers.offset(x, 4)
+        o != 0 && return FlatBuffers.get(x, o + FlatBuffers.pos(x), UnionMode)
+        return UnionMode.Sparse
+    elseif field === :typeIds
+        o = FlatBuffers.offset(x, 6)
+        o != 0 && return FlatBuffers.Array{Int32}(x, o)
+    end
+    return nothing
+end
+
+unionStart(b::FlatBuffers.Builder) = FlatBuffers.startobject!(b, 2)
+unionAddMode(b::FlatBuffers.Builder, mode::UnionMode) = FlatBuffers.prependslot!(b, 0, mode, 0)
+unionAddTypeIds(b::FlatBuffers.Builder, typeIds::FlatBuffers.UOffsetT) = FlatBuffers.prependoffsetslot!(b, 1, typeIds, 0)
+unionStartTypeIdsVector(b::FlatBuffers.Builder, numelems) = FlatBuffers.startvector!(b, 4, numelems, 4)
+unionEnd(b::FlatBuffers.Builder) = FlatBuffers.endobject!(b)
+
+struct Int <: FlatBuffers.Table
+    bytes::Vector{UInt8}
+    pos::Base.Int
+end
+
+Base.propertynames(x::Int) = (:bitWidth, :is_signed)
+
+function Base.getproperty(x::Int, field::Symbol)
+    if field === :bitWidth
+        o = FlatBuffers.offset(x, 4)
+        o != 0 && return FlatBuffers.get(x, o + FlatBuffers.pos(x), Int32)
+    elseif field === :is_signed
+        o = FlatBuffers.offset(x, 6)
+        o != 0 && return FlatBuffers.get(x, o + FlatBuffers.pos(x), Base.Bool)
+        return false
+    end
+    return nothing
+end
+
+intStart(b::FlatBuffers.Builder) = FlatBuffers.startobject!(b, 2)
+intAddBitWidth(b::FlatBuffers.Builder, bitwidth::Int32) = FlatBuffers.prependslot!(b, 0, bitwidth, 0)
+intAddIsSigned(b::FlatBuffers.Builder, issigned::Base.Bool) = FlatBuffers.prependslot!(b, 1, issigned, 0)
+intEnd(b::FlatBuffers.Builder) = FlatBuffers.endobject!(b)
+
+FlatBuffers.@scopedenum Precision::Int16 HALF SINGLE DOUBLE
+
+struct FloatingPoint <: FlatBuffers.Table
+    bytes::Vector{UInt8}
+    pos::Base.Int
+end
+
+Base.propertynames(x::FloatingPoint) = (:precision,)
+
+function Base.getproperty(x::FloatingPoint, field::Symbol)
+    if field === :precision
+        o = FlatBuffers.offset(x, 4)
+        o != 0 && return FlatBuffers.get(x, o + FlatBuffers.pos(x), Precision)
+        return Precision.HALF
+    end
+    return nothing
+end
+
+floatingPointStart(b::FlatBuffers.Builder) = FlatBuffers.startobject!(b, 1)
+floatingPointAddPrecision(b::FlatBuffers.Builder, precision::Precision) = FlatBuffers.prependslot!(b, 0, precision, 0)
+floatingPointEnd(b::FlatBuffers.Builder) = FlatBuffers.endobject!(b)
+
+struct Utf8 <: FlatBuffers.Table
+    bytes::Vector{UInt8}
+    pos::Base.Int
+end
+
+Base.propertynames(x::Utf8) = ()
+
+utf8Start(b::FlatBuffers.Builder) = FlatBuffers.startobject!(b, 0)
+utf8End(b::FlatBuffers.Builder) = FlatBuffers.endobject!(b)
+
+struct Binary <: FlatBuffers.Table
+    bytes::Vector{UInt8}
+    pos::Base.Int
+end
+
+Base.propertynames(x::Binary) = ()
+
+binaryStart(b::FlatBuffers.Builder) = FlatBuffers.startobject!(b, 0)
+binaryEnd(b::FlatBuffers.Builder) = FlatBuffers.endobject!(b)
+
+struct LargeUtf8 <: FlatBuffers.Table
+    bytes::Vector{UInt8}
+    pos::Base.Int
+end
+
+Base.propertynames(x::LargeUtf8) = ()
+
+largUtf8Start(b::FlatBuffers.Builder) = FlatBuffers.startobject!(b, 0)
+largUtf8End(b::FlatBuffers.Builder) = FlatBuffers.endobject!(b)
+
+struct LargeBinary <: FlatBuffers.Table
+    bytes::Vector{UInt8}
+    pos::Base.Int
+end
+
+Base.propertynames(x::LargeBinary) = ()
+
+largeBinaryStart(b::FlatBuffers.Builder) = FlatBuffers.startobject!(b, 0)
+largeBinaryEnd(b::FlatBuffers.Builder) = FlatBuffers.endobject!(b)
+
+struct FixedSizeBinary <: FlatBuffers.Table
+    bytes::Vector{UInt8}
+    pos::Base.Int
+end
+
+Base.propertynames(x::FixedSizeBinary) = (:byteWidth,)
+
+function Base.getproperty(x::FixedSizeBinary, field::Symbol)
+    if field === :byteWidth
+        o = FlatBuffers.offset(x, 4)
+        o != 0 && return FlatBuffers.get(x, o + FlatBuffers.pos(x), Int32)
+    end
+    return nothing
+end
+
+fixedSizeBinaryStart(b::FlatBuffers.Builder) = FlatBuffers.startobject!(b, 1)
+fixedSizeBinaryAddByteWidth(b::FlatBuffers.Builder, bytewidth::Int32) = FlatBuffers.prependslot!(b, 0, bytewidth, 0)
+fixedSizeBinaryEnd(b::FlatBuffers.Builder) = FlatBuffers.endobject!(b)
+
+struct Bool <: FlatBuffers.Table
+    bytes::Vector{UInt8}
+    pos::Base.Int
+end
+
+Base.propertynames(x::Bool) = ()
+
+boolStart(b::FlatBuffers.Builder) = FlatBuffers.startobject!(b, 0)
+boolEnd(b::FlatBuffers.Builder) = FlatBuffers.endobject!(b)
+
+struct Decimal <: FlatBuffers.Table
+    bytes::Vector{UInt8}
+    pos::Base.Int
+end
+
+Base.propertynames(x::Decimal) = (:precision, :scale)
+
+function Base.getproperty(x::Decimal, field::Symbol)
+    if field === :precision
+        o = FlatBuffers.offset(x, 4)
+        o != 0 && return FlatBuffers.get(x, o + FlatBuffers.pos(x), Int32)
+        return Int32(0)
+    elseif field === :scale
+        o = FlatBuffers.offset(x, 6)
+        o != 0 && return FlatBuffers.get(x, o + FlatBuffers.pos(x), Int32)
+        return Int32(0)
+    end
+    return nothing
+end
+
+decimalStart(b::FlatBuffers.Builder) = FlatBuffers.startobject!(b, 2)
+decimalAddPrecision(b::FlatBuffers.Builder, precision::Int32) = FlatBuffers.prependslot!(b, 0, precision, 0)
+decimalAddScale(b::FlatBuffers.Builder, scale::Int32) = FlatBuffers.prependslot!(b, 1, scale, 0)
+decimalEnd(b::FlatBuffers.Builder) = FlatBuffers.endobject!(b)
+
+FlatBuffers.@scopedenum DateUnit::Int16 DAY MILLISECOND
+
+struct Date <: FlatBuffers.Table
+    bytes::Vector{UInt8}
+    pos::Base.Int
+end
+
+Base.propertynames(x::Date) = (:unit,)
+
+function Base.getproperty(x::Date, field::Symbol)
+    if field === :unit
+        o = FlatBuffers.offset(x, 4)
+        o != 0 && return FlatBuffers.get(x, o + FlatBuffers.pos(x), DateUnit)
+        return DateUnit.MILLISECOND
+    end
+    return nothing
+end
+
+dateStart(b::FlatBuffers.Builder) = FlatBuffers.startobject!(b, 1)
+dateAddUnit(b::FlatBuffers.Builder, unit::DateUnit) = FlatBuffers.prependslot!(b, 0, unit, 1)
+dateEnd(b::FlatBuffers.Builder) = FlatBuffers.endobject!(b)
+
+FlatBuffers.@scopedenum TimeUnit::Int16 SECOND MILLISECOND MICROSECOND NANOSECOND
+
+struct Time <: FlatBuffers.Table
+    bytes::Vector{UInt8}
+    pos::Base.Int
+end
+
+Base.propertynames(x::Time) = (:unit, :bitWidth)
+
+function Base.getproperty(x::Time, field::Symbol)
+    if field === :unit
+        o = FlatBuffers.offset(x, 4)
+        o != 0 && return FlatBuffers.get(x, o + FlatBuffers.pos(x), TimeUnit)
+        return TimeUnit.MILLISECOND
+    elseif field === :bitWidth
+        o = FlatBuffers.offset(x, 6)
+        o != 0 && return FlatBuffers.get(x, o + FlatBuffers.pos(x), Int32)
+        return 32
+    end
+    return nothing
+end
+
+timeStart(b::FlatBuffers.Builder) = FlatBuffers.startobject!(b, 2)
+timeAddUnit(b::FlatBuffers.Builder, unit::TimeUnit) = FlatBuffers.prependslot!(b, 0, unit, 1)
+timeAddBitWidth(b::FlatBuffers.Builder, bitwidth::Int32) = FlatBuffers.prependslot!(b, 1, bitwidth, 32)
+timeEnd(b::FlatBuffers.Builder) = FlatBuffers.endobject!(b)
+
+struct Timestamp <: FlatBuffers.Table
+    bytes::Vector{UInt8}
+    pos::Base.Int
+end
+
+Base.propertynames(x::Timestamp) = (:unit, :timezone)
+
+function Base.getproperty(x::Timestamp, field::Symbol)
+    if field === :unit
+        o = FlatBuffers.offset(x, 4)
+        o != 0 && return FlatBuffers.get(x, o + FlatBuffers.pos(x), TimeUnit)
+        return TimeUnit.SECOND
+    elseif field === :timezone
+        o = FlatBuffers.offset(x, 6)
+        o != 0 && return String(x, o + FlatBuffers.pos(x))
+    end
+    return nothing
+end
+
+timestampStart(b::FlatBuffers.Builder) = FlatBuffers.startobject!(b, 2)
+timestampAddUnit(b::FlatBuffers.Builder, unit::TimeUnit) = FlatBuffers.prependslot!(b, 0, unit, 0)
+timestampAddTimezone(b::FlatBuffers.Builder, timezone::FlatBuffers.UOffsetT) = FlatBuffers.prependoffsetslot!(b, 1, timezone, 0)
+timestampEnd(b::FlatBuffers.Builder) = FlatBuffers.endobject!(b)
+
+FlatBuffers.@scopedenum IntervalUnit::Int16 YEAR_MONTH DAY_TIME
+
+struct Interval <: FlatBuffers.Table
+    bytes::Vector{UInt8}
+    pos::Base.Int
+end
+
+Base.propertynames(x::Interval) = (:unit,)
+
+function Base.getproperty(x::Interval, field::Symbol)
+    if field === :unit
+        o = FlatBuffers.offset(x, 4)
+        o != 0 && return FlatBuffers.get(x, o + FlatBuffers.pos(x), IntervalUnit)
+        return IntervalUnit.YEAR_MONTH
+    end
+    return nothing
+end
+
+intervalStart(b::FlatBuffers.Builder) = FlatBuffers.startobject!(b, 1)
+intervalAddUnit(b::FlatBuffers.Builder, unit::IntervalUnit) = FlatBuffers.prependslot!(b, 0, unit, 0)
+intervalEnd(b::FlatBuffers.Builder) = FlatBuffers.endobject!(b)
+
+struct Duration <: FlatBuffers.Table
+    bytes::Vector{UInt8}
+    pos::Base.Int
+end
+
+Base.propertynames(x::Duration) = (:unit,)
+
+function Base.getproperty(x::Duration, field::Symbol)
+    if field === :unit
+        o = FlatBuffers.offset(x, 4)
+        o != 0 && return FlatBuffers.get(x, o + FlatBuffers.pos(x), TimeUnit)
+        return TimeUnit.MILLISECOND
+    end
+    return nothing
+end
+
+durationStart(b::FlatBuffers.Builder) = FlatBuffers.startobject!(b, 1)
+durationAddUnit(b::FlatBuffers.Builder, unit::TimeUnit) = FlatBuffers.prependslot!(b, 0, unit, 1)
+durationEnd(b::FlatBuffers.Builder) = FlatBuffers.endobject!(b)
+
+function Type(b::UInt8)
+    b == 1 && return Null
+    b == 2 && return Int
+    b == 3 && return FloatingPoint
+    b == 4 && return Binary
+    b == 5 && return Utf8
+    b == 6 && return Bool
+    b == 7 && return Decimal
+    b == 8 && return Date
+    b == 9 && return Time
+    b == 10 && return Timestamp
+    b == 11 && return Interval
+    b == 12 && return List
+    b == 13 && return Struct
+    b == 14 && return Union
+    b == 15 && return FixedSizeBinary
+    b == 16 && return FixedSizeList
+    b == 17 && return Map
+    b == 18 && return Duration
+    b == 19 && return LargeBinary
+    b == 20 && return LargeUtf8
+    b == 21 && return LargeList
+    return nothing
+end
+
+function Type(::Base.Type{T})::Int16 where {T}
+    T == Null && return 1
+    T == Int && return 2
+    T == FloatingPoint && return 3
+    T == Binary && return 4
+    T == Utf8 && return 5
+    T == Bool && return 6
+    T == Decimal && return 7
+    T == Date && return 8
+    T == Time && return 9
+    T == Timestamp && return 10
+    T == Interval && return 11
+    T == List && return 12
+    T == Struct && return 13
+    T == Union && return 14
+    T == FixedSizeBinary && return 15
+    T == FixedSizeList && return 16
+    T == Map && return 17
+    T == Duration && return 18
+    T == LargeBinary && return 19
+    T == LargeUtf8 && return 20
+    T == LargeList && return 21
+    return 0
+end
+
+struct KeyValue <: FlatBuffers.Table
+    bytes::Vector{UInt8}
+    pos::Base.Int
+end
+
+Base.propertynames(x::KeyValue) = (:key, :value)
+
+function Base.getproperty(x::KeyValue, field::Symbol)
+    if field === :key
+        o = FlatBuffers.offset(x, 4)
+        o != 0 && return String(x, o + FlatBuffers.pos(x))
+    elseif field === :value
+        o = FlatBuffers.offset(x, 6)
+        o != 0 && return String(x, o + FlatBuffers.pos(x))
+    end
+    return nothing
+end
+
+keyValueStart(b::FlatBuffers.Builder) = FlatBuffers.startobject!(b, 2)
+keyValueAddKey(b::FlatBuffers.Builder, key::FlatBuffers.UOffsetT) = FlatBuffers.prependoffsetslot!(b, 0, key, 0)
+keyValueAddValue(b::FlatBuffers.Builder, value::FlatBuffers.UOffsetT) = FlatBuffers.prependoffsetslot!(b, 1, value, 0)
+keyValueEnd(b::FlatBuffers.Builder) = FlatBuffers.endobject!(b)
+
+FlatBuffers.@scopedenum DictionaryKind::Int16 DenseArray
+
+struct DictionaryEncoding <: FlatBuffers.Table
+    bytes::Vector{UInt8}
+    pos::Base.Int
+end
+
+Base.propertynames(x::DictionaryEncoding) = (:id, :indexType, :isOrdered, :dictionaryKind)
+
+function Base.getproperty(x::DictionaryEncoding, field::Symbol)
+    if field === :id
+        o = FlatBuffers.offset(x, 4)
+        o != 0 && return FlatBuffers.get(x, o + FlatBuffers.pos(x), Int64)
+        return Int64(0)
+    elseif field === :indexType
+        o = FlatBuffers.offset(x, 6)
+        if o != 0
+            y = FlatBuffers.indirect(x, o + FlatBuffers.pos(x))
+            return FlatBuffers.init(Int, FlatBuffers.bytes(x), y)
+        end
+    elseif field === :isOrdered
+        o = FlatBuffers.offset(x, 8)
+        o != 0 && return FlatBuffers.get(x, o + FlatBuffers.pos(x), Base.Bool)
+        return false
+    elseif field === :dictionaryKind
+        o = FlatBuffers.offset(x, 10)
+        o != 0 && return FlatBuffers.get(x, o + FlatBuffers.pos(x), DictionaryKind)
+    end
+    return nothing
+end
+
+dictionaryEncodingStart(b::FlatBuffers.Builder) = FlatBuffers.startobject!(b, 3)
+dictionaryEncodingAddId(b::FlatBuffers.Builder, id::Int64) = FlatBuffers.prependslot!(b, 0, id, 0)
+dictionaryEncodingAddIndexType(b::FlatBuffers.Builder, indextype::FlatBuffers.UOffsetT) = FlatBuffers.prependoffsetslot!(b, 1, indextype, 0)
+dictionaryEncodingAddIsOrdered(b::FlatBuffers.Builder, isordered::Base.Bool) = FlatBuffers.prependslot!(b, 1, isordered, 0)
+dictionaryEncodingEnd(b::FlatBuffers.Builder) = FlatBuffers.endobject!(b)
+
+struct Field <: FlatBuffers.Table
+    bytes::Vector{UInt8}
+    pos::Base.Int
+end
+
+Base.propertynames(x::Field) = (:name, :nullable, :type, :dictionary, :children, :custom_metadata)
+
+function Base.getproperty(x::Field, field::Symbol)
+    if field === :name
+        o = FlatBuffers.offset(x, 4)
+        o != 0 && return String(x, o + FlatBuffers.pos(x))
+    elseif field === :nullable
+        o = FlatBuffers.offset(x, 6)
+        o != 0 && return FlatBuffers.get(x, o + FlatBuffers.pos(x), Base.Bool)
+        return false
+    elseif field === :type
+        o = FlatBuffers.offset(x, 8)
+        if o != 0
+            T = Type(FlatBuffers.get(x, o + FlatBuffers.pos(x), UInt8))
+            o = FlatBuffers.offset(x, 10)
+            pos = FlatBuffers.union(x, o)
+            if o != 0
+                return FlatBuffers.init(T, FlatBuffers.bytes(x), pos)
+            end
+        end
+    elseif field === :dictionary
+        o = FlatBuffers.offset(x, 12)
+        if o != 0
+            y = FlatBuffers.indirect(x, o + FlatBuffers.pos(x))
+            return FlatBuffers.init(DictionaryEncoding, FlatBuffers.bytes(x), y)
+        end
+    elseif field === :children
+        o = FlatBuffers.offset(x, 14)
+        if o != 0
+            return FlatBuffers.Array{Field}(x, o)
+        end
+    elseif field === :custom_metadata
+        o = FlatBuffers.offset(x, 16)
+        if o != 0
+            return FlatBuffers.Array{KeyValue}(x, o)
+        end
+    end
+    return nothing
+end
+
+fieldStart(b::FlatBuffers.Builder) = FlatBuffers.startobject!(b, 7)
+fieldAddName(b::FlatBuffers.Builder, name::FlatBuffers.UOffsetT) = FlatBuffers.prependoffsetslot!(b, 0, name, 0)
+fieldAddNullable(b::FlatBuffers.Builder, nullable::Base.Bool) = FlatBuffers.prependslot!(b, 1, nullable, false)
+fieldAddTypeType(b::FlatBuffers.Builder, ::Core.Type{T}) where {T} = FlatBuffers.prependslot!(b, 2, Type(T), 0)
+fieldAddType(b::FlatBuffers.Builder, type::FlatBuffers.UOffsetT) = FlatBuffers.prependoffsetslot!(b, 3, type, 0)
+fieldAddDictionary(b::FlatBuffers.Builder, dictionary::FlatBuffers.UOffsetT) = FlatBuffers.prependoffsetslot!(b, 4, dictionary, 0)
+fieldAddChildren(b::FlatBuffers.Builder, children::FlatBuffers.UOffsetT) = FlatBuffers.prependoffsetslot!(b, 5, children, 0)
+fieldStartChildrenVector(b::FlatBuffers.Builder, numelems) = FlatBuffers.startvector!(b, 4, numelems, 4)
+fieldAddCustomMetadata(b::FlatBuffers.Builder, custommetadata::FlatBuffers.UOffsetT) = FlatBuffers.prependoffsetslot!(b, 6, custommetadata, 0)
+fieldStartCustomMetadataVector(b::FlatBuffers.Builder, numelems) = FlatBuffers.startvector!(b, 4, numelems, 4)
+fieldEnd(b::FlatBuffers.Builder) = FlatBuffers.endobject!(b)
+
+FlatBuffers.@scopedenum Endianness::Int16 Little Big
+
+struct Buffer <: FlatBuffers.Struct
+    bytes::Vector{UInt8}
+    pos::Base.Int
+end
+
+FlatBuffers.structsizeof(::Base.Type{Buffer}) = 16
+
+Base.propertynames(x::Buffer) = (:offset, :length)
+
+function Base.getproperty(x::Buffer, field::Symbol)
+    if field === :offset
+        return FlatBuffers.get(x, FlatBuffers.pos(x), Int64)
+    elseif field === :length
+        return FlatBuffers.get(x, FlatBuffers.pos(x) + 8, Int64)
+    end
+    return nothing
+end
+
+function createBuffer(b::FlatBuffers.Builder, offset::Int64, length::Int64)
+    FlatBuffers.prep!(b, 8, 16)
+    prepend!(b, length)
+    prepend!(b, offset)
+    return FlatBuffers.offset(b)
+end
+
+struct Schema <: FlatBuffers.Table
+    bytes::Vector{UInt8}
+    pos::Base.Int
+end
+
+Base.propertynames(x::Schema) = (:endianness, :fields, :custom_metadata)
+
+function Base.getproperty(x::Schema, field::Symbol)
+    if field === :endianness
+        o = FlatBuffers.offset(x, 4)
+        o != 0 && return FlatBuffers.get(x, o + FlatBuffers.pos(x), Endianness)
+    elseif field === :fields
+        o = FlatBuffers.offset(x, 6)
+        if o != 0
+            return FlatBuffers.Array{Field}(x, o)
+        end
+    elseif field === :custom_metadata
+        o = FlatBuffers.offset(x, 8)
+        if o != 0
+            return FlatBuffers.Array{KeyValue}(x, o)
+        end
+    end
+    return nothing
+end
+
+schemaStart(b::FlatBuffers.Builder) = FlatBuffers.startobject!(b, 3)
+schemaAddEndianness(b::FlatBuffers.Builder, endianness::Endianness) = FlatBuffers.prependslot!(b, 0, endianness, 0)
+schemaAddFields(b::FlatBuffers.Builder, fields::FlatBuffers.UOffsetT) = FlatBuffers.prependoffsetslot!(b, 1, fields, 0)
+schemaStartFieldsVector(b::FlatBuffers.Builder, numelems) = FlatBuffers.startvector!(b, 4, numelems, 4)
+schemaAddCustomMetadata(b::FlatBuffers.Builder, custommetadata::FlatBuffers.UOffsetT) = FlatBuffers.prependoffsetslot!(b, 2, custommetadata, 0)
+schemaStartCustomMetadataVector(b::FlatBuffers.Builder, numelems) = FlatBuffers.startvector!(b, 4, numelems, 4)
+schemaEnd(b::FlatBuffers.Builder) = FlatBuffers.endobject!(b)
diff --git a/julia/Arrow/src/table.jl b/julia/Arrow/src/table.jl
new file mode 100644
index 0000000..721d208
--- /dev/null
+++ b/julia/Arrow/src/table.jl
@@ -0,0 +1,556 @@
+# 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.
+
+struct BatchIterator
+    bytes::Vector{UInt8}
+    startpos::Int
+end
+
+"""
+    Arrow.Stream(io::IO; convert::Bool=true)
+    Arrow.Stream(file::String; convert::Bool=true)
+    Arrow.Stream(bytes::Vector{UInt8}, pos=1, len=nothing; convert::Bool=true)
+
+Start reading an arrow formatted table, from:
+ * `io`, bytes will be read all at once via `read(io)`
+ * `file`, bytes will be read via `Mmap.mmap(file)`
+ * `bytes`, a byte vector directly, optionally allowing specifying the starting byte position `pos` and `len`
+
+Reads the initial schema message from the arrow stream/file, then returns an `Arrow.Stream` object
+which will iterate over record batch messages, producing an [`Arrow.Table`](@ref) on each iteration.
+
+By iterating [`Arrow.Table`](@ref), `Arrow.Stream` satisfies the `Tables.partitions` interface, and as such can
+be passed to Tables.jl-compatible sink functions.
+
+This allows iterating over extremely large "arrow tables" in chunks represented as record batches.
+
+Supports the `convert` keyword argument which controls whether certain arrow primitive types will be
+lazily converted to more friendly Julia defaults; by default, `convert=true`.
+"""
+struct Stream
+    batchiterator::BatchIterator
+    pos::Int
+    names::Vector{Symbol}
+    schema::Meta.Schema
+    dictencodings::Dict{Int64, DictEncoding} # dictionary id => DictEncoding
+    dictencoded::Dict{Int64, Meta.Field} # dictionary id => field
+    convert::Bool
+end
+
+Tables.partitions(x::Stream) = x
+
+Stream(io::IO, pos::Integer=1, len=nothing; convert::Bool=true) = Stream(Base.read(io), pos, len; convert=convert)
+Stream(str::String, pos::Integer=1, len=nothing; convert::Bool=true) = isfile(str) ? Stream(Mmap.mmap(str), pos, len; convert=convert) :
+    throw(ArgumentError("$str is not a valid arrow file"))
+
+# will detect whether we're reading a Stream from a file or stream
+function Stream(bytes::Vector{UInt8}, off::Integer=1, tlen::Union{Integer, Nothing}=nothing; convert::Bool=true)
+    len = something(tlen, length(bytes))
+    if len > 24 &&
+        _startswith(bytes, off, FILE_FORMAT_MAGIC_BYTES) &&
+        _endswith(bytes, off + len - 1, FILE_FORMAT_MAGIC_BYTES)
+        off += 8 # skip past magic bytes + padding
+    end
+    dictencodings = Dict{Int64, DictEncoding}() # dictionary id => DictEncoding
+    dictencoded = Dict{Int64, Meta.Field}() # dictionary id => field
+    batchiterator = BatchIterator(bytes, off)
+    state = iterate(batchiterator)
+    state === nothing && throw(ArgumentError("no arrow ipc messages found in provided input"))
+    batch, (pos, id) = state
+    schema = batch.msg.header
+    schema isa Meta.Schema || throw(ArgumentError("first arrow ipc message MUST be a schema message"))
+    # assert endianness?
+    # store custom_metadata?
+    names = Symbol[]
+    for (i, field) in enumerate(schema.fields)
+        push!(names, Symbol(field.name))
+        # recursively find any dictionaries for any fields
+        getdictionaries!(dictencoded, field)
+        @debug 1 "parsed column from schema: field = $field"
+    end
+    return Stream(batchiterator, pos, names, schema, dictencodings, dictencoded, convert)
+end
+
+function Base.iterate(x::Stream, (pos, id)=(x.pos, 1))
+    columns = AbstractVector[]
+    while true
+        state = iterate(x.batchiterator, (pos, id))
+        state === nothing && return nothing
+        batch, (pos, id) = state
+        header = batch.msg.header
+        if header isa Meta.DictionaryBatch
+            id = header.id
+            recordbatch = header.data
+            @debug 1 "parsing dictionary batch message: id = $id, compression = $(recordbatch.compression)"
+            if haskey(x.dictencodings, id) && header.isDelta
+                # delta
+                field = x.dictencoded[id]
+                values, _, _ = build(field, field.type, batch, recordbatch, x.dictencodings, Int64(1), Int64(1), x.convert)
+                dictencoding = x.dictencodings[id]
+                append!(dictencoding.data, values)
+                continue
+            end
+            # new dictencoding or replace
+            field = x.dictencoded[id]
+            values, _, _ = build(field, field.type, batch, recordbatch, x.dictencodings, Int64(1), Int64(1), x.convert)
+            A = ChainedVector([values])
+            x.dictencodings[id] = DictEncoding{eltype(A), typeof(A)}(id, A, field.dictionary.isOrdered)
+            @debug 1 "parsed dictionary batch message: id=$id, data=$values\n"
+        elseif header isa Meta.RecordBatch
+            @debug 1 "parsing record batch message: compression = $(header.compression)"
+            for vec in VectorIterator(x.schema, batch, x.dictencodings, x.convert)
+                push!(columns, vec)
+            end
+            break
+        else
+            throw(ArgumentError("unsupported arrow message type: $(typeof(header))"))
+        end
+    end
+    lookup = Dict{Symbol, AbstractVector}()
+    types = Type[]
+    for (nm, col) in zip(x.names, columns)
+        lookup[nm] = col
+        push!(types, eltype(col))
+    end
+    return Table(x.names, types, columns, lookup, Ref(x.schema)), (pos, id)
+end
+
+"""
+    Arrow.Table(io::IO; convert::Bool=true)
+    Arrow.Table(file::String; convert::Bool=true)
+    Arrow.Table(bytes::Vector{UInt8}, pos=1, len=nothing; convert::Bool=true)
+
+Read an arrow formatted table, from:
+ * `io`, bytes will be read all at once via `read(io)`
+ * `file`, bytes will be read via `Mmap.mmap(file)`
+ * `bytes`, a byte vector directly, optionally allowing specifying the starting byte position `pos` and `len`
+
+Returns a `Arrow.Table` object that allows column access via `table.col1`, `table[:col1]`, or `table[1]`.
+
+NOTE: the columns in an `Arrow.Table` are views into the original arrow memory, and hence are not easily
+modifiable (with e.g. `push!`, `append!`, etc.). To mutate arrow columns, call `copy(x)` to materialize
+the arrow data as a normal Julia array.
+
+`Arrow.Table` also satisfies the [Tables.jl](https://github.com/JuliaData/Tables.jl) interface, and so can easily be materialied via any supporting
+sink function: e.g. `DataFrame(Arrow.Table(file))`, `SQLite.load!(db, "table", Arrow.Table(file))`, etc.
+
+Supports the `convert` keyword argument which controls whether certain arrow primitive types will be
+lazily converted to more friendly Julia defaults; by default, `convert=true`.
+"""
+struct Table <: Tables.AbstractColumns
+    names::Vector{Symbol}
+    types::Vector{Type}
+    columns::Vector{AbstractVector}
+    lookup::Dict{Symbol, AbstractVector}
+    schema::Ref{Meta.Schema}
+end
+
+Table() = Table(Symbol[], Type[], AbstractVector[], Dict{Symbol, AbstractVector}(), Ref{Meta.Schema}())
+
+names(t::Table) = getfield(t, :names)
+types(t::Table) = getfield(t, :types)
+columns(t::Table) = getfield(t, :columns)
+lookup(t::Table) = getfield(t, :lookup)
+schema(t::Table) = getfield(t, :schema)
+
+Tables.istable(::Table) = true
+Tables.columnaccess(::Table) = true
+Tables.columns(t::Table) = Tables.CopiedColumns(t)
+Tables.schema(t::Table) = Tables.Schema(names(t), types(t))
+Tables.columnnames(t::Table) = names(t)
+Tables.getcolumn(t::Table, i::Int) = columns(t)[i]
+Tables.getcolumn(t::Table, nm::Symbol) = lookup(t)[nm]
+
+# high-level user API functions
+Table(io::IO, pos::Integer=1, len=nothing; convert::Bool=true) = Table(Base.read(io), pos, len; convert=convert)
+Table(str::String, pos::Integer=1, len=nothing; convert::Bool=true) = isfile(str) ? Table(Mmap.mmap(str), pos, len; convert=convert) :
+    throw(ArgumentError("$str is not a valid arrow file"))
+
+# will detect whether we're reading a Table from a file or stream
+function Table(bytes::Vector{UInt8}, off::Integer=1, tlen::Union{Integer, Nothing}=nothing; convert::Bool=true)
+    len = something(tlen, length(bytes))
+    if len > 24 &&
+        _startswith(bytes, off, FILE_FORMAT_MAGIC_BYTES) &&
+        _endswith(bytes, off + len - 1, FILE_FORMAT_MAGIC_BYTES)
+        off += 8 # skip past magic bytes + padding
+    end
+    t = Table()
+    sch = nothing
+    dictencodings = Dict{Int64, DictEncoding}() # dictionary id => DictEncoding
+    dictencoded = Dict{Int64, Meta.Field}() # dictionary id => field
+    tsks = Channel{Task}(Inf)
+    tsk = Threads.@spawn begin
+        i = 1
+        for tsk in tsks
+            cols = fetch(tsk)
+            if i == 1
+                foreach(x -> push!(columns(t), x), cols)
+            elseif i == 2
+                foreach(1:length(cols)) do i
+                    columns(t)[i] = ChainedVector([columns(t)[i], cols[i]])
+                end
+            else
+                foreach(1:length(cols)) do i
+                    append!(columns(t)[i], cols[i])
+                end
+            end
+            i += 1
+        end
+    end
+    for batch in BatchIterator(bytes, off)
+        # store custom_metadata of batch.msg?
+        header = batch.msg.header
+        if header isa Meta.Schema
+            @debug 1 "parsing schema message"
+            # assert endianness?
+            # store custom_metadata?
+            for (i, field) in enumerate(header.fields)
+                push!(names(t), Symbol(field.name))
+                # recursively find any dictionaries for any fields
+                getdictionaries!(dictencoded, field)
+                @debug 1 "parsed column from schema: field = $field"
+            end
+            sch = header
+            schema(t)[] = sch
+        elseif header isa Meta.DictionaryBatch
+            id = header.id
+            recordbatch = header.data
+            @debug 1 "parsing dictionary batch message: id = $id, compression = $(recordbatch.compression)"
+            if haskey(dictencodings, id) && header.isDelta
+                # delta
+                field = dictencoded[id]
+                values, _, _ = build(field, field.type, batch, recordbatch, dictencodings, Int64(1), Int64(1), convert)
+                dictencoding = dictencodings[id]
+                append!(dictencoding.data, values)
+                continue
+            end
+            # new dictencoding or replace
+            field = dictencoded[id]
+            values, _, _ = build(field, field.type, batch, recordbatch, dictencodings, Int64(1), Int64(1), convert)
+            A = ChainedVector([values])
+            dictencodings[id] = DictEncoding{eltype(A), typeof(A)}(id, A, field.dictionary.isOrdered)
+            @debug 1 "parsed dictionary batch message: id=$id, data=$values\n"
+        elseif header isa Meta.RecordBatch
+            @debug 1 "parsing record batch message: compression = $(header.compression)"
+            put!(tsks, Threads.@spawn begin
+                collect(VectorIterator(sch, batch, dictencodings, convert))
+            end)
+        else
+            throw(ArgumentError("unsupported arrow message type: $(typeof(header))"))
+        end
+    end
+    close(tsks)
+    wait(tsk)
+    lu = lookup(t)
+    ty = types(t)
+    for (nm, col) in zip(names(t), columns(t))
+        lu[nm] = col
+        push!(ty, eltype(col))
+    end
+    meta = sch !== nothing ? sch.custom_metadata : nothing
+    if meta !== nothing
+        setmetadata!(t, Dict(String(kv.key) => String(kv.value) for kv in meta))
+    end
+    return t
+end
+
+function getdictionaries!(dictencoded, field)
+    d = field.dictionary
+    if d !== nothing
+        dictencoded[d.id] = field
+    end
+    for child in field.children
+        getdictionaries!(dictencoded, child)
+    end
+    return
+end
+
+struct Batch
+    msg::Meta.Message
+    bytes::Vector{UInt8}
+    pos::Int
+    id::Int
+end
+
+function Base.iterate(x::BatchIterator, (pos, id)=(x.startpos, 0))
+    @debug 1 "checking for next arrow message: pos = $pos"
+    if pos + 3 > length(x.bytes)
+        @debug 1 "not enough bytes left for another batch message"
+        return nothing
+    end
+    if readbuffer(x.bytes, pos, UInt32) != CONTINUATION_INDICATOR_BYTES
+        @debug 1 "didn't find continuation byte to keep parsing messages: $(readbuffer(x.bytes, pos, UInt32))"
+        return nothing
+    end
+    pos += 4
+    if pos + 3 > length(x.bytes)
+        @debug 1 "not enough bytes left to read length of another batch message"
+        return nothing
+    end
+    msglen = readbuffer(x.bytes, pos, Int32)
+    if msglen == 0
+        @debug 1 "message has 0 length; terminating message parsing"
+        return nothing
+    end
+    pos += 4
+    msg = FlatBuffers.getrootas(Meta.Message, x.bytes, pos-1)
+    pos += msglen
+    # pos now points to message body
+    @debug 1 "parsing message: pos = $pos, msglen = $msglen, bodyLength = $(msg.bodyLength)"
+    return Batch(msg, x.bytes, pos, id), (pos + msg.bodyLength, id + 1)
+end
+
+struct VectorIterator
+    schema::Meta.Schema
+    batch::Batch # batch.msg.header MUST BE RecordBatch
+    dictencodings::Dict{Int64, DictEncoding}
+    convert::Bool
+end
+
+buildmetadata(f::Meta.Field) = buildmetadata(f.custom_metadata)
+buildmetadata(meta) = Dict(String(kv.key) => String(kv.value) for kv in meta)
+buildmetadata(::Nothing) = nothing
+
+function Base.iterate(x::VectorIterator, (columnidx, nodeidx, bufferidx)=(Int64(1), Int64(1), Int64(1)))
+    columnidx > length(x.schema.fields) && return nothing
+    field = x.schema.fields[columnidx]
+    @debug 2 "building top-level column: field = $(field), columnidx = $columnidx, nodeidx = $nodeidx, bufferidx = $bufferidx"
+    A, nodeidx, bufferidx = build(field, x.batch, x.batch.msg.header, x.dictencodings, nodeidx, bufferidx, x.convert)
+    @debug 2 "built top-level column: A = $(typeof(A)), columnidx = $columnidx, nodeidx = $nodeidx, bufferidx = $bufferidx"
+    @debug 3 A
+    return A, (columnidx + 1, nodeidx, bufferidx)
+end
+
+Base.length(x::VectorIterator) = length(x.schema.fields)
+
+const ListTypes = Union{Meta.Utf8, Meta.LargeUtf8, Meta.Binary, Meta.LargeBinary, Meta.List, Meta.LargeList}
+const LargeLists = Union{Meta.LargeUtf8, Meta.LargeBinary, Meta.LargeList}
+
+function build(field::Meta.Field, batch, rb, de, nodeidx, bufferidx, convert)
+    d = field.dictionary
+    if d !== nothing
+        validity = buildbitmap(batch, rb, nodeidx, bufferidx)
+        bufferidx += 1
+        buffer = rb.buffers[bufferidx]
+        S = d.indexType === nothing ? Int32 : juliaeltype(field, d.indexType, false)
+        bytes, indices = reinterp(S, batch, buffer, rb.compression)
+        encoding = de[d.id]
+        A = DictEncoded(bytes, validity, indices, encoding, buildmetadata(field.custom_metadata))
+        nodeidx += 1
+        bufferidx += 1
+    else
+        A, nodeidx, bufferidx = build(field, field.type, batch, rb, de, nodeidx, bufferidx, convert)
+    end
+    return A, nodeidx, bufferidx
+end
+
+function buildbitmap(batch, rb, nodeidx, bufferidx)
+    buffer = rb.buffers[bufferidx]
+    voff = batch.pos + buffer.offset
+    node = rb.nodes[nodeidx]
+    if rb.compression === nothing
+        return ValidityBitmap(batch.bytes, voff, node.length, node.null_count)
+    else
+        # compressed
+        ptr = pointer(batch.bytes, voff)
+        _, decodedbytes = uncompress(ptr, buffer, rb.compression)
+        return ValidityBitmap(decodedbytes, 1, node.length, node.null_count)
+    end
+end
+
+function uncompress(ptr::Ptr{UInt8}, buffer, compression)
+    if buffer.length == 0
+        return 0, UInt8[]
+    end
+    len = unsafe_load(convert(Ptr{Int64}, ptr))
+    ptr += 8 # skip past uncompressed length as Int64
+    encodedbytes = unsafe_wrap(Array, ptr, buffer.length - 8)
+    if compression.codec === Meta.CompressionType.LZ4_FRAME
+        decodedbytes = transcode(LZ4FrameDecompressor, encodedbytes)
+    elseif compression.codec === Meta.CompressionType.ZSTD
+        decodedbytes = transcode(ZstdDecompressor, encodedbytes)
+    else
+        error("unsupported compression type when reading arrow buffers: $(typeof(compression.codec))")
+    end
+    return len, decodedbytes
+end
+
+function reinterp(::Type{T}, batch, buf, compression) where {T}
+    ptr = pointer(batch.bytes, batch.pos + buf.offset)
+    if compression === nothing
+        return batch.bytes, unsafe_wrap(Array, convert(Ptr{T}, ptr), div(buf.length, sizeof(T)))
+    else
+        # compressed
+        len, decodedbytes = uncompress(ptr, buf, compression)
+        return decodedbytes, unsafe_wrap(Array, convert(Ptr{T}, pointer(decodedbytes)), div(len, sizeof(T)))
+    end
+end
+
+function build(f::Meta.Field, L::ListTypes, batch, rb, de, nodeidx, bufferidx, convert)
+    @debug 2 "building array: L = $L"
+    validity = buildbitmap(batch, rb, nodeidx, bufferidx)
+    bufferidx += 1
+    buffer = rb.buffers[bufferidx]
+    ooff = batch.pos + buffer.offset
+    OT = L isa LargeLists ? Int64 : Int32
+    bytes, offs = reinterp(OT, batch, buffer, rb.compression)
+    offsets = Offsets(bytes, offs)
+    bufferidx += 1
+    len = rb.nodes[nodeidx].length
+    nodeidx += 1
+    if L isa Meta.Utf8 || L isa Meta.LargeUtf8 || L isa Meta.Binary || L isa Meta.LargeBinary
+        buffer = rb.buffers[bufferidx]
+        bytes, A = reinterp(UInt8, batch, buffer, rb.compression)
+        bufferidx += 1
+    else
+        bytes = UInt8[]
+        A, nodeidx, bufferidx = build(f.children[1], batch, rb, de, nodeidx, bufferidx, convert)
+    end
+    meta = buildmetadata(f.custom_metadata)
+    T = juliaeltype(f, meta, convert)
+    return List{T, OT, typeof(A)}(bytes, validity, offsets, A, len, meta), nodeidx, bufferidx
+end
+
+function build(f::Meta.Field, L::Union{Meta.FixedSizeBinary, Meta.FixedSizeList}, batch, rb, de, nodeidx, bufferidx, convert)
+    @debug 2 "building array: L = $L"
+    validity = buildbitmap(batch, rb, nodeidx, bufferidx)
+    bufferidx += 1
+    len = rb.nodes[nodeidx].length
+    nodeidx += 1
+    if L isa Meta.FixedSizeBinary
+        buffer = rb.buffers[bufferidx]
+        bytes, A = reinterp(UInt8, batch, buffer, rb.compression)
+        bufferidx += 1
+    else
+        bytes = UInt8[]
+        A, nodeidx, bufferidx = build(f.children[1], batch, rb, de, nodeidx, bufferidx, convert)
+    end
+    meta = buildmetadata(f.custom_metadata)
+    T = juliaeltype(f, meta, convert)
+    return FixedSizeList{T, typeof(A)}(bytes, validity, A, len, meta), nodeidx, bufferidx
+end
+
+function build(f::Meta.Field, L::Meta.Map, batch, rb, de, nodeidx, bufferidx, convert)
+    @debug 2 "building array: L = $L"
+    validity = buildbitmap(batch, rb, nodeidx, bufferidx)
+    bufferidx += 1
+    buffer = rb.buffers[bufferidx]
+    ooff = batch.pos + buffer.offset
+    OT = Int32
+    bytes, offs = reinterp(OT, batch, buffer, rb.compression)
+    offsets = Offsets(bytes, offs)
+    bufferidx += 1
+    len = rb.nodes[nodeidx].length
+    nodeidx += 1
+    A, nodeidx, bufferidx = build(f.children[1], batch, rb, de, nodeidx, bufferidx, convert)
+    meta = buildmetadata(f.custom_metadata)
+    T = juliaeltype(f, meta, convert)
+    return Map{T, OT, typeof(A)}(validity, offsets, A, len, meta), nodeidx, bufferidx
+end
+
+function build(f::Meta.Field, L::Meta.Struct, batch, rb, de, nodeidx, bufferidx, convert)
+    @debug 2 "building array: L = $L"
+    validity = buildbitmap(batch, rb, nodeidx, bufferidx)
+    bufferidx += 1
+    len = rb.nodes[nodeidx].length
+    vecs = []
+    nodeidx += 1
+    for child in f.children
+        A, nodeidx, bufferidx = build(child, batch, rb, de, nodeidx, bufferidx, convert)
+        push!(vecs, A)
+    end
+    data = Tuple(vecs)
+    meta = buildmetadata(f.custom_metadata)
+    T = juliaeltype(f, meta, convert)
+    return Struct{T, typeof(data)}(validity, data, len, meta), nodeidx, bufferidx
+end
+
+function build(f::Meta.Field, L::Meta.Union, batch, rb, de, nodeidx, bufferidx, convert)
+    @debug 2 "building array: L = $L"
+    buffer = rb.buffers[bufferidx]
+    bytes, typeIds = reinterp(UInt8, batch, buffer, rb.compression)
+    bufferidx += 1
+    if L.mode == Meta.UnionMode.Dense
+        buffer = rb.buffers[bufferidx]
+        bytes2, offsets = reinterp(Int32, batch, buffer, rb.compression)
+        bufferidx += 1
+    end
+    vecs = []
+    nodeidx += 1
+    for child in f.children
+        A, nodeidx, bufferidx = build(child, batch, rb, de, nodeidx, bufferidx, convert)
+        push!(vecs, A)
+    end
+    data = Tuple(vecs)
+    meta = buildmetadata(f.custom_metadata)
+    T = juliaeltype(f, meta, convert)
+    if L.mode == Meta.UnionMode.Dense
+        B = DenseUnion{T, typeof(data)}(bytes, bytes2, typeIds, offsets, data, meta)
+    else
+        B = SparseUnion{T, typeof(data)}(bytes, typeIds, data, meta)
+    end
+    return B, nodeidx, bufferidx
+end
+
+function build(f::Meta.Field, L::Meta.Null, batch, rb, de, nodeidx, bufferidx, convert)
+    @debug 2 "building array: L = $L"
+    return MissingVector(rb.nodes[nodeidx].length), nodeidx + 1, bufferidx
+end
+
+# primitives
+function build(f::Meta.Field, ::L, batch, rb, de, nodeidx, bufferidx, convert) where {L}
+    @debug 2 "building array: L = $L"
+    validity = buildbitmap(batch, rb, nodeidx, bufferidx)
+    bufferidx += 1
+    buffer = rb.buffers[bufferidx]
+    meta = buildmetadata(f.custom_metadata)
+    # get storage type (non-converted)
+    T = juliaeltype(f, nothing, false)
+    @debug 2 "storage type for primitive: T = $T"
+    bytes, A = reinterp(Base.nonmissingtype(T), batch, buffer, rb.compression)
+    len = rb.nodes[nodeidx].length
+    T = juliaeltype(f, meta, convert)
+    @debug 2 "final julia type for primitive: T = $T"
+    return Primitive(T, bytes, validity, A, len, meta), nodeidx + 1, bufferidx + 1
+end
+
+function build(f::Meta.Field, L::Meta.Bool, batch, rb, de, nodeidx, bufferidx, convert)
+    @debug 2 "building array: L = $L"
+    validity = buildbitmap(batch, rb, nodeidx, bufferidx)
+    bufferidx += 1
+    buffer = rb.buffers[bufferidx]
+    meta = buildmetadata(f.custom_metadata)
+    # get storage type (non-converted)
+    T = juliaeltype(f, nothing, false)
+    @debug 2 "storage type for primitive: T = $T"
+    buffer = rb.buffers[bufferidx]
+    voff = batch.pos + buffer.offset
+    node = rb.nodes[nodeidx]
+    if rb.compression === nothing
+        decodedbytes = batch.bytes
+        pos = voff
+        # return ValidityBitmap(batch.bytes, voff, node.length, node.null_count)
+    else
+        # compressed
+        ptr = pointer(batch.bytes, voff)
+        _, decodedbytes = uncompress(ptr, buffer, rb.compression)
+        pos = 1
+        # return ValidityBitmap(decodedbytes, 1, node.length, node.null_count)
+    end
+    len = rb.nodes[nodeidx].length
+    T = juliaeltype(f, meta, convert)
+    return BoolVector{T}(decodedbytes, pos, validity, len, meta), nodeidx + 1, bufferidx + 1
+end
diff --git a/julia/Arrow/src/utils.jl b/julia/Arrow/src/utils.jl
new file mode 100644
index 0000000..d1b851b
--- /dev/null
+++ b/julia/Arrow/src/utils.jl
@@ -0,0 +1,197 @@
+# 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.
+
+# Determines the total number of bytes needed to store `n` bytes with padding.
+# Note that the Arrow standard requires buffers to be aligned to 8-byte boundaries.
+padding(n::Integer, alignment) = ((n + alignment - 1) ÷ alignment) * alignment
+
+paddinglength(n::Integer, alignment) = padding(n, alignment) - n
+
+function writezeros(io::IO, n::Integer)
+    s = 0
+    for i ∈ 1:n
+        s += Base.write(io, 0x00)
+    end
+    s
+end
+
+# efficient writing of arrays
+writearray(io, col) = writearray(io, maybemissing(eltype(col)), col)
+
+function writearray(io::IO, ::Type{T}, col) where {T}
+    if col isa Vector{T}
+        n = Base.write(io, col)
+    elseif isbitstype(T) && (col isa Vector{Union{T, Missing}} || col isa SentinelVector{T, T, Missing, Vector{T}})
+        # need to write the non-selector bytes of isbits Union Arrays
+        n = Base.unsafe_write(io, pointer(col), sizeof(T) * length(col))
+    elseif col isa ChainedVector
+        n = 0
+        for A in col.arrays
+            n += writearray(io, T, A)
+        end
+    else
+        n = 0
+        for x in col
+            n += Base.write(io, coalesce(x, ArrowTypes.default(T)))
+        end
+    end
+    return n
+end
+
+getbit(v::UInt8, n::Integer) = Bool((v & 0x02^(n - 1)) >> (n - 1))
+
+function setbit(v::UInt8, b::Bool, n::Integer)
+    if b
+        v | 0x02^(n - 1)
+    else
+        v & (0xff ⊻ 0x02^(n - 1))
+    end
+end
+
+# Determines the number of bytes used by `n` bits, optionally with padding.
+function bitpackedbytes(n::Integer, alignment)
+    ℓ = cld(n, 8)
+    return ℓ + paddinglength(ℓ, alignment)
+end
+
+# count # of missing elements in an iterable
+nullcount(col) = count(ismissing, col)
+
+# like startswith/endswith for strings, but on byte buffers
+function _startswith(a::AbstractVector{UInt8}, pos::Integer, b::AbstractVector{UInt8})
+    for i = 1:length(b)
+        @inbounds check = a[pos + i - 1] == b[i]
+        check || return false
+    end
+    return true
+end
+
+function _endswith(a::AbstractVector{UInt8}, endpos::Integer, b::AbstractVector{UInt8})
+    aoff = endpos - length(b) + 1
+    for i = 1:length(b)
+        @inbounds check = a[aoff] == b[i]
+        check || return false
+        aoff += 1
+    end
+    return true
+end
+
+# read a single element from a byte vector
+# copied from read(::IOBuffer, T) in Base
+function readbuffer(t::AbstractVector{UInt8}, pos::Integer, ::Type{T}) where {T}
+    GC.@preserve t begin
+        ptr::Ptr{T} = pointer(t, pos)
+        x = unsafe_load(ptr)
+    end
+end
+
+# given a number of unique values; what dict encoding _index_ type is most appropriate
+encodingtype(n) = n < div(typemax(Int8), 2) ? Int8 : n < div(typemax(Int16), 2) ? Int16 : n < div(typemax(Int32), 2) ? Int32 : Int64
+
+# lazily call convert(T, x) on getindex for each x in data
+struct Converter{T, A} <: AbstractVector{T}
+    data::A
+end
+
+converter(::Type{T}, x::A) where {T, A} = Converter{eltype(A) >: Missing ? Union{T, Missing} : T, A}(x)
+converter(::Type{T}, x::ChainedVector{A}) where {T, A} = ChainedVector([converter(T, x) for x in x.arrays])
+
+Base.IndexStyle(::Type{<:Converter}) = Base.IndexLinear()
+Base.size(x::Converter) = (length(x.data),)
+Base.eltype(x::Converter{T, A}) where {T, A} = T
+Base.getindex(x::Converter{T}, i::Int) where {T} = ArrowTypes.arrowconvert(T, getindex(x.data, i))
+
+maybemissing(::Type{T}) where {T} = T === Missing ? Missing : Base.nonmissingtype(T)
+
+function getfooter(filebytes)
+    len = readbuffer(filebytes, length(filebytes) - 9, Int32)
+    FlatBuffers.getrootas(Meta.Footer, filebytes[end-(9 + len):end-10], 0)
+end
+
+function getrb(filebytes)
+    f = getfooter(filebytes)
+    rb = f.recordBatches[1]
+    return filebytes[rb.offset+1:(rb.offset+1+rb.metaDataLength)]
+    # FlatBuffers.getrootas(Meta.Message, filebytes, rb.offset)
+end
+
+function readmessage(filebytes, off=9)
+    @assert readbuffer(filebytes, off, UInt32) === 0xFFFFFFFF
+    len = readbuffer(filebytes, off + 4, Int32)
+
+    FlatBuffers.getrootas(Meta.Message, filebytes, off + 8)
+end
+
+# a custom Channel type that only allows put!-ing objects in a specific, monotonically increasing order
+struct OrderedChannel{T}
+    chan::Channel{T}
+    cond::Threads.Condition
+    i::Ref{Int}
+end
+
+OrderedChannel{T}(sz) where {T} = OrderedChannel{T}(Channel{T}(sz), Threads.Condition(), Ref(1))
+Base.iterate(ch::OrderedChannel, st...) = iterate(ch.chan, st...)
+
+macro lock(obj, expr)
+    esc(quote
+        lock($obj)
+        try
+            $expr
+        finally
+            unlock($obj)
+        end
+    end)
+end
+
+# when put!-ing an object, operation may have to wait until other tasks have put their
+# objects to ensure the channel is ordered correctly
+function Base.put!(ch::OrderedChannel{T}, x::T, i::Integer, incr::Bool=false) where {T}
+    @lock ch.cond begin
+        while ch.i[] < i
+            # channel index too early, need to wait for other tasks to put their objects first
+            wait(ch.cond)
+        end
+        # now it's our turn
+        put!(ch.chan, x)
+        if incr
+            ch.i[] += 1
+        end
+        # wake up tasks that may be waiting to put their objects
+        notify(ch.cond)
+    end
+    return
+end
+
+function Base.close(ch::OrderedChannel)
+    @lock ch.cond begin
+        # just need to ensure any tasks waiting to put their tasks have had a chance to put
+        while Base.n_waiters(ch.cond) > 0
+            wait(ch.cond)
+        end
+        close(ch.chan)
+    end
+    return
+end
+
+struct Lockable{T}
+    x::T
+    lock::ReentrantLock
+end
+
+Lockable(x::T) where {T} = Lockable{T}(x, ReentrantLock())
+
+Base.lock(x::Lockable) = lock(x.lock)
+Base.unlock(x::Lockable) = unlock(x.lock)
diff --git a/julia/Arrow/src/write.jl b/julia/Arrow/src/write.jl
new file mode 100644
index 0000000..77d954f
--- /dev/null
+++ b/julia/Arrow/src/write.jl
@@ -0,0 +1,454 @@
+# 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.
+
+const OBJ_METADATA = IdDict{Any, Dict{String, String}}()
+
+"""
+    Arrow.setmetadata!(x, metadata::Dict{String, String})
+
+Set the metadata for any object, provided as a `Dict{String, String}`.
+Metadata attached to a table or column will be serialized when written
+as a stream or file.
+"""
+function setmetadata!(x, meta::Dict{String, String})
+    OBJ_METADATA[x] = meta
+    return
+end
+
+"""
+    Arrow.getmetadata(x) => Dict{String, String}
+
+Retrieve any metadata (as a `Dict{String, String}`) attached to an object.
+
+Metadata may be attached to any object via [`Arrow.setmetadata!`](@ref),
+or deserialized via the arrow format directly (the format allows attaching metadata
+to table, column, and other objects).
+"""
+getmetadata(x, default=nothing) = get(OBJ_METADATA, x, default)
+
+"""
+    Arrow.write(io::IO, tbl)
+    Arrow.write(file::String, tbl)
+    tbl |> Arrow.write(io_or_file)
+
+Write any [Tables.jl](https://github.com/JuliaData/Tables.jl)-compatible `tbl` out as arrow formatted data.
+Providing an `io::IO` argument will cause the data to be written to it
+in the ["streaming" format](https://arrow.apache.org/docs/format/Columnar.html#ipc-streaming-format), unless `file=true` keyword argument is passed.
+Providing a `file::String` argument will result in the ["file" format](https://arrow.apache.org/docs/format/Columnar.html#ipc-file-format) being written.
+
+Multiple record batches will be written based on the number of
+`Tables.partitions(tbl)` that are provided; by default, this is just
+one for a given table, but some table sources support automatic
+partitioning. Note you can turn multiple table objects into partitions
+by doing `Tables.partitioner([tbl1, tbl2, ...])`, but note that
+each table must have the exact same `Tables.Schema`.
+
+By default, `Arrow.write` will use multiple threads to write multiple
+record batches simultaneously (e.g. if julia is started with `julia -t 8` or the `JULIA_NUM_THREADS` environment variable is set).
+
+Supported keyword arguments to `Arrow.write` include:
+  * `compress`: possible values include `:lz4`, `:zstd`, or your own initialized `LZ4FrameCompressor` or `ZstdCompressor` objects; will cause all buffers in each record batch to use the respective compression encoding
+  * `alignment::Int=8`: specify the number of bytes to align buffers to when written in messages; strongly recommended to only use alignment values of 8 or 64 for modern memory cache line optimization
+  * `dictencode::Bool=false`: whether all columns should use dictionary encoding when being written; to dict encode specific columns, wrap the column/array in `Arrow.DictEncode(col)`
+  * `dictencodenested::Bool=false`: whether nested data type columns should also dict encode nested arrays/buffers; other language implementations [may not support this](https://arrow.apache.org/docs/status.html)
+  * `denseunions::Bool=true`: whether Julia `Vector{<:Union}` arrays should be written using the dense union layout; passing `false` will result in the sparse union layout
+  * `largelists::Bool=false`: causes list column types to be written with Int64 offset arrays; mainly for testing purposes; by default, Int64 offsets will be used only if needed
+  * `file::Bool=false`: if a an `io` argument is being written to, passing `file=true` will cause the arrow file format to be written instead of just IPC streaming
+"""
+function write end
+
+write(io_or_file; kw...) = x -> write(io_or_file, x; kw...)
+
+function write(file::String, tbl; largelists::Bool=false, compress::Union{Nothing, Symbol, LZ4FrameCompressor, ZstdCompressor}=nothing, denseunions::Bool=true, dictencode::Bool=false, dictencodenested::Bool=false, alignment::Int=8)
+    open(file, "w") do io
+        write(io, tbl, true, largelists, compress, denseunions, dictencode, dictencodenested, alignment)
+    end
+    return file
+end
+
+function write(io::IO, tbl; largelists::Bool=false, compress::Union{Nothing, Symbol, LZ4FrameCompressor, ZstdCompressor}=nothing, denseunions::Bool=true, dictencode::Bool=false, dictencodenested::Bool=false, alignment::Int=8, file::Bool=false)
+    return write(io, tbl, file, largelists, compress, denseunions, dictencode, dictencodenested, alignment)
+end
+
+function write(io, source, writetofile, largelists, compress, denseunions, dictencode, dictencodenested, alignment)
+    if compress === :lz4
+        compress = LZ4_FRAME_COMPRESSOR[]
+    elseif compress === :zstd
+        compress = ZSTD_COMPRESSOR[]
+    elseif compress isa Symbol
+        throw(ArgumentError("unsupported compress keyword argument value: $compress. Valid values include `:lz4` or `:zstd`"))
+    end
+    if writetofile
+        @debug 1 "starting write of arrow formatted file"
+        Base.write(io, "ARROW1\0\0")
+    end
+    msgs = OrderedChannel{Message}(Inf)
+    # build messages
+    sch = Ref{Tables.Schema}()
+    firstcols = Ref{Any}()
+    dictencodings = Dict{Int64, Any}() # Lockable{DictEncoding}
+    blocks = (Block[], Block[])
+    # start message writing from channel
+    tsk = Threads.@spawn for msg in msgs
+        Base.write(io, msg, blocks, sch, alignment)
+    end
+    @sync for (i, tbl) in enumerate(Tables.partitions(source))
+        @debug 1 "processing table partition i = $i"
+        if i == 1
+            cols = toarrowtable(tbl, dictencodings, largelists, compress, denseunions, dictencode, dictencodenested)
+            sch[] = Tables.schema(cols)
+            firstcols[] = cols
+            put!(msgs, makeschemamsg(sch[], cols), i)
+            if !isempty(dictencodings)
+                des = sort!(collect(dictencodings); by=x->x.first, rev=true)
+                for (id, delock) in des
+                    # assign dict encoding ids
+                    de = delock.x
+                    dictsch = Tables.Schema((:col,), (eltype(de.data),))
+                    put!(msgs, makedictionarybatchmsg(dictsch, (col=de.data,), id, false, alignment), i)
+                end
+            end
+            put!(msgs, makerecordbatchmsg(sch[], cols, alignment), i, true)
+        else
+            Threads.@spawn begin
+                cols = toarrowtable(tbl, dictencodings, largelists, compress, denseunions, dictencode, dictencodenested)
+                if !isempty(cols.dictencodingdeltas)
+                    for de in cols.dictencodingdeltas
+                        dictsch = Tables.Schema((:col,), (eltype(de.data),))
+                        put!(msgs, makedictionarybatchmsg(dictsch, (col=de.data,), de.id, true, alignment), i)
+                    end
+                end
+                put!(msgs, makerecordbatchmsg(sch[], cols, alignment), i, true)
+            end
+        end
+    end
+    # close our message-writing channel, no further put!-ing is allowed
+    close(msgs)
+    # now wait for our message-writing task to finish writing
+    wait(tsk)
+    # write empty message
+    if !writetofile
+        Base.write(io, Message(UInt8[], nothing, 0, true, false), blocks, sch, alignment)
+    end
+    if writetofile
+        b = FlatBuffers.Builder(1024)
+        schfoot = makeschema(b, sch[], firstcols[])
+        if !isempty(blocks[1])
+            N = length(blocks[1])
+            Meta.footerStartRecordBatchesVector(b, N)
+            for blk in Iterators.reverse(blocks[1])
+                Meta.createBlock(b, blk.offset, blk.metaDataLength, blk.bodyLength)
+            end
+            recordbatches = FlatBuffers.endvector!(b, N)
+        else
+            recordbatches = FlatBuffers.UOffsetT(0)
+        end
+        if !isempty(blocks[2])
+            N = length(blocks[2])
+            Meta.footerStartDictionariesVector(b, N)
+            for blk in Iterators.reverse(blocks[2])
+                Meta.createBlock(b, blk.offset, blk.metaDataLength, blk.bodyLength)
+            end
+            dicts = FlatBuffers.endvector!(b, N)
+        else
+            dicts = FlatBuffers.UOffsetT(0)
+        end
+        Meta.footerStart(b)
+        Meta.footerAddVersion(b, Meta.MetadataVersion.V4)
+        Meta.footerAddSchema(b, schfoot)
+        Meta.footerAddDictionaries(b, dicts)
+        Meta.footerAddRecordBatches(b, recordbatches)
+        foot = Meta.footerEnd(b)
+        FlatBuffers.finish!(b, foot)
+        footer = FlatBuffers.finishedbytes(b)
+        Base.write(io, footer)
+        Base.write(io, Int32(length(footer)))
+        Base.write(io, "ARROW1")
+    end
+    return io
+end
+
+struct ToArrowTable
+    sch::Tables.Schema
+    cols::Vector{Any}
+    metadata::Union{Nothing, Dict{String, String}}
+    dictencodingdeltas::Vector{DictEncoding}
+end
+
+function toarrowtable(x, dictencodings, largelists, compress, denseunions, dictencode, dictencodenested)
+    @debug 1 "converting input table to arrow formatted columns"
+    cols = Tables.columns(x)
+    meta = getmetadata(cols)
+    sch = Tables.schema(cols)
+    types = collect(sch.types)
+    N = length(types)
+    newcols = Vector{Any}(undef, N)
+    newtypes = Vector{Type}(undef, N)
+    dictencodingdeltas = DictEncoding[]
+    Tables.eachcolumn(sch, cols) do col, i, nm
+        newcol = toarrowvector(col, i, dictencodings, dictencodingdeltas; compression=compress, largelists=largelists, denseunions=denseunions, dictencode=dictencode, dictencodenested=dictencodenested)
+        newtypes[i] = eltype(newcol)
+        newcols[i] = newcol
+    end
+    return ToArrowTable(Tables.Schema(sch.names, newtypes), newcols, meta, dictencodingdeltas)
+end
+
+Tables.columns(x::ToArrowTable) = x
+Tables.rowcount(x::ToArrowTable) = length(x.cols) == 0 ? 0 : length(x.cols[1])
+Tables.schema(x::ToArrowTable) = x.sch
+Tables.columnnames(x::ToArrowTable) = x.sch.names
+Tables.getcolumn(x::ToArrowTable, i::Int) = x.cols[i]
+
+struct Message
+    msgflatbuf
+    columns
+    bodylen
+    isrecordbatch::Bool
+    blockmsg::Bool
+end
+
+struct Block
+    offset::Int64
+    metaDataLength::Int32
+    bodyLength::Int64
+end
+
+function Base.write(io::IO, msg::Message, blocks, sch, alignment)
+    metalen = padding(length(msg.msgflatbuf), alignment)
+    @debug 1 "writing message: metalen = $metalen, bodylen = $(msg.bodylen), isrecordbatch = $(msg.isrecordbatch)"
+    if msg.blockmsg
+        push!(blocks[msg.isrecordbatch ? 1 : 2], Block(position(io), metalen + 8, msg.bodylen))
+    end
+    # now write the final message spec out
+    # continuation byte
+    n = Base.write(io, 0xFFFFFFFF)
+    # metadata length
+    n += Base.write(io, Int32(metalen))
+    # message flatbuffer
+    n += Base.write(io, msg.msgflatbuf)
+    n += writezeros(io, paddinglength(length(msg.msgflatbuf), alignment))
+    # message body
+    if msg.columns !== nothing
+        # write out buffers
+        for col in Tables.Columns(msg.columns)
+            writebuffer(io, col, alignment)
+        end
+    end
+    return n
+end
+
+function makemessage(b, headerType, header, columns=nothing, bodylen=0)
+    # write the message flatbuffer object
+    Meta.messageStart(b)
+    Meta.messageAddVersion(b, Meta.MetadataVersion.V5)
+    Meta.messageAddHeaderType(b, headerType)
+    Meta.messageAddHeader(b, header)
+    Meta.messageAddBodyLength(b, Int64(bodylen))
+    # Meta.messageAddCustomMetadata(b, meta)
+    # Meta.messageStartCustomMetadataVector(b, num_meta_elems)
+    msg = Meta.messageEnd(b)
+    FlatBuffers.finish!(b, msg)
+    return Message(FlatBuffers.finishedbytes(b), columns, bodylen, headerType == Meta.RecordBatch, headerType == Meta.RecordBatch || headerType == Meta.DictionaryBatch)
+end
+
+function makeschema(b, sch::Tables.Schema{names}, columns) where {names}
+    # build Field objects
+    N = length(names)
+    fieldoffsets = [fieldoffset(b, names[i], columns.cols[i]) for i = 1:N]
+    Meta.schemaStartFieldsVector(b, N)
+    for off in Iterators.reverse(fieldoffsets)
+        FlatBuffers.prependoffset!(b, off)
+    end
+    fields = FlatBuffers.endvector!(b, N)
+    if columns.metadata !== nothing
+        kvs = columns.metadata
+        kvoffs = Vector{FlatBuffers.UOffsetT}(undef, length(kvs))
+        for (i, (k, v)) in enumerate(kvs)
+            koff = FlatBuffers.createstring!(b, String(k))
+            voff = FlatBuffers.createstring!(b, String(v))
+            Meta.keyValueStart(b)
+            Meta.keyValueAddKey(b, koff)
+            Meta.keyValueAddValue(b, voff)
+            kvoffs[i] = Meta.keyValueEnd(b)
+        end
+        Meta.schemaStartCustomMetadataVector(b, length(kvs))
+        for off in Iterators.reverse(kvoffs)
+            FlatBuffers.prependoffset!(b, off)
+        end
+        meta = FlatBuffers.endvector!(b, length(kvs))
+    else
+        meta = FlatBuffers.UOffsetT(0)
+    end
+    # write schema object
+    Meta.schemaStart(b)
+    Meta.schemaAddEndianness(b, Meta.Endianness.Little)
+    Meta.schemaAddFields(b, fields)
+    Meta.schemaAddCustomMetadata(b, meta)
+    return Meta.schemaEnd(b)
+end
+
+function makeschemamsg(sch::Tables.Schema, columns)
+    @debug 1 "building schema message: sch = $sch"
+    b = FlatBuffers.Builder(1024)
+    schema = makeschema(b, sch, columns)
+    return makemessage(b, Meta.Schema, schema)
+end
+
+function fieldoffset(b, name, col)
+    nameoff = FlatBuffers.createstring!(b, String(name))
+    T = eltype(col)
+    nullable = T >: Missing
+    # check for custom metadata
+    if getmetadata(col) !== nothing
+        kvs = getmetadata(col)
+        kvoffs = Vector{FlatBuffers.UOffsetT}(undef, length(kvs))
+        for (i, (k, v)) in enumerate(kvs)
+            koff = FlatBuffers.createstring!(b, String(k))
+            voff = FlatBuffers.createstring!(b, String(v))
+            Meta.keyValueStart(b)
+            Meta.keyValueAddKey(b, koff)
+            Meta.keyValueAddValue(b, voff)
+            kvoffs[i] = Meta.keyValueEnd(b)
+        end
+        Meta.fieldStartCustomMetadataVector(b, length(kvs))
+        for off in Iterators.reverse(kvoffs)
+            FlatBuffers.prependoffset!(b, off)
+        end
+        meta = FlatBuffers.endvector!(b, length(kvs))
+    else
+        meta = FlatBuffers.UOffsetT(0)
+    end
+    # build dictionary
+    if isdictencoded(col)
+        encodingtype = indtype(col)
+        IT, inttype, _ = arrowtype(b, encodingtype)
+        Meta.dictionaryEncodingStart(b)
+        Meta.dictionaryEncodingAddId(b, Int64(getid(col)))
+        Meta.dictionaryEncodingAddIndexType(b, inttype)
+        # TODO: support isOrdered?
+        Meta.dictionaryEncodingAddIsOrdered(b, false)
+        dict = Meta.dictionaryEncodingEnd(b)
+    else
+        dict = FlatBuffers.UOffsetT(0)
+    end
+    type, typeoff, children = arrowtype(b, col)
+    if children !== nothing
+        Meta.fieldStartChildrenVector(b, length(children))
+        for off in Iterators.reverse(children)
+            FlatBuffers.prependoffset!(b, off)
+        end
+        children = FlatBuffers.endvector!(b, length(children))
+    else
+        Meta.fieldStartChildrenVector(b, 0)
+        children = FlatBuffers.endvector!(b, 0)
+    end
+    # build field object
+    if isdictencoded(col)
+        @debug 1 "building field: name = $name, nullable = $nullable, T = $T, type = $type, inttype = $IT, dictionary id = $(getid(col))"
+    else
+        @debug 1 "building field: name = $name, nullable = $nullable, T = $T, type = $type"
+    end
+    Meta.fieldStart(b)
+    Meta.fieldAddName(b, nameoff)
+    Meta.fieldAddNullable(b, nullable)
+    Meta.fieldAddTypeType(b, type)
+    Meta.fieldAddType(b, typeoff)
+    Meta.fieldAddDictionary(b, dict)
+    Meta.fieldAddChildren(b, children)
+    Meta.fieldAddCustomMetadata(b, meta)
+    return Meta.fieldEnd(b)
+end
+
+struct FieldNode
+    length::Int64
+    null_count::Int64
+end
+
+struct Buffer
+    offset::Int64
+    length::Int64
+end
+
+function makerecordbatchmsg(sch::Tables.Schema{names, types}, columns, alignment) where {names, types}
+    b = FlatBuffers.Builder(1024)
+    recordbatch, bodylen = makerecordbatch(b, sch, columns, alignment)
+    return makemessage(b, Meta.RecordBatch, recordbatch, columns, bodylen)
+end
+
+function makerecordbatch(b, sch::Tables.Schema{names, types}, columns, alignment) where {names, types}
+    nrows = Tables.rowcount(columns)
+    
+    compress = nothing
+    fieldnodes = FieldNode[]
+    fieldbuffers = Buffer[]
+    bufferoffset = 0
+    for col in Tables.Columns(columns)
+        if col isa Compressed
+            compress = compressiontype(col)
+        end
+        bufferoffset = makenodesbuffers!(col, fieldnodes, fieldbuffers, bufferoffset, alignment)
+    end
+    @debug 1 "building record batch message: nrows = $nrows, sch = $sch, compress = $compress"
+
+    # write field nodes objects
+    FN = length(fieldnodes)
+    Meta.recordBatchStartNodesVector(b, FN)
+    for fn in Iterators.reverse(fieldnodes)
+        Meta.createFieldNode(b, fn.length, fn.null_count)
+    end
+    nodes = FlatBuffers.endvector!(b, FN)
+
+    # write buffer objects
+    bodylen = 0
+    BN = length(fieldbuffers)
+    Meta.recordBatchStartBuffersVector(b, BN)
+    for buf in Iterators.reverse(fieldbuffers)
+        Meta.createBuffer(b, buf.offset, buf.length)
+        bodylen += padding(buf.length, alignment)
+    end
+    buffers = FlatBuffers.endvector!(b, BN)
+
+    # compression
+    if compress !== nothing
+        Meta.bodyCompressionStart(b)
+        Meta.bodyCompressionAddCodec(b, compress)
+        Meta.bodyCompressionAddMethod(b, Meta.BodyCompressionMethod.BUFFER)
+        compression = Meta.bodyCompressionEnd(b)
+    else
+        compression = FlatBuffers.UOffsetT(0)
+    end
+
+    # write record batch object
+    @debug 1 "built record batch message: nrows = $nrows, nodes = $fieldnodes, buffers = $fieldbuffers, compress = $compress, bodylen = $bodylen"
+    Meta.recordBatchStart(b)
+    Meta.recordBatchAddLength(b, Int64(nrows))
+    Meta.recordBatchAddNodes(b, nodes)
+    Meta.recordBatchAddBuffers(b, buffers)
+    Meta.recordBatchAddCompression(b, compression)
+    return Meta.recordBatchEnd(b), bodylen
+end
+
+function makedictionarybatchmsg(sch, columns, id, isdelta, alignment)
+    @debug 1 "building dictionary message: id = $id, sch = $sch, isdelta = $isdelta"
+    b = FlatBuffers.Builder(1024)
+    recordbatch, bodylen = makerecordbatch(b, sch, columns, alignment)
+    Meta.dictionaryBatchStart(b)
+    Meta.dictionaryBatchAddId(b, Int64(id))
+    Meta.dictionaryBatchAddData(b, recordbatch)
+    Meta.dictionaryBatchAddIsDelta(b, isdelta)
+    dictionarybatch = Meta.dictionaryBatchEnd(b)
+    return makemessage(b, Meta.DictionaryBatch, dictionarybatch, columns, bodylen)
+end
diff --git a/julia/Arrow/test/arrowjson.jl b/julia/Arrow/test/arrowjson.jl
new file mode 100644
index 0000000..7921e1f
--- /dev/null
+++ b/julia/Arrow/test/arrowjson.jl
@@ -0,0 +1,611 @@
+# 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.
+
+module ArrowJSON
+
+using Mmap
+using StructTypes, JSON3, Tables, SentinelArrays, Arrow
+
+# read json files as "table"
+# write to arrow stream/file
+# read arrow stream/file back
+
+abstract type Type end
+Type() = Null("null")
+StructTypes.StructType(::Base.Type{Type}) = StructTypes.AbstractType()
+
+children(::Base.Type{T}) where {T} = Field[]
+
+mutable struct Int <: Type
+    name::String
+    bitWidth::Int64
+    isSigned::Base.Bool
+end
+
+Int() = Int("", 0, true)
+Type(::Base.Type{T}) where {T <: Integer} = Int("int", 8 * sizeof(T), T <: Signed)
+StructTypes.StructType(::Base.Type{Int}) = StructTypes.Mutable()
+function juliatype(f, x::Int)
+    T = x.bitWidth == 8 ? Int8 : x.bitWidth == 16 ? Int16 :
+        x.bitWidth == 32 ? Int32 : x.bitWidth == 64 ? Int64 : Int128
+    return x.isSigned ? T : unsigned(T)
+end
+
+struct FloatingPoint <: Type
+    name::String
+    precision::String
+end
+
+Type(::Base.Type{T}) where {T <: AbstractFloat} = FloatingPoint("floatingpoint", T == Float16 ? "HALF" : T == Float32 ? "SINGLE" : "DOUBLE")
+StructTypes.StructType(::Base.Type{FloatingPoint}) = StructTypes.Struct()
+juliatype(f, x::FloatingPoint) = x.precision == "HALF" ? Float16 : x.precision == "SINGLE" ? Float32 : Float64
+
+struct FixedSizeBinary <: Type
+    name::String
+    byteWidth::Int64
+end
+
+Type(::Base.Type{NTuple{N, UInt8}}) where {N} = FixedSizeBinary("fixedsizebinary", N)
+children(::Base.Type{NTuple{N, UInt8}}) where {N} = Field[]
+StructTypes.StructType(::Base.Type{FixedSizeBinary}) = StructTypes.Struct()
+juliatype(f, x::FixedSizeBinary) = NTuple{x.byteWidth, UInt8}
+
+struct Decimal <: Type
+    name::String
+    precision::Int32
+    scale::Int32
+end
+
+Type(::Base.Type{Arrow.Decimal{P, S}}) where {P, S} = Decimal("decimal", P, S)
+StructTypes.StructType(::Base.Type{Decimal}) = StructTypes.Struct()
+juliatype(f, x::Decimal) = Arrow.Decimal{x.precision, x.scale}
+
+mutable struct Timestamp <: Type
+    name::String
+    unit::String
+    timezone::Union{Nothing ,String}
+end
+
+Timestamp() = Timestamp("", "", nothing)
+unit(U) = U == Arrow.Meta.TimeUnit.SECOND ? "SECOND" :
+          U == Arrow.Meta.TimeUnit.MILLISECOND ? "MILLISECOND" :
+          U == Arrow.Meta.TimeUnit.MICROSECOND ? "MICROSECOND" : "NANOSECOND"
+Type(::Base.Type{Arrow.Timestamp{U, TZ}}) where {U, TZ} = Timestamp("timestamp", unit(U), TZ === nothing ? nothing : String(TZ))
+StructTypes.StructType(::Base.Type{Timestamp}) = StructTypes.Mutable()
+unitT(u) = u == "SECOND" ? Arrow.Meta.TimeUnit.SECOND :
+           u == "MILLISECOND" ? Arrow.Meta.TimeUnit.MILLISECOND :
+           u == "MICROSECOND" ? Arrow.Meta.TimeUnit.MICROSECOND : Arrow.Meta.TimeUnit.NANOSECOND
+juliatype(f, x::Timestamp) = Arrow.Timestamp{unitT(x.unit), x.timezone === nothing ? nothing : Symbol(x.timezone)}
+
+struct Duration <: Type
+    name::String
+    unit::String
+end
+
+Type(::Base.Type{Arrow.Duration{U}}) where {U} = Duration("duration", unit(U))
+StructTypes.StructType(::Base.Type{Duration}) = StructTypes.Struct()
+juliatype(f, x::Duration) = Arrow.Duration{unit%(x.unit)}
+
+struct Date <: Type
+    name::String
+    unit::String
+end
+
+Type(::Base.Type{Arrow.Date{U, T}}) where {U, T} = Date("date", U == Arrow.Meta.DateUnit.DAY ? "DAY" : "MILLISECOND")
+StructTypes.StructType(::Base.Type{Date}) = StructTypes.Struct()
+juliatype(f, x::Date) = Arrow.Date{x.unit == "DAY" ? Arrow.Meta.DateUnit.DAY : Arrow.Meta.DateUnit.MILLISECOND, x.unit == "DAY" ? Int32 : Int64}
+
+struct Time <: Type
+    name::String
+    unit::String
+    bitWidth::Int64
+end
+
+Type(::Base.Type{Arrow.Time{U, T}}) where {U, T} = Time("time", unit(U), 8 * sizeof(T))
+StructTypes.StructType(::Base.Type{Time}) = StructTypes.Struct()
+juliatype(f, x::Time) = Arrow.Time{unitT(x.unit), x.unit == "SECOND" || x.unit == "MILLISECOND" ? Int32 : Int64}
+
+struct Interval <: Type
+    name::String
+    unit::String
+end
+
+Type(::Base.Type{Arrow.Interval{U, T}}) where {U, T} = Interval("interval", U == Arrow.Meta.IntervalUnit.YEAR_MONTH ? "YEAR_MONTH" : "DAY_TIME")
+StructTypes.StructType(::Base.Type{Interval}) = StructTypes.Struct()
+juliatype(f, x::Interval) = Arrow.Interval{x.unit == "YEAR_MONTH" ? Arrow.Meta.IntervalUnit.YEAR_MONTH : Arrow.Meta.IntervalUnit.DAY_TIME, x.unit == "YEAR_MONTH" ? Int32 : Int64}
+
+struct UnionT <: Type
+    name::String
+    mode::String
+    typIds::Vector{Int64}
+end
+
+Type(::Base.Type{Arrow.UnionT{T, typeIds, U}}) where {T, typeIds, U} = UnionT("union", T == Arrow.Meta.UnionMode.Dense ? "DENSE" : "SPARSE", collect(typeIds))
+children(::Base.Type{Arrow.UnionT{T, typeIds, U}}) where {T, typeIds, U} = Field[Field("", fieldtype(U, i), nothing) for i = 1:fieldcount(U)]
+StructTypes.StructType(::Base.Type{UnionT}) = StructTypes.Struct()
+juliatype(f, x::UnionT) = Arrow.UnionT{x.mode == "DENSE" ? Arrow.Meta.UnionMode.DENSE : Arrow.Meta.UnionMode.SPARSE, Tuple(x.typeIds), Tuple{(juliatype(y) for y in f.children)...}}
+
+struct List <: Type
+    name::String
+end
+
+Type(::Base.Type{Vector{T}}) where {T} = List("list")
+children(::Base.Type{Vector{T}}) where {T} = [Field("item", T, nothing)]
+StructTypes.StructType(::Base.Type{List}) = StructTypes.Struct()
+juliatype(f, x::List) = Vector{juliatype(f.children[1])}
+
+struct LargeList <: Type
+    name::String
+end
+
+StructTypes.StructType(::Base.Type{LargeList}) = StructTypes.Struct()
+juliatype(f, x::LargeList) = Vector{juliatype(f.children[1])}
+
+struct FixedSizeList <: Type
+    name::String
+    listSize::Int64
+end
+
+Type(::Base.Type{NTuple{N, T}}) where {N, T} = FixedSizeList("fixedsizelist", N)
+children(::Base.Type{NTuple{N, T}}) where {N, T} = [Field("item", T, nothing)]
+StructTypes.StructType(::Base.Type{FixedSizeList}) = StructTypes.Struct()
+juliatype(f, x::FixedSizeList) = NTuple{x.listSize, juliatype(f.children[1])}
+
+struct Struct <: Type
+    name::String
+end
+
+Type(::Base.Type{NamedTuple{names, types}}) where {names, types} = Struct("struct")
+children(::Base.Type{NamedTuple{names, types}}) where {names, types} = [Field(names[i], fieldtype(types, i), nothing) for i = 1:length(names)]
+StructTypes.StructType(::Base.Type{Struct}) = StructTypes.Struct()
+juliatype(f, x::Struct) = NamedTuple{Tuple(Symbol(x.name) for x in f.children), Tuple{(juliatype(y) for y in f.children)...}}
+
+struct Map <: Type
+    name::String
+    keysSorted::Base.Bool
+end
+
+Type(::Base.Type{Dict{K, V}}) where {K, V} = Map("map", false)
+children(::Base.Type{Dict{K, V}}) where {K, V} = [Field("entries", Arrow.KeyValue{K, V}, nothing)]
+StructTypes.StructType(::Base.Type{Map}) = StructTypes.Struct()
+juliatype(f, x::Map) = Dict{juliatype(f.children[1].children[1]), juliatype(f.children[1].children[2])}
+
+Type(::Base.Type{Arrow.KeyValue{K, V}}) where {K, V} = Struct("struct")
+children(::Base.Type{Arrow.KeyValue{K, V}}) where {K, V} = [Field("key", K, nothing), Field("value", V, nothing)]
+
+struct Null <: Type
+    name::String
+end
+
+Type(::Base.Type{Missing}) = Null("null")
+StructTypes.StructType(::Base.Type{Null}) = StructTypes.Struct()
+juliatype(f, x::Null) = Missing
+
+struct Utf8 <: Type
+    name::String
+end
+
+Type(::Base.Type{<:String}) = Utf8("utf8")
+StructTypes.StructType(::Base.Type{Utf8}) = StructTypes.Struct()
+juliatype(f, x::Utf8) = String
+
+struct LargeUtf8 <: Type
+    name::String
+end
+
+StructTypes.StructType(::Base.Type{LargeUtf8}) = StructTypes.Struct()
+juliatype(f, x::LargeUtf8) = String
+
+struct Binary <: Type
+    name::String
+end
+
+Type(::Base.Type{Vector{UInt8}}) = Binary("binary")
+children(::Base.Type{Vector{UInt8}}) = Field[]
+StructTypes.StructType(::Base.Type{Binary}) = StructTypes.Struct()
+juliatype(f, x::Binary) = Vector{UInt8}
+
+struct LargeBinary <: Type
+    name::String
+end
+
+StructTypes.StructType(::Base.Type{LargeBinary}) = StructTypes.Struct()
+juliatype(f, x::LargeBinary) = Vector{UInt8}
+
+struct Bool <: Type
+    name::String
+end
+
+Type(::Base.Type{Base.Bool}) = Bool("bool")
+StructTypes.StructType(::Base.Type{Bool}) = StructTypes.Struct()
+juliatype(f, x::Bool) = Base.Bool
+
+StructTypes.subtypekey(::Base.Type{Type}) = :name
+
+const SUBTYPES = @eval (
+    int=Int,
+    floatingpoint=FloatingPoint,
+    fixedsizebinary=FixedSizeBinary,
+    decimal=Decimal,
+    timestamp=Timestamp,
+    duration=Duration,
+    date=Date,
+    time=Time,
+    interval=Interval,
+    union=UnionT,
+    list=List,
+    largelist=LargeList,
+    fixedsizelist=FixedSizeList,
+    $(Symbol("struct"))=Struct,
+    map=Map,
+    null=Null,
+    utf8=Utf8,
+    largeutf8=LargeUtf8,
+    binary=Binary,
+    largebinary=LargeBinary,
+    bool=Bool
+)
+
+StructTypes.subtypes(::Base.Type{Type}) = SUBTYPES
+
+const Metadata = Union{Nothing, Vector{NamedTuple{(:key, :value), Tuple{String, String}}}}
+Metadata() = nothing
+
+mutable struct DictEncoding
+    id::Int64
+    indexType::Type
+    isOrdered::Base.Bool
+end
+
+DictEncoding() = DictEncoding(0, Type(), false)
+StructTypes.StructType(::Base.Type{DictEncoding}) = StructTypes.Mutable()
+
+mutable struct Field
+    name::String
+    nullable::Base.Bool
+    type::Type
+    children::Vector{Field}
+    dictionary::Union{DictEncoding, Nothing}
+    metadata::Metadata
+end
+
+Field() = Field("", true, Type(), Field[], nothing, Metadata())
+StructTypes.StructType(::Base.Type{Field}) = StructTypes.Mutable()
+Base.copy(f::Field) = Field(f.name, f.nullable, f.type, f.children, f.dictionary, f.metadata)
+
+function juliatype(f::Field)
+    T = juliatype(f, f.type)
+    return f.nullable ? Union{T, Missing} : T
+end
+
+function Field(nm, ::Base.Type{T}, dictencodings) where {T}
+    S = Arrow.maybemissing(T)
+    type = Type(S)
+    ch = children(S)
+    if dictencodings !== nothing && haskey(dictencodings, nm)
+        dict = dictencodings[nm]
+    else
+        dict = nothing
+    end
+    return Field(nm, T !== S, type, ch, dict, nothing)
+end
+
+mutable struct Schema
+    fields::Vector{Field}
+    metadata::Metadata
+end
+
+Schema() = Schema(Field[], Metadata())
+StructTypes.StructType(::Base.Type{Schema}) = StructTypes.Mutable()
+
+struct Offsets{T} <: AbstractVector{T}
+    data::Vector{T}
+end
+
+Base.size(x::Offsets) = size(x.data)
+Base.getindex(x::Offsets, i::Base.Int) = getindex(x.data, i)
+
+mutable struct FieldData
+    name::String
+    count::Int64
+    VALIDITY::Union{Nothing, Vector{Int8}}
+    OFFSET::Union{Nothing, Offsets}
+    TYPE_ID::Union{Nothing, Vector{Int8}}
+    DATA::Union{Nothing, Vector{Any}}
+    children::Vector{FieldData}
+end
+
+FieldData() = FieldData("", 0, nothing, nothing, nothing, nothing, FieldData[])
+StructTypes.StructType(::Base.Type{FieldData}) = StructTypes.Mutable()
+
+function FieldData(nm, ::Base.Type{T}, col, dictencodings) where {T}
+    if dictencodings !== nothing && haskey(dictencodings, nm)
+        refvals = DataAPI.refarray(col.data)
+        if refvals !== col.data
+            IT = eltype(refvals)
+            col = (x - one(T) for x in refvals)
+        else
+            _, de = dictencodings[nm]
+            IT = de.indexType
+            vals = unique(col)
+            col = Arrow.DictEncoder(col, vals, Arrow.encodingtype(length(vals)))
+        end
+        return FieldData(nm, IT, col, nothing)
+    end
+    S = Arrow.maybemissing(T)
+    len = Arrow._length(col)
+    VALIDITY = OFFSET = TYPE_ID = DATA = nothing
+    children = FieldData[]
+    if S <: Pair
+        return FieldData(nm, Vector{Arrow.KeyValue{Arrow._keytype(S), Arrow._valtype(S)}}, (Arrow.KeyValue(k, v) for (k, v) in pairs(col)))
+    elseif S !== Missing
+        # VALIDITY
+        VALIDITY = Int8[!ismissing(x) for x in col]
+        # OFFSET
+        if S <: Vector || S == String
+            lenfun = S == String ? x->ismissing(x) ? 0 : sizeof(x) : x->ismissing(x) ? 0 : length(x)
+            tot = sum(lenfun, col)
+            if tot > 2147483647
+                OFFSET = String[String(lenfun(x)) for x in col]
+                pushfirst!(OFFSET, "0")
+            else
+                OFFSET = Int32[ismissing(x) ? 0 : lenfun(x) for x in col]
+                pushfirst!(OFFSET, 0)
+            end
+            OFFSET = Offsets(OFFSET)
+            push!(children, FieldData("item", eltype(S), Arrow.flatten(skipmissing(col)), dictencodings))
+        elseif S <: NTuple
+            if Arrow.ArrowTypes.gettype(S) == UInt8
+                DATA = [ismissing(x) ? Arrow.ArrowTypes.default(S) : String(collect(x)) for x in col]
+            else
+                push!(children, FieldData("item", Arrow.ArrowTypes.gettype(S), Arrow.flatten(coalesce(x, Arrow.ArrowTypes.default(S)) for x in col), dictencodings))
+            end
+        elseif S <: NamedTuple
+            for (nm, typ) in zip(fieldnames(S), fieldtypes(S))
+                push!(children, FieldData(String(nm), typ, (getfield(x, nm) for x in col), dictencodings))
+            end
+        elseif S <: Arrow.UnionT
+            U = eltype(S)
+            tids = Arrow.typeids(S) === nothing ? (0:fieldcount(U)) : Arrow.typeids(S)
+            TYPE_ID = [x === missing ? 0 : tids[Arrow.isatypeid(x, U)] for x in col]
+            if Arrow.unionmode(S) == Arrow.Meta.UnionMode.Dense
+                offs = zeros(Int32, fieldcount(U))
+                OFFSET = Int32[]
+                for x in col
+                    idx = x === missing ? 1 : Arrow.isatypeid(x, U)
+                    push!(OFFSET, offs[idx])
+                    offs[idx] += 1
+                end
+                for i = 1:fieldcount(U)
+                    SS = fieldtype(U, i)
+                    push!(children, FieldData("$i", SS, Arrow.filtered(i == 1 ? Union{SS, Missing} : Arrow.maybemissing(SS), col), dictencodings))
+                end
+            else
+                for i = 1:fieldcount(U)
+                    SS = fieldtype(U, i)
+                    push!(children, FieldData("$i", SS, Arrow.replaced(SS, col), dictencodings))
+                end
+            end
+        elseif S <: KeyValue
+            push!(children, FieldData("key", Arrow.keyvalueK(S), (x.key for x in col), dictencodings))
+            push!(children, FieldData("value", Arrow.keyvalueV(S), (x.value for x in col), dictencodings))
+        end
+    end
+    return FieldData(nm, len, VALIDITY, OFFSET, TYPE_ID, DATA, children)
+end
+
+mutable struct RecordBatch
+    count::Int64
+    columns::Vector{FieldData}
+end
+
+RecordBatch() = RecordBatch(0, FieldData[])
+StructTypes.StructType(::Base.Type{RecordBatch}) = StructTypes.Mutable()
+
+mutable struct DictionaryBatch
+    id::Int64
+    data::RecordBatch
+end
+
+DictionaryBatch() = DictionaryBatch(0, RecordBatch())
+StructTypes.StructType(::Base.Type{DictionaryBatch}) = StructTypes.Mutable()
+
+mutable struct DataFile <: Tables.AbstractColumns
+    schema::Schema
+    batches::Vector{RecordBatch}
+    dictionaries::Vector{DictionaryBatch}
+end
+
+Base.propertynames(x::DataFile) = (:schema, :batches, :dictionaries)
+
+function Base.getproperty(df::DataFile, nm::Symbol)
+    if nm === :schema
+        return getfield(df, :schema)
+    elseif nm === :batches
+        return getfield(df, :batches)
+    elseif nm === :dictionaries
+        return getfield(df, :dictionaries)
+    end
+    return Tables.getcolumn(df, nm)
+end
+
+DataFile() = DataFile(Schema(), RecordBatch[], DictionaryBatch[])
+StructTypes.StructType(::Base.Type{DataFile}) = StructTypes.Mutable()
+
+parsefile(file) = JSON3.read(Mmap.mmap(file), DataFile)
+
+# make DataFile satisfy Tables.jl interface
+function Tables.partitions(x::DataFile)
+    if isempty(x.batches)
+        # special case empty batches by producing a single DataFile w/ schema
+        return (DataFile(x.schema, RecordBatch[], x.dictionaries),)
+    else
+        return (DataFile(x.schema, [x.batches[i]], x.dictionaries) for i = 1:length(x.batches))
+    end
+end
+
+Tables.columns(x::DataFile) = x
+
+function Tables.schema(x::DataFile)
+    names = map(x -> x.name, x.schema.fields)
+    types = map(x -> juliatype(x), x.schema.fields)
+    return Tables.Schema(names, types)
+end
+
+Tables.columnnames(x::DataFile) =  map(x -> Symbol(x.name), x.schema.fields)
+
+function Tables.getcolumn(x::DataFile, i::Base.Int)
+    field = x.schema.fields[i]
+    type = juliatype(field)
+    return ChainedVector(ArrowArray{type}[ArrowArray{type}(field, length(x.batches) > 0 ? x.batches[j].columns[i] : FieldData(), x.dictionaries) for j = 1:length(x.batches)])
+end
+
+function Tables.getcolumn(x::DataFile, nm::Symbol)
+    i = findfirst(x -> x.name == String(nm), x.schema.fields)
+    return Tables.getcolumn(x, i)
+end
+
+struct ArrowArray{T} <: AbstractVector{T}
+    field::Field
+    fielddata::FieldData
+    dictionaries::Vector{DictionaryBatch}
+end
+ArrowArray(f::Field, fd::FieldData, d) = ArrowArray{juliatype(f)}(f, fd, d)
+Base.size(x::ArrowArray) = (x.fielddata.count,)
+
+function Base.getindex(x::ArrowArray{T}, i::Base.Int) where {T}
+    @boundscheck checkbounds(x, i)
+    S = Base.nonmissingtype(T)
+    if x.field.dictionary !== nothing
+        fielddata = x.dictionaries[findfirst(y -> y.id == x.field.dictionary.id, x.dictionaries)].data.columns[1]
+        field = copy(x.field)
+        field.dictionary = nothing
+        idx = x.fielddata.DATA[i] + 1
+        return ArrowArray(field, fielddata, x.dictionaries)[idx]
+    end
+    if T === Missing
+        return missing
+    elseif S <: UnionT
+        U = eltype(S)
+        tids = Arrow.typeids(S) === nothing ? (0:fieldcount(U)) : Arrow.typeids(S)
+        typeid = tids[x.fielddata.TYPE_ID[i]]
+        if Arrow.unionmode(S) == Arrow.Meta.UnionMode.DENSE
+            off = x.fielddata.OFFSET[i]
+            return ArrowArray(x.field.children[typeid+1], x.fielddata.children[typeid+1], x.dictionaries)[off]
+        else
+            return ArrowArray(x.field.children[typeid+1], x.fielddata.children[typeid+1], x.dictionaries)[i]
+        end
+    end
+    x.fielddata.VALIDITY[i] == 0 && return missing
+    if S <: Vector{UInt8}
+        return copy(codeunits(x.fielddata.DATA[i]))
+    elseif S <: String
+        return x.fielddata.DATA[i]
+    elseif S <: Vector
+        offs = x.fielddata.OFFSET
+        A = ArrowArray{eltype(S)}(x.field.children[1], x.fielddata.children[1], x.dictionaries)
+        return A[(offs[i] + 1):offs[i + 1]]
+    elseif S <: Dict
+        offs = x.fielddata.OFFSET
+        A = ArrowArray(x.field.children[1], x.fielddata.children[1], x.dictionaries)
+        return Dict(y.key => y.value for y in A[(offs[i] + 1):offs[i + 1]])
+    elseif S <: Tuple
+        if Arrow.ArrowTypes.gettype(S) == UInt8
+            A = x.fielddata.DATA
+            return Tuple(map(UInt8, collect(A[i][1:x.field.type.byteWidth])))
+        else
+            sz = x.field.type.listSize
+            A = ArrowArray{Arrow.ArrowTypes.gettype(S)}(x.field.children[1], x.fielddata.children[1], x.dictionaries)
+            off = (i - 1) * sz + 1
+            return Tuple(A[off:(off + sz - 1)])
+        end
+    elseif S <: NamedTuple
+        data = (ArrowArray(x.field.children[j], x.fielddata.children[j], x.dictionaries)[i] for j = 1:length(x.field.children))
+        return NamedTuple{fieldnames(S)}(Tuple(data))
+    elseif S == Int64 || S == UInt64
+        return parse(S, x.fielddata.DATA[i])
+    elseif S <: Arrow.Decimal
+        str = x.fielddata.DATA[i]
+        return S(parse(Int128, str))
+    elseif S <: Arrow.Date || S <: Arrow.Time
+        val = x.fielddata.DATA[i]
+        return Arrow.storagetype(S) == Int32 ? S(val) : S(parse(Int64, val))
+    elseif S <: Arrow.Timestamp
+        return S(parse(Int64, x.fielddata.DATA[i]))
+    else
+        return S(x.fielddata.DATA[i])
+    end
+end
+
+# take any Tables.jl source and write out arrow json datafile
+function DataFile(source)
+    fields = Field[]
+    metadata = nothing # TODO?
+    batches = RecordBatch[]
+    dictionaries = DictionaryBatch[]
+    dictencodings = Dict{String, Tuple{Base.Type, DictEncoding}}()
+    dictid = Ref(0)
+    for (i, tbl1) in Tables.partitions(source)
+        tbl = Arrow.toarrowtable(tbl1)
+        if i == 1
+            sch = Tables.schema(tbl)
+            for (nm, T, col) in zip(sch.names, sch.types, Tables.Columns(tbl))
+                if col isa Arrow.DictEncode
+                    id = dictid[]
+                    dictid[] += 1
+                    codes = DataAPI.refarray(col.data)
+                    if codes !== col.data
+                        IT = Type(eltype(codes))
+                    else
+                        IT = Type(Arrow.encodingtype(length(unique(col))))
+                    end
+                    dictencodings[String(nm)] = (T, DictEncoding(id, IT, false))
+                end
+                push!(fields, Field(String(nm), T, dictencodings))
+            end
+        end
+        # build record batch
+        len = Tables.rowcount(tbl)
+        columns = FieldData[]
+        for (nm, T, col) in zip(sch.names, sch.types, Tables.Columns(tbl))
+            push!(columns, FieldData(String(nm), T, col, dictencodings))
+        end
+        push!(batches, RecordBatch(len, columns))
+        # build dictionaries
+        for (nm, (T, dictencoding)) in dictencodings
+            column = FieldData(nm, T, Tables.getcolumn(tbl, nm), nothing)
+            recordbatch = RecordBatch(len, [column])
+            push!(dictionaries, DictionaryBatch(dictencoding.id, recordbatch))
+        end
+    end
+    schema = Schema(fields, metadata)
+    return DataFile(schema, batches, dictionaries)
+end
+
+function Base.isequal(df::DataFile, tbl::Arrow.Table)
+    Tables.schema(df) == Tables.schema(tbl) || return false
+    i = 1
+    for (col1, col2) in zip(Tables.Columns(df), Tables.Columns(tbl))
+        if !isequal(col1, col2)
+            @show i
+            return false
+        end
+        i += 1
+    end
+    return true
+end
+
+end
\ No newline at end of file
diff --git a/julia/Arrow/test/arrowjson/datetime.json b/julia/Arrow/test/arrowjson/datetime.json
new file mode 100644
index 0000000..35eace4
--- /dev/null
+++ b/julia/Arrow/test/arrowjson/datetime.json
@@ -0,0 +1,911 @@
+{
+  "schema": {
+    "fields": [
+      {
+        "name": "f0",
+        "type": {
+          "name": "date",
+          "unit": "DAY"
+        },
+        "nullable": true,
+        "children": []
+      },
+      {
+        "name": "f1",
+        "type": {
+          "name": "date",
+          "unit": "MILLISECOND"
+        },
+        "nullable": true,
+        "children": []
+      },
+      {
+        "name": "f2",
+        "type": {
+          "name": "time",
+          "unit": "SECOND",
+          "bitWidth": 32
+        },
+        "nullable": true,
+        "children": []
+      },
+      {
+        "name": "f3",
+        "type": {
+          "name": "time",
+          "unit": "MILLISECOND",
+          "bitWidth": 32
+        },
+        "nullable": true,
+        "children": []
+      },
+      {
+        "name": "f4",
+        "type": {
+          "name": "time",
+          "unit": "MICROSECOND",
+          "bitWidth": 64
+        },
+        "nullable": true,
+        "children": []
+      },
+      {
+        "name": "f5",
+        "type": {
+          "name": "time",
+          "unit": "NANOSECOND",
+          "bitWidth": 64
+        },
+        "nullable": true,
+        "children": []
+      },
+      {
+        "name": "f6",
+        "type": {
+          "name": "timestamp",
+          "unit": "SECOND"
+        },
+        "nullable": true,
+        "children": []
+      },
+      {
+        "name": "f7",
+        "type": {
+          "name": "timestamp",
+          "unit": "MILLISECOND"
+        },
+        "nullable": true,
+        "children": []
+      },
+      {
+        "name": "f8",
+        "type": {
+          "name": "timestamp",
+          "unit": "MICROSECOND"
+        },
+        "nullable": true,
+        "children": []
+      },
+      {
+        "name": "f9",
+        "type": {
+          "name": "timestamp",
+          "unit": "NANOSECOND"
+        },
+        "nullable": true,
+        "children": []
+      },
+      {
+        "name": "f10",
+        "type": {
+          "name": "timestamp",
+          "unit": "MILLISECOND"
+        },
+        "nullable": true,
+        "children": []
+      },
+      {
+        "name": "f11",
+        "type": {
+          "name": "timestamp",
+          "unit": "SECOND",
+          "timezone": "UTC"
+        },
+        "nullable": true,
+        "children": []
+      },
+      {
+        "name": "f12",
+        "type": {
+          "name": "timestamp",
+          "unit": "MILLISECOND",
+          "timezone": "US/Eastern"
+        },
+        "nullable": true,
+        "children": []
+      },
+      {
+        "name": "f13",
+        "type": {
+          "name": "timestamp",
+          "unit": "MICROSECOND",
+          "timezone": "Europe/Paris"
+        },
+        "nullable": true,
+        "children": []
+      },
+      {
+        "name": "f14",
+        "type": {
+          "name": "timestamp",
+          "unit": "NANOSECOND",
+          "timezone": "US/Pacific"
+        },
+        "nullable": true,
+        "children": []
+      }
+    ]
+  },
+  "batches": [
+    {
+      "count": 7,
+      "columns": [
+        {
+          "name": "f0",
+          "count": 7,
+          "VALIDITY": [
+            0,
+            0,
+            0,
+            1,
+            0,
+            1,
+            0
+          ],
+          "DATA": [
+            -719162,
+            2932896,
+            1251583,
+            -354578,
+            1947695,
+            -669151,
+            26653
+          ]
+        },
+        {
+          "name": "f1",
+          "count": 7,
+          "VALIDITY": [
+            1,
+            1,
+            0,
+            0,
+            1,
+            1,
+            0
+          ],
+          "DATA": [
+            "-62135596800000",
+            "253402214400000",
+            "-2820221740189",
+            "71549882314362",
+            "125067746235030",
+            "-27038791348254",
+            "42137473450326"
+          ]
+        },
+        {
+          "name": "f2",
+          "count": 7,
+          "VALIDITY": [
+            1,
+            0,
+            1,
+            1,
+            0,
+            0,
+            1
+          ],
+          "DATA": [
+            0,
+            86400,
+            76127,
+            6700,
+            27103,
+            56151,
+            56654
+          ]
+        },
+        {
+          "name": "f3",
+          "count": 7,
+          "VALIDITY": [
+            1,
+            0,
+            0,
+            1,
+            0,
+            1,
+            0
+          ],
+          "DATA": [
+            0,
+            86400000,
+            17984286,
+            76370591,
+            60937117,
+            2240575,
+            8788989
+          ]
+        },
+        {
+          "name": "f4",
+          "count": 7,
+          "VALIDITY": [
+            0,
+            1,
+            0,
+            1,
+            1,
+            0,
+            1
+          ],
+          "DATA": [
+            "0",
+            "86400000000",
+            "74085029005",
+            "23058796418",
+            "5827157574",
+            "30753386088",
+            "41165364667"
+          ]
+        },
+        {
+          "name": "f5",
+          "count": 7,
+          "VALIDITY": [
+            0,
+            1,
+            0,
+            0,
+            1,
+            1,
+            1
+          ],
+          "DATA": [
+            "0",
+            "86400000000000",
+            "23907445718784",
+            "74345421086418",
+            "75233481254444",
+            "82172159793710",
+            "58497242525071"
+          ]
+        },
+        {
+          "name": "f6",
+          "count": 7,
+          "VALIDITY": [
+            1,
+            1,
+            0,
+            1,
+            1,
+            1,
+            1
+          ],
+          "DATA": [
+            "-62135596800",
+            "253402214400",
+            "-61131551586",
+            "109841560561",
+            "-3916465142",
+            "146694684650",
+            "138850275868"
+          ]
+        },
+        {
+          "name": "f7",
+          "count": 7,
+          "VALIDITY": [
+            1,
+            0,
+            0,
+            1,
+            1,
+            1,
+            0
+          ],
+          "DATA": [
+            "-62135596800000",
+            "253402214400000",
+            "73881152631437",
+            "69908713976427",
+            "252339755754438",
+            "-24746530024729",
+            "169302540975380"
+          ]
+        },
+        {
+          "name": "f8",
+          "count": 7,
+          "VALIDITY": [
+            1,
+            1,
+            1,
+            1,
+            0,
+            1,
+            0
+          ],
+          "DATA": [
+            "-62135596800000000",
+            "253402214400000000",
+            "201464679490582249",
+            "168126161155504013",
+            "-23403778250906066",
+            "205706554937392102",
+            "45776665091115087"
+          ]
+        },
+        {
+          "name": "f9",
+          "count": 7,
+          "VALIDITY": [
+            0,
+            1,
+            0,
+            1,
+            1,
+            1,
+            1
+          ],
+          "DATA": [
+            "-9223372036854775808",
+            "9223372036854775807",
+            "-6357255048670867880",
+            "-8661447973993819541",
+            "-8212788386909103318",
+            "-8530954041419345600",
+            "-4218486829304453721"
+          ]
+        },
+        {
+          "name": "f10",
+          "count": 7,
+          "VALIDITY": [
+            0,
+            1,
+            1,
+            1,
+            1,
+            1,
+            0
+          ],
+          "DATA": [
+            "-62135596800000",
+            "253402214400000",
+            "206983911256609",
+            "94973546379549",
+            "-18473207641060",
+            "36529119814530",
+            "143273969098011"
+          ]
+        },
+        {
+          "name": "f11",
+          "count": 7,
+          "VALIDITY": [
+            1,
+            0,
+            1,
+            0,
+            1,
+            0,
+            0
+          ],
+          "DATA": [
+            "-62135596800",
+            "253402214400",
+            "225567855249",
+            "18090198256",
+            "-18645631593",
+            "1683299996",
+            "240974238031"
+          ]
+        },
+        {
+          "name": "f12",
+          "count": 7,
+          "VALIDITY": [
+            1,
+            0,
+            1,
+            1,
+            1,
+            0,
+            1
+          ],
+          "DATA": [
+            "-62135596800000",
+            "253402214400000",
+            "-41888047432132",
+            "142167692985706",
+            "96450253340232",
+            "-28800292871111",
+            "31551906541089"
+          ]
+        },
+        {
+          "name": "f13",
+          "count": 7,
+          "VALIDITY": [
+            1,
+            1,
+            0,
+            1,
+            1,
+            1,
+            0
+          ],
+          "DATA": [
+            "-62135596800000000",
+            "253402214400000000",
+            "-55035741541368439",
+            "110555619232926841",
+            "13584197914180451",
+            "-40099398122556776",
+            "154575532939365500"
+          ]
+        },
+        {
+          "name": "f14",
+          "count": 7,
+          "VALIDITY": [
+            0,
+            1,
+            0,
+            1,
+            1,
+            1,
+            1
+          ],
+          "DATA": [
+            "-9223372036854775808",
+            "9223372036854775807",
+            "5175948389020765869",
+            "5557679156666679724",
+            "4250919303876106324",
+            "9160676477011889469",
+            "8585006913301874724"
+          ]
+        }
+      ]
+    },
+    {
+      "count": 10,
+      "columns": [
+        {
+          "name": "f0",
+          "count": 10,
+          "VALIDITY": [
+            1,
+            0,
+            0,
+            1,
+            0,
+            1,
+            0,
+            1,
+            1,
+            0
+          ],
+          "DATA": [
+            -719162,
+            2932896,
+            1879965,
+            -566367,
+            37728,
+            1761040,
+            279144,
+            1056794,
+            756303,
+            525725
+          ]
+        },
+        {
+          "name": "f1",
+          "count": 10,
+          "VALIDITY": [
+            1,
+            1,
+            1,
+            1,
+            1,
+            1,
+            1,
+            0,
+            0,
+            0
+          ],
+          "DATA": [
+            "-62135596800000",
+            "253402214400000",
+            "-38092110755085",
+            "-28445105640862",
+            "82150583163219",
+            "54419670636015",
+            "157522888788052",
+            "-41135178703404",
+            "-55692081078291",
+            "23161948344048"
+          ]
+        },
+        {
+          "name": "f2",
+          "count": 10,
+          "VALIDITY": [
+            1,
+            1,
+            0,
+            0,
+            1,
+            1,
+            0,
+            1,
+            0,
+            0
+          ],
+          "DATA": [
+            0,
+            86400,
+            68158,
+            43645,
+            82390,
+            63272,
+            8687,
+            73663,
+            41080,
+            16606
+          ]
+        },
+        {
+          "name": "f3",
+          "count": 10,
+          "VALIDITY": [
+            1,
+            0,
+            1,
+            0,
+            0,
+            0,
+            0,
+            1,
+            0,
+            0
+          ],
+          "DATA": [
+            0,
+            86400000,
+            4300160,
+            71414742,
+            77204018,
+            20632726,
+            31365614,
+            66601445,
+            59573489,
+            62138475
+          ]
+        },
+        {
+          "name": "f4",
+          "count": 10,
+          "VALIDITY": [
+            1,
+            1,
+            1,
+            1,
+            1,
+            1,
+            1,
+            1,
+            0,
+            0
+          ],
+          "DATA": [
+            "0",
+            "86400000000",
+            "28651881349",
+            "76962235570",
+            "72557141940",
+            "81551970477",
+            "20740172257",
+            "69927019990",
+            "76743031592",
+            "82821335874"
+          ]
+        },
+        {
+          "name": "f5",
+          "count": 10,
+          "VALIDITY": [
+            0,
+            1,
+            1,
+            1,
+            0,
+            1,
+            1,
+            0,
+            0,
+            0
+          ],
+          "DATA": [
+            "0",
+            "86400000000000",
+            "69757112637445",
+            "57552711513942",
+            "29426946816946",
+            "34878855922929",
+            "33344575898987",
+            "80887006335433",
+            "34037765279999",
+            "51577535310194"
+          ]
+        },
+        {
+          "name": "f6",
+          "count": 10,
+          "VALIDITY": [
+            0,
+            1,
+            1,
+            1,
+            0,
+            1,
+            1,
+            1,
+            1,
+            1
+          ],
+          "DATA": [
+            "-62135596800",
+            "253402214400",
+            "17362133914",
+            "-45891238920",
+            "184875963653",
+            "189805054827",
+            "-58368591641",
+            "11979945774",
+            "-42159999942",
+            "-40114167869"
+          ]
+        },
+        {
+          "name": "f7",
+          "count": 10,
+          "VALIDITY": [
+            0,
+            1,
+            1,
+            0,
+            1,
+            0,
+            1,
+            1,
+            1,
+            0
+          ],
+          "DATA": [
+            "-62135596800000",
+            "253402214400000",
+            "228991365143433",
+            "169961535994833",
+            "200469360821110",
+            "69234108321383",
+            "198319780924526",
+            "191497586634193",
+            "175427870270356",
+            "57342673854963"
+          ]
+        },
+        {
+          "name": "f8",
+          "count": 10,
+          "VALIDITY": [
+            0,
+            1,
+            1,
+            1,
+            1,
+            0,
+            0,
+            1,
+            1,
+            1
+          ],
+          "DATA": [
+            "-62135596800000000",
+            "253402214400000000",
+            "56980200145982394",
+            "181929648203745781",
+            "139102923751831867",
+            "208997257530770666",
+            "16136961511557279",
+            "41555612864958844",
+            "13419848118557598",
+            "87383692083185618"
+          ]
+        },
+        {
+          "name": "f9",
+          "count": 10,
+          "VALIDITY": [
+            1,
+            0,
+            1,
+            1,
+            0,
+            1,
+            0,
+            1,
+            1,
+            1
+          ],
+          "DATA": [
+            "-9223372036854775808",
+            "9223372036854775807",
+            "-7031715621840828119",
+            "6059066797068981521",
+            "-1646261793033501417",
+            "-4220806875454311426",
+            "-5197114782094970725",
+            "-8786449967831538943",
+            "2974021310284646715",
+            "-8065049992539820014"
+          ]
+        },
+        {
+          "name": "f10",
+          "count": 10,
+          "VALIDITY": [
+            0,
+            0,
+            1,
+            0,
+            1,
+            0,
+            1,
+            0,
+            0,
+            1
+          ],
+          "DATA": [
+            "-62135596800000",
+            "253402214400000",
+            "178310981489839",
+            "-57632494932898",
+            "-49036210908585",
+            "52895117552494",
+            "171282515196488",
+            "132014017559614",
+            "37589110284897",
+            "-58565057255450"
+          ]
+        },
+        {
+          "name": "f11",
+          "count": 10,
+          "VALIDITY": [
+            1,
+            0,
+            0,
+            1,
+            1,
+            1,
+            1,
+            1,
+            0,
+            1
+          ],
+          "DATA": [
+            "-62135596800",
+            "253402214400",
+            "-25861647763",
+            "225566577159",
+            "239974631847",
+            "9594019763",
+            "195861877033",
+            "-19961060193",
+            "75621579368",
+            "-37516489502"
+          ]
+        },
+        {
+          "name": "f12",
+          "count": 10,
+          "VALIDITY": [
+            1,
+            1,
+            0,
+            1,
+            1,
+            0,
+            1,
+            1,
+            1,
+            1
+          ],
+          "DATA": [
+            "-62135596800000",
+            "253402214400000",
+            "197770622812426",
+            "195993483135189",
+            "187243903796151",
+            "193213107363200",
+            "83570298570259",
+            "252571502045214",
+            "129428288356579",
+            "-6553516468568"
+          ]
+        },
+        {
+          "name": "f13",
+          "count": 10,
+          "VALIDITY": [
+            1,
+            1,
+            0,
+            1,
+            0,
+            1,
+            1,
+            1,
+            0,
+            1
+          ],
+          "DATA": [
+            "-62135596800000000",
+            "253402214400000000",
+            "227405247332731417",
+            "-23876629555725661",
+            "172967389442803645",
+            "214366955371313241",
+            "86933375268516953",
+            "162567381239071692",
+            "40270626452354761",
+            "-9766478375147980"
+          ]
+        },
+        {
+          "name": "f14",
+          "count": 10,
+          "VALIDITY": [
+            1,
+            0,
+            0,
+            1,
+            1,
+            0,
+            0,
+            1,
+            1,
+            1
+          ],
+          "DATA": [
+            "-9223372036854775808",
+            "9223372036854775807",
+            "-3367778002617009344",
+            "-921482854487607004",
+            "1280046021732622411",
+            "-2084512789553264359",
+            "6774927372092824293",
+            "2301253808511314195",
+            "-447823511949250637",
+            "-9137589927882857097"
+          ]
+        }
+      ]
+    }
+  ]
+}
\ No newline at end of file
diff --git a/julia/Arrow/test/arrowjson/decimal.json b/julia/Arrow/test/arrowjson/decimal.json
new file mode 100644
index 0000000..6c27800
--- /dev/null
+++ b/julia/Arrow/test/arrowjson/decimal.json
@@ -0,0 +1,32948 @@
+{
+  "schema": {
+    "fields": [
+      {
+        "name": "f0",
+        "type": {
+          "name": "decimal",
+          "precision": 3,
+          "scale": 2
+        },
+        "nullable": true,
+        "children": []
+      },
+      {
+        "name": "f1",
+        "type": {
+          "name": "decimal",
+          "precision": 4,
+          "scale": 2
+        },
+        "nullable": true,
+        "children": []
+      },
+      {
+        "name": "f2",
+        "type": {
+          "name": "decimal",
+          "precision": 5,
+          "scale": 2
+        },
+        "nullable": true,
+        "children": []
+      },
+      {
+        "name": "f3",
+        "type": {
+          "name": "decimal",
+          "precision": 6,
+          "scale": 2
+        },
+        "nullable": true,
+        "children": []
+      },
+      {
+        "name": "f4",
+        "type": {
+          "name": "decimal",
+          "precision": 7,
+          "scale": 2
+        },
+        "nullable": true,
+        "children": []
+      },
+      {
+        "name": "f5",
+        "type": {
+          "name": "decimal",
+          "precision": 8,
+          "scale": 2
+        },
+        "nullable": true,
+        "children": []
+      },
+      {
+        "name": "f6",
+        "type": {
+          "name": "decimal",
+          "precision": 9,
+          "scale": 2
+        },
+        "nullable": true,
+        "children": []
+      },
+      {
+        "name": "f7",
+        "type": {
+          "name": "decimal",
+          "precision": 10,
+          "scale": 2
+        },
+        "nullable": true,
+        "children": []
+      },
+      {
+        "name": "f8",
+        "type": {
+          "name": "decimal",
+          "precision": 11,
+          "scale": 2
+        },
+        "nullable": true,
+        "children": []
+      },
+      {
+        "name": "f9",
+        "type": {
+          "name": "decimal",
+          "precision": 12,
+          "scale": 2
+        },
+        "nullable": true,
+        "children": []
+      },
+      {
+        "name": "f10",
+        "type": {
+          "name": "decimal",
+          "precision": 13,
+          "scale": 2
+        },
+        "nullable": true,
+        "children": []
+      },
+      {
+        "name": "f11",
+        "type": {
+          "name": "decimal",
+          "precision": 14,
+          "scale": 2
+        },
+        "nullable": true,
+        "children": []
+      },
+      {
+        "name": "f12",
+        "type": {
+          "name": "decimal",
+          "precision": 15,
+          "scale": 2
+        },
+        "nullable": true,
+        "children": []
+      },
+      {
+        "name": "f13",
+        "type": {
+          "name": "decimal",
+          "precision": 16,
+          "scale": 2
+        },
+        "nullable": true,
+        "children": []
+      },
+      {
+        "name": "f14",
+        "type": {
+          "name": "decimal",
+          "precision": 17,
+          "scale": 2
+        },
+        "nullable": true,
+        "children": []
+      },
+      {
+        "name": "f15",
+        "type": {
+          "name": "decimal",
+          "precision": 18,
+          "scale": 2
+        },
+        "nullable": true,
+        "children": []
+      },
+      {
+        "name": "f16",
+        "type": {
+          "name": "decimal",
+          "precision": 19,
+          "scale": 2
+        },
+        "nullable": true,
+        "children": []
+      },
+      {
+        "name": "f17",
+        "type": {
+          "name": "decimal",
+          "precision": 20,
+          "scale": 2
+        },
+        "nullable": true,
+        "children": []
+      },
+      {
+        "name": "f18",
+        "type": {
+          "name": "decimal",
+          "precision": 21,
+          "scale": 2
+        },
+        "nullable": true,
+        "children": []
+      },
+      {
+        "name": "f19",
+        "type": {
+          "name": "decimal",
+          "precision": 22,
+          "scale": 2
+        },
+        "nullable": true,
+        "children": []
+      },
+      {
+        "name": "f20",
+        "type": {
+          "name": "decimal",
+          "precision": 23,
+          "scale": 2
+        },
+        "nullable": true,
+        "children": []
+      },
+      {
+        "name": "f21",
+        "type": {
+          "name": "decimal",
+          "precision": 24,
+          "scale": 2
+        },
+        "nullable": true,
+        "children": []
+      },
+      {
+        "name": "f22",
+        "type": {
+          "name": "decimal",
+          "precision": 25,
+          "scale": 2
+        },
+        "nullable": true,
+        "children": []
+      },
+      {
+        "name": "f23",
+        "type": {
+          "name": "decimal",
+          "precision": 26,
+          "scale": 2
+        },
+        "nullable": true,
+        "children": []
+      },
+      {
+        "name": "f24",
+        "type": {
+          "name": "decimal",
+          "precision": 27,
+          "scale": 2
+        },
+        "nullable": true,
+        "children": []
+      },
+      {
+        "name": "f25",
+        "type": {
+          "name": "decimal",
+          "precision": 28,
+          "scale": 2
+        },
+        "nullable": true,
+        "children": []
+      },
+      {
+        "name": "f26",
+        "type": {
+          "name": "decimal",
+          "precision": 29,
+          "scale": 2
+        },
+        "nullable": true,
+        "children": []
+      },
+      {
+        "name": "f27",
+        "type": {
+          "name": "decimal",
+          "precision": 30,
+          "scale": 2
+        },
+        "nullable": true,
+        "children": []
+      },
+      {
+        "name": "f28",
+        "type": {
+          "name": "decimal",
+          "precision": 31,
+          "scale": 2
+        },
+        "nullable": true,
+        "children": []
+      },
+      {
+        "name": "f29",
+        "type": {
+          "name": "decimal",
+          "precision": 32,
+          "scale": 2
+        },
+        "nullable": true,
+        "children": []
+      },
+      {
+        "name": "f30",
+        "type": {
+          "name": "decimal",
+          "precision": 33,
+          "scale": 2
+        },
+        "nullable": true,
+        "children": []
+      },
+      {
+        "name": "f31",
+        "type": {
+          "name": "decimal",
+          "precision": 34,
+          "scale": 2
+        },
+        "nullable": true,
+        "children": []
+      },
+      {
+        "name": "f32",
+        "type": {
+          "name": "decimal",
+          "precision": 35,
+          "scale": 2
+        },
+        "nullable": true,
+        "children": []
+      },
+      {
+        "name": "f33",
+        "type": {
+          "name": "decimal",
+          "precision": 36,
+          "scale": 2
+        },
+        "nullable": true,
+        "children": []
+      },
+      {
+        "name": "f34",
+        "type": {
+          "name": "decimal",
+          "precision": 37,
+          "scale": 2
+        },
+        "nullable": true,
+        "children": []
+      },
+      {
+        "name": "f35",
+        "type": {
+          "name": "decimal",
+          "precision": 38,
+          "scale": 2
+        },
+        "nullable": true,
+        "children": []
+      }
+    ]
+  },
+  "batches": [
+    {
+      "count": 7,
+      "columns": [
+        {
+          "name": "f0",
+          "count": 7,
+          "VALIDITY": [
+            1,
+            1,
+            1,
+            1,
+            0,
+            1,
+            1
+          ],
+          "DATA": [
+            "-14265",
+            "-3301",
+            "-20359",
+            "5259",
+            "-10833",
+            "-19794",
+            "22904"
+          ]
+        },
+        {
+          "name": "f1",
+          "count": 7,
+          "VALIDITY": [
+            1,
+            0,
+            1,
+            1,
+            0,
+            1,
+            1
+          ],
+          "DATA": [
+            "-7020",
+            "7396",
+            "9702",
+            "-29862",
+            "25915",
+            "17934",
+            "23441"
+          ]
+        },
+        {
+          "name": "f2",
+          "count": 7,
+          "VALIDITY": [
+            1,
+            1,
+            0,
+            0,
+            0,
+            0,
+            0
+          ],
+          "DATA": [
+            "-6799031",
+            "-6350361",
+            "-7919057",
+            "-2688856",
+            "-2460218",
+            "-5498780",
+            "5580928"
+          ]
+        },
+        {
+          "name": "f3",
+          "count": 7,
+          "VALIDITY": [
+            0,
+            1,
+            1,
+            0,
+            1,
+            1,
+            1
+          ],
+          "DATA": [
+            "-3533649",
+            "643928",
+            "2739361",
+            "-2006582",
+            "7501109",
+            "-2029241",
+            "-6554749"
+          ]
+        },
+        {
+          "name": "f4",
+          "count": 7,
+          "VALIDITY": [
+            1,
+            0,
+            1,
+            1,
+            1,
+            1,
+            0
+          ],
+          "DATA": [
+            "-1383097445",
+            "-317364446",
+            "39137303",
+            "-203304373",
+            "284864794",
+            "-1902924832",
+            "694509255"
+          ]
+        },
+        {
+          "name": "f5",
+          "count": 7,
+          "VALIDITY": [
+            1,
+            1,
+            1,
+            1,
+            0,
+            1,
+            0
+          ],
+          "DATA": [
+            "1445584597",
+            "-1479256303",
+            "1986678586",
+            "653640925",
+            "-1377647126",
+            "163472005",
+            "-985666433"
+          ]
+        },
+        {
+          "name": "f6",
+          "count": 7,
+          "VALIDITY": [
+            0,
+            1,
+            1,
+            0,
+            1,
+            0,
+            0
+          ],
+          "DATA": [
+            "-1032313347",
+            "1968833128",
+            "2108909581",
+            "1727353475",
+            "912414766",
+            "-635823470",
+            "1712241290"
+          ]
+        },
+        {
+          "name": "f7",
+          "count": 7,
+          "VALIDITY": [
+            1,
+            1,
+            1,
+            0,
+            0,
+            0,
+            1
+          ],
+          "DATA": [
+            "154945363887",
+            "416184610386",
+            "339640718067",
+            "517615365091",
+            "-225329280656",
+            "170183803130",
+            "-151819857736"
+          ]
+        },
+        {
+          "name": "f8",
+          "count": 7,
+          "VALIDITY": [
+            0,
+            0,
+            1,
+            0,
+            1,
+            1,
+            0
+          ],
+          "DATA": [
+            "415862710871",
+            "-497367616337",
+            "-513589024815",
+            "365532163027",
+            "162771025458",
+            "-217545761908",
+            "-177764161272"
+          ]
+        },
+        {
+          "name": "f9",
+          "count": 7,
+          "VALIDITY": [
+            0,
+            1,
+            1,
+            1,
+            1,
+            1,
+            1
+          ],
+          "DATA": [
+            "139749710369981",
+            "-83794245890178",
+            "-91856283653391",
+            "136480936919831",
+            "139751556918685",
+            "-128711073562986",
+            "57528145123438"
+          ]
+        },
+        {
+          "name": "f10",
+          "count": 7,
+          "VALIDITY": [
+            1,
+            1,
+            0,
+            1,
+            1,
+            0,
+            1
+          ],
+          "DATA": [
+            "52917228652967",
+            "-94943960868145",
+            "-95787469176401",
+            "138513251818631",
+            "72391840904205",
+            "15456369093039",
+            "43627762817987"
+          ]
+        },
+        {
+          "name": "f11",
+          "count": 7,
+          "VALIDITY": [
+            1,
+            1,
+            1,
+            1,
+            1,
+            0,
+            1
+          ],
+          "DATA": [
+            "-55939733680307",
+            "109774177882810",
+            "-18751094013174",
+            "66217182114674",
+            "54552088012955",
+            "10184533351464",
+            "-42623419179005"
+          ]
+        },
+        {
+          "name": "f12",
+          "count": 7,
+          "VALIDITY": [
+            0,
+            0,
+            0,
+            0,
+            1,
+            1,
+            0
+          ],
+          "DATA": [
+            "5323426016158598",
+            "-14889118959500841",
+            "17725275930019497",
+            "8976278451851381",
+            "-13047873474817232",
+            "-35698038632707430",
+            "35553131382011652"
+          ]
+        },
+        {
+          "name": "f13",
+          "count": 7,
+          "VALIDITY": [
+            0,
+            0,
+            1,
+            0,
+            0,
+            1,
+            1
+          ],
+          "DATA": [
+            "22248347423524568",
+            "21382628128477388",
+            "10729166497278728",
+            "33552782977485637",
+            "-27049764851522362",
+            "-22662588671534830",
+            "34450594992450092"
+          ]
+        },
+        {
+          "name": "f14",
+          "count": 7,
+          "VALIDITY": [
+            0,
+            0,
+            1,
+            0,
+            1,
+            0,
+            0
+          ],
+          "DATA": [
+            "1793092340943680288",
+            "6687492261570159758",
+            "-8215335212004105263",
+            "4478178378408252470",
+            "163214921176454093",
+            "8663994206540321487",
+            "-7336047061160684053"
+          ]
+        },
+        {
+          "name": "f15",
+          "count": 7,
+          "VALIDITY": [
+            1,
+            1,
+            1,
+            1,
+            0,
+            1,
+            0
+          ],
+          "DATA": [
+            "-8553613303342920727",
+            "2303004565228604365",
+            "4387581718276767427",
+            "-9190240538897215152",
+            "-2359611618219705826",
+            "-4394705260138635628",
+            "-5098494694180487526"
+          ]
+        },
+        {
+          "name": "f16",
+          "count": 7,
+          "VALIDITY": [
+            1,
+            1,
+            0,
+            0,
+            1,
+            1,
+            1
+          ],
+          "DATA": [
+            "-2066857943452927830158",
+            "903014216615599800157",
+            "-550458216531247342288",
+            "2017634601926315282348",
+            "848752616371714143556",
+            "1142061465959813542452",
+            "1385502695475659971908"
+          ]
+        },
+        {
+          "name": "f17",
+          "count": 7,
+          "VALIDITY": [
+            1,
+            0,
+            0,
+            1,
+            0,
+            0,
+            1
+          ],
+          "DATA": [
+            "-2114118262682166206082",
+            "1902359162690886678168",
+            "416686278682682920537",
+            "1670406372017974831657",
+            "1672178198289799650411",
+            "1290828393749566925129",
+            "477509832520793484657"
+          ]
+        },
+        {
+          "name": "f18",
+          "count": 7,
+          "VALIDITY": [
+            1,
+            0,
+            1,
+            0,
+            1,
+            1,
+            1
+          ],
+          "DATA": [
+            "-761286776745227142069",
+            "-519173571795485925518",
+            "-1152371818412313184820",
+            "-1693854308571834153795",
+            "-2338147549154180008842",
+            "-722980793690531284755",
+            "-308453791763555488903"
+          ]
+        },
+        {
+          "name": "f19",
+          "count": 7,
+          "VALIDITY": [
+            1,
+            0,
+            1,
+            1,
+            1,
+            0,
+            1
+          ],
+          "DATA": [
+            "-363139155978036972417607",
+            "-67592154020748584606597",
+            "229641197562527780640996",
+            "-346708211516644122649482",
+            "-1250859224618770910347",
+            "-324094365896255288686912",
+            "587754147918610672847722"
+          ]
+        },
+        {
+          "name": "f20",
+          "count": 7,
+          "VALIDITY": [
+            0,
+            0,
+            0,
+            0,
+            1,
+            1,
+            1
+          ],
+          "DATA": [
+            "-260044582391618089260319",
+            "-571174996847059618302630",
+            "-109712997473546598340520",
+            "-504887878361994310822232",
+            "-79933142040493800702044",
+            "391593215117382882517477",
+            "253411954510878438009626"
+          ]
+        },
+        {
+          "name": "f21",
+          "count": 7,
+          "VALIDITY": [
+            0,
+            1,
+            0,
+            1,
+            0,
+            1,
+            0
+          ],
+          "DATA": [
+            "78355683904145007775288840",
+            "-119384708322041630321535990",
+            "-100920737167981116982294279",
+            "83826802691186830123887155",
+            "-142861875795075155563507787",
+            "-145798388286082279657253594",
+            "-25329913740178438718347749"
+          ]
+        },
+        {
+          "name": "f22",
+          "count": 7,
+          "VALIDITY": [
+            1,
+            1,
+            1,
+            0,
+            0,
+            0,
+            1
+          ],
+          "DATA": [
+            "2728894406873273319876599",
+            "-26824077854062930008319059",
+            "59282342970725803527905051",
+            "-102858143095488614827267122",
+            "-86746033992194633744162126",
+            "151758737255322612002715481",
+            "-7882766239372883938103148"
+          ]
+        },
+        {
+          "name": "f23",
+          "count": 7,
+          "VALIDITY": [
+            0,
+            1,
+            0,
+            1,
+            0,
+            1,
+            1
+          ],
+          "DATA": [
+            "-79786812518583385435068134",
+            "-60631765880118763666388680",
+            "-6864681441231111204611249",
+            "13050926774682302829016376",
+            "-70829524080848069150697712",
+            "-82269446396753351313982437",
+            "-64035656687419899366647308"
+          ]
+        },
+        {
+          "name": "f24",
+          "count": 7,
+          "VALIDITY": [
+            1,
+            0,
+            1,
+            0,
+            1,
+            1,
+            1
+          ],
+          "DATA": [
+            "3955725931924121460907631531",
+            "-21274213085809772733586643935",
+            "-12093030231048856037056919233",
+            "1591415318816864321772119800",
+            "15686477033795236409156083537",
+            "35545421326820517900037881163",
+            "22425766368694890650567525141"
+          ]
+        },
+        {
+          "name": "f25",
+          "count": 7,
+          "VALIDITY": [
+            0,
+            1,
+            1,
+            1,
+            0,
+            0,
+            1
+          ],
+          "DATA": [
+            "29018707613584242707294878326",
+            "17791141760763489912825555133",
+            "16531138885547875931113958621",
+            "-30513053118126324165999828125",
+            "15960846333545963474586001659",
+            "36917332355702074649716398096",
+            "-23564003365919469704120973292"
+          ]
+        },
+        {
+          "name": "f26",
+          "count": 7,
+          "VALIDITY": [
+            1,
+            0,
+            0,
+            1,
+            0,
+            1,
+            1
+          ],
+          "DATA": [
+            "9213099815853647106837374141135",
+            "-2927321977169069503340222047887",
+            "6334956127042213437236348937051",
+            "421008034487997672364705982278",
+            "-8216535847017377542720322569325",
+            "9010595708854142678424220967811",
+            "4188490944857456828411574213081"
+          ]
+        },
+        {
+          "name": "f27",
+          "count": 7,
+          "VALIDITY": [
+            1,
+            1,
+            1,
+            1,
+            1,
+            0,
+            0
+          ],
+          "DATA": [
+            "-8400462145843923761912990483225",
+            "-9532520685799568447195818877630",
+            "6941627626857671447061360399506",
+            "-1279966284289260405732779325351",
+            "1498483597016609631513149683949",
+            "-1529118713555007787773854671666",
+            "8107117611898163619507416633696"
+          ]
+        },
+        {
+          "name": "f28",
+          "count": 7,
+          "VALIDITY": [
+            1,
+            0,
+            1,
+            1,
+            0,
+            1,
+            0
+          ],
+          "DATA": [
+            "-2915756808824546582752189119657",
+            "689849096940399949037495607644",
+            "6949792968358403147068782203692",
+            "-9209851065708847333103701669232",
+            "-9846735769797668003358129310662",
+            "-828325389508874726576445634733",
+            "-548287410953591263203641276502"
+          ]
+        },
+        {
+          "name": "f29",
+          "count": 7,
+          "VALIDITY": [
+            0,
+            1,
+            1,
+            1,
+            1,
+            1,
+            1
+          ],
+          "DATA": [
+            "2329344668638312257280241058269271",
+            "-752344765844397811032529707212356",
+            "147451832276294483763966453842451",
+            "2043297198679260475716881353145488",
+            "-1700852196536476328772217191241342",
+            "395231723125832272118398102832808",
+            "1021064751191444192162606435133102"
+          ]
+        },
+        {
+          "name": "f30",
+          "count": 7,
+          "VALIDITY": [
+            1,
+            1,
+            0,
+            0,
+            1,
+            1,
+            1
+          ],
+          "DATA": [
+            "1059349538643979619795009909676873",
+            "1077517333972932491341963859241254",
+            "-1269329697335528941024149247301687",
+            "1492334714292199711015081732157237",
+            "1164940978846205931486582828955479",
+            "1226701026828005364221866490976254",
+            "-2246642220670050837117291416085961"
+          ]
+        },
+        {
+          "name": "f31",
+          "count": 7,
+          "VALIDITY": [
+            0,
+            1,
+            0,
+            0,
+            0,
+            0,
+            0
+          ],
+          "DATA": [
+            "-453427466718461089284644212936857043",
+            "401968647744354893846373970134628607",
+            "296564973980089833086319484394180932",
+            "300255396607176659100995955356045655",
+            "-128210286701929565620215086982795175",
+            "-76575845460847329054975157675278272",
+            "150610860039362064995272346289079343"
+          ]
+        },
+        {
+          "name": "f32",
+          "count": 7,
+          "VALIDITY": [
+            1,
+            1,
+            1,
+            1,
+            1,
+            1,
+            1
+          ],
+          "DATA": [
+            "421046835537097379208584523685561537",
+            "-12336249922904854000276177315936166",
+            "-643245709818789885569562430725515554",
+            "28125849177083902136768190312091645",
+            "-144070901322770869842559076535428168",
+            "404632889001498427711768281303970316",
+            "484413224979251839912542328701460679"
+          ]
+        },
+        {
+          "name": "f33",
+          "count": 7,
+          "VALIDITY": [
+            1,
+            0,
+            1,
+            1,
+            1,
+            1,
+            1
+          ],
+          "DATA": [
+            "-74076465584946023556734876157740408561",
+            "48775465105440153520372738047296715437",
+            "-27256511570535798216063178902711504522",
+            "17556306662405983159878112274642973563",
+            "-100278707484244689483962841662232621044",
+            "61342616787415403733699116416597761322",
+            "65469616326882669620827062523697818819"
+          ]
+        },
+        {
+          "name": "f34",
+          "count": 7,
+          "VALIDITY": [
+            0,
+            1,
+            1,
+            0,
+            0,
+            0,
+            0
+          ],
+          "DATA": [
+            "160988666415816646705442929165000844171",
+            "-150079614246235114587531344502672720143",
+            "130458754430497218349443163826152965786",
+            "162593370774960128510764087469483971628",
+            "100335479568560073327353331226427021396",
+            "154469868317889131666703500879972187378",
+            "109851602321776291877341624608658829262"
+          ]
+        },
+        {
+          "name": "f35",
+          "count": 7,
+          "VALIDITY": [
+            1,
+            1,
+            0,
+            0,
+            1,
+            1,
+            1
+          ],
+          "DATA": [
+            "90507703396950205824782315866890653146",
+            "33101701517846982239639071710846694736",
+            "-162364169137146690363873495754205930932",
+            "6552361136224904568043204720462599330",
+            "55934073603932057089039430724186991179",
+            "-40518421289171987097923903446970623529",
+            "21964873705187790865866750301783750442"
+          ]
+        }
+      ]
+    },
+    {
+      "count": 10,
+      "columns": [
+        {
+          "name": "f0",
+          "count": 10,
+          "VALIDITY": [
+            1,
+            0,
+            0,
+            1,
+            1,
+            1,
+            0,
+            1,
+            0,
+            0
+          ],
+          "DATA": [
+            "-4326",
+            "-4742",
+            "24127",
+            "-28455",
+            "-25093",
+            "18893",
+            "6010",
+            "18847",
+            "-29502",
+            "32119"
+          ]
+        },
+        {
+          "name": "f1",
+          "count": 10,
+          "VALIDITY": [
+            1,
+            0,
+            1,
+            1,
+            0,
+            0,
+            1,
+            1,
+            1,
+            1
+          ],
+          "DATA": [
+            "-4350",
+            "6406",
+            "-22249",
+            "-3026",
+            "-27892",
+            "12582",
+            "-26038",
+            "-26810",
+            "-10412",
+            "12059"
+          ]
+        },
+        {
+          "name": "f2",
+          "count": 10,
+          "VALIDITY": [
+            0,
+            0,
+            1,
+            1,
+            0,
+            1,
+            0,
+            1,
+            0,
+            1
+          ],
+          "DATA": [
+            "-5437050",
+            "-2975935",
+            "-2708551",
+            "5905122",
+            "7598831",
+            "1837938",
+            "3161296",
+            "4144482",
+            "7971237",
+            "3812932"
+          ]
+        },
+        {
+          "name": "f3",
+          "count": 10,
+          "VALIDITY": [
+            0,
+            0,
+            1,
+            0,
+            1,
+            0,
+            0,
+            1,
+            1,
+            0
+          ],
+          "DATA": [
+            "8307058",
+            "5437059",
+            "7848415",
+            "210888",
+            "-7438002",
+            "1251661",
+            "1558298",
+            "-8295786",
+            "-1337540",
+            "4329127"
+          ]
+        },
+        {
+          "name": "f4",
+          "count": 10,
+          "VALIDITY": [
+            1,
+            0,
+            1,
+            1,
+            1,
+            1,
+            1,
+            1,
+            0,
+            1
+          ],
+          "DATA": [
+            "2143281149",
+            "1036205716",
+            "-483960365",
+            "464783350",
+            "44702295",
+            "1500496151",
+            "2007499481",
+            "-1980167635",
+            "-491561248",
+            "137927958"
+          ]
+        },
+        {
+          "name": "f5",
+          "count": 10,
+          "VALIDITY": [
+            0,
+            1,
+            0,
+            1,
+            1,
+            0,
+            0,
+            1,
+            0,
+            1
+          ],
+          "DATA": [
+            "385115557",
+            "-468314229",
+            "-1806182189",
+            "826968059",
+            "1973820915",
+            "-1678321968",
+            "282860534",
+            "-429116070",
+            "-485370474",
+            "-1175964348"
+          ]
+        },
+        {
+          "name": "f6",
+          "count": 10,
+          "VALIDITY": [
+            0,
+            0,
+            1,
+            0,
+            1,
+            1,
+            0,
+            1,
+            1,
+            1
+          ],
+          "DATA": [
+            "-1660401943",
+            "858637970",
+            "1954585251",
+            "896247082",
+            "-1324508546",
+            "1856733620",
+            "-508486488",
+            "-2144890404",
+            "-595464217",
+            "-541883970"
+          ]
+        },
+        {
+          "name": "f7",
+          "count": 10,
+          "VALIDITY": [
+            1,
+            1,
+            1,
+            0,
+            1,
+            1,
+            0,
+            0,
+            0,
+            1
+          ],
+          "DATA": [
+            "-475292084638",
+            "-151527930749",
+            "-136244496864",
+            "475724641544",
+            "490808185901",
+            "-380524076056",
+            "347742169602",
+            "-208359763265",
+            "500407573029",
+            "-390649746248"
+          ]
+        },
+        {
+          "name": "f8",
+          "count": 10,
+          "VALIDITY": [
+            0,
+            1,
+            0,
+            0,
+            0,
+            1,
+            0,
+            0,
+            1,
+            0
+          ],
+          "DATA": [
+            "-463878565883",
+            "-440350084389",
+            "-489309228695",
+            "403236358497",
+            "22420180364",
+            "461550541832",
+            "166802945594",
+            "-75395045157",
+            "349097107753",
+            "-31923850582"
+          ]
+        },
+        {
+          "name": "f9",
+          "count": 10,
+          "VALIDITY": [
+            1,
+            1,
+            0,
+            1,
+            0,
+            1,
+            1,
+            1,
+            1,
+            1
+          ],
+          "DATA": [
+            "13351341741346",
+            "132672835377977",
+            "-104121852651737",
+            "-116578873884859",
+            "-93368786381550",
+            "-130866514927019",
+            "-130245922532357",
+            "119422720388976",
+            "39514069962031",
+            "129449354245745"
+          ]
+        },
+        {
+          "name": "f10",
+          "count": 10,
+          "VALIDITY": [
+            0,
+            0,
+            0,
+            1,
+            0,
+            0,
+            1,
+            0,
+            1,
+            1
+          ],
+          "DATA": [
+            "117562734639864",
+            "-69335041870511",
+            "88065868490266",
+            "-88825080775852",
+            "-33000609471354",
+            "45253821354104",
+            "-19572766862990",
+            "-20998291433421",
+            "-138710717300131",
+            "105346771642837"
+          ]
+        },
+        {
+          "name": "f11",
+          "count": 10,
+          "VALIDITY": [
+            0,
+            0,
+            1,
+            1,
+            0,
+            1,
+            1,
+            1,
+            0,
+            0
+          ],
+          "DATA": [
+            "23613726765605",
+            "-122540956874454",
+            "43298244246934",
+            "38914627030131",
+            "-96822287655033",
+            "-117561922155895",
+            "-116901550232631",
+            "-33238646153535",
+            "87255230245952",
+            "-103811103056354"
+          ]
+        },
+        {
+          "name": "f12",
+          "count": 10,
+          "VALIDITY": [
+            0,
+            1,
+            0,
+            1,
+            1,
+            1,
+            0,
+            1,
+            0,
+            1
+          ],
+          "DATA": [
+            "14326292611678389",
+            "11679216137082255",
+            "30602968082225040",
+            "-35083463698552038",
+            "13793460528237121",
+            "29475754221579432",
+            "11956818381525770",
+            "25264515687485849",
+            "-9442305734363215",
+            "32709604673194504"
+          ]
+        },
+        {
+          "name": "f13",
+          "count": 10,
+          "VALIDITY": [
+            0,
+            0,
+            1,
+            1,
+            0,
+            1,
+            1,
+            1,
+            0,
+            0
+          ],
+          "DATA": [
+            "10752127983223695",
+            "-17671172332493417",
+            "23204581064269137",
+            "25912317958066911",
+            "-6684281122027701",
+            "-5393708227406178",
+            "-705876736520327",
+            "-8271615012961841",
+            "-2359308340581953",
+            "-10178212042804721"
+          ]
+        },
+        {
+          "name": "f14",
+          "count": 10,
+          "VALIDITY": [
+            0,
+            1,
+            0,
+            1,
+            0,
+            0,
+            0,
+            1,
+            0,
+            0
+          ],
+          "DATA": [
+            "-785858494820999025",
+            "-1596606269767712061",
+            "-1418974761760001783",
+            "5406463592868106826",
+            "8383949914321587435",
+            "-8953833508549969767",
+            "7221522693757617783",
+            "7517732845880583602",
+            "-919064621805709634",
+            "-5495596733594283265"
+          ]
+        },
+        {
+          "name": "f15",
+          "count": 10,
+          "VALIDITY": [
+            1,
+            0,
+            1,
+            1,
+            0,
+            1,
+            0,
+            1,
+            1,
+            1
+          ],
+          "DATA": [
+            "-8483275882311463690",
+            "8973190785377545539",
+            "5542339023127975216",
+            "-1516710247958149161",
+            "-6505879801430717005",
+            "2140391223240971073",
+            "-582269641777058598",
+            "-5184242226957355683",
+            "5206992463328246120",
+            "-6079741749013867959"
+          ]
+        },
+        {
+          "name": "f16",
+          "count": 10,
+          "VALIDITY": [
+            1,
+            0,
+            1,
+            1,
+            0,
+            1,
+            1,
+            0,
+            0,
+            1
+          ],
+          "DATA": [
+            "1593226799266187966947",
+            "-1812161934828717889262",
+            "-2122590131716118763227",
+            "-1027437168205961683538",
+            "1846540008444089681004",
+            "-1156292451907349126710",
+            "-1957435108076953499486",
+            "-950458759560036353191",
+            "-1125440584483224915212",
+            "2247219347276423832973"
+          ]
+        },
+        {
+          "name": "f17",
+          "count": 10,
+          "VALIDITY": [
+            1,
+            1,
+            0,
+            0,
+            1,
+            0,
+            1,
+            0,
+            1,
+            1
+          ],
+          "DATA": [
+            "-635230024201890240957",
+            "180963226287776474583",
+            "-1878152791054730240606",
+            "-1904422293952570992101",
+            "1052835439226626094544",
+            "-1858821189756108906198",
+            "1853668161015001150059",
+            "511899624049166705494",
+            "610084498432376101375",
+            "-1029828641365425116105"
+          ]
+        },
+        {
+          "name": "f18",
+          "count": 10,
+          "VALIDITY": [
+            1,
+            0,
+            1,
+            0,
+            0,
+            1,
+            0,
+            0,
+            1,
+            0
+          ],
+          "DATA": [
+            "1885658882244978382784",
+            "565125911580367694969",
+            "-1652560178353597573029",
+            "2133173545067165238416",
+            "-486940365592293262895",
+            "-2060709217874894710408",
+            "1586493850244052619259",
+            "2178470762303491581080",
+            "-1293832362817784170345",
+            "-1858110114261315174517"
+          ]
+        },
+        {
+          "name": "f19",
+          "count": 10,
+          "VALIDITY": [
+            1,
+            1,
+            0,
+            1,
+            1,
+            0,
+            1,
+            1,
+            0,
+            1
+          ],
+          "DATA": [
+            "-369225411953896208185462",
+            "-432125813663454581808312",
+            "-144256737060926287909147",
+            "437531850902150565537267",
+            "-135152047695882388519227",
+            "200989985671128391254839",
+            "412064182108667107703143",
+            "155783836458581176854752",
+            "-227172778832455276260133",
+            "212767460062823457968604"
+          ]
+        },
+        {
+          "name": "f20",
+          "count": 10,
+          "VALIDITY": [
+            1,
+            1,
+            1,
+            1,
+            1,
+            0,
+            1,
+            1,
+            1,
+            0
+          ],
+          "DATA": [
+            "403503153546865033134324",
+            "286440584824595490396339",
+            "191634056603308191156299",
+            "530416421236246257260161",
+            "-317587706869080743041040",
+            "-171530648886789759204250",
+            "222798032246148363279215",
+            "72219173848107712300756",
+            "-281960273386152702267354",
+            "-495494776734887426338667"
+          ]
+        },
+        {
+          "name": "f21",
+          "count": 10,
+          "VALIDITY": [
+            1,
+            0,
+            0,
+            1,
+            0,
+            1,
+            0,
+            1,
+            1,
+            0
+          ],
+          "DATA": [
+            "122137352759155073399221518",
+            "-78109201384474477075454853",
+            "-119068241308521301444272353",
+            "-108756965225823937786847241",
+            "-15428358494009873779245484",
+            "19614899796809003767318009",
+            "-115295286224016867112586811",
+            "-8633742807795124483538483",
+            "-150410722825301769338372723",
+            "124250719445278286695627796"
+          ]
+        },
+        {
+          "name": "f22",
+          "count": 10,
+          "VALIDITY": [
+            1,
+            1,
+            0,
+            1,
+            1,
+            1,
+            0,
+            1,
+            0,
+            0
+          ],
+          "DATA": [
+            "-99183180993590467578076798",
+            "-36808132925962854413539892",
+            "-125679607816807665333773367",
+            "-136622111609623584591267546",
+            "-17266188400280841341905013",
+            "-71769330665427864647331885",
+            "-145814031084605056753603019",
+            "-125056942233093610827766296",
+            "9249166279701860198655516",
+            "126878824313892664602568663"
+          ]
+        },
+        {
+          "name": "f23",
+          "count": 10,
+          "VALIDITY": [
+            0,
+            1,
+            1,
+            1,
+            1,
+            0,
+            0,
+            1,
+            1,
+            1
+          ],
+          "DATA": [
+            "69744825329510933906746340",
+            "-30122586028854363823971329",
+            "22212149164944035060004867",
+            "-25973928380545009708020589",
+            "50567555142199657207098957",
+            "-40021098669624532176421808",
+            "-33330934345289112585983041",
+            "-124012214741805393399462125",
+            "-138226312121129798637686830",
+            "2273005251425309465489619"
+          ]
+        },
+        {
+          "name": "f24",
+          "count": 10,
+          "VALIDITY": [
+            1,
+            0,
+            1,
+            0,
+            1,
+            1,
+            0,
+            1,
+            1,
+            1
+          ],
+          "DATA": [
+            "26842282533862498382939405997",
+            "17787994211657168910392131484",
+            "7523952681100831594207033636",
+            "18257283108725825658512455316",
+            "-15469531382712880431453128966",
+            "-5009369258227088832031787745",
+            "-35598690213497115663830724084",
+            "22246953732785347649151349772",
+            "9330915514049730010034508310",
+            "-26800441656550330938009306707"
+          ]
+        },
+        {
+          "name": "f25",
+          "count": 10,
+          "VALIDITY": [
+            1,
+            0,
+            1,
+            1,
+            1,
+            1,
+            0,
+            1,
+            0,
+            0
+          ],
+          "DATA": [
+            "-36331212919615640889272411104",
+            "36192450254329815033783240839",
+            "37749265523377026412760210674",
+            "1776086174556031952147671519",
+            "15362218353534633991283119679",
+            "34413146026389235392142970662",
+            "14292777901465553588604435837",
+            "-17090082469158479303098433453",
+            "18688801441734020761551793736",
+            "-23859805527670127110643731069"
+          ]
+        },
+        {
+          "name": "f26",
+          "count": 10,
+          "VALIDITY": [
+            1,
+            0,
+            1,
+            1,
+            1,
+            0,
+            1,
+            1,
+            1,
+            0
+          ],
+          "DATA": [
+            "2220103303731789573334199701663",
+            "6237111177014715580774407162796",
+            "-6338471790385983381447055194941",
+            "2250388387229112362557012222382",
+            "-3919498269201660475165992143963",
+            "-1799934177913445118487985955729",
+            "-2323304519983949754039480799426",
+            "-1301131662575316944670816531905",
+            "-4265160933868529886993472786495",
+            "-4292953046034527295812111533975"
+          ]
+        },
+        {
+          "name": "f27",
+          "count": 10,
+          "VALIDITY": [
+            1,
+            0,
+            1,
+            0,
+            0,
+            1,
+            1,
+            0,
+            0,
+            1
+          ],
+          "DATA": [
+            "-3484925219213002239805523064883",
+            "6528552859477682991951516050785",
+            "-765962107198382441283095455373",
+            "-1397137479831402026027607048248",
+            "3961682476407229854971103624981",
+            "5975195883181022585096451556775",
+            "-9676800477604513004081790052790",
+            "-5703219619447316623534739055181",
+            "6129137758274201914352345247446",
+            "771936840958552475578155475366"
+          ]
+        },
+        {
+          "name": "f28",
+          "count": 10,
+          "VALIDITY": [
+            1,
+            1,
+            1,
+            1,
+            0,
+            1,
+            1,
+            0,
+            0,
+            1
+          ],
+          "DATA": [
+            "7726836802487400261012835642739",
+            "-2899737712900797452349686903367",
+            "-4699841764135292946953397202917",
+            "7418179251651709234455608105088",
+            "8120397131521471010221697932900",
+            "6526213244777384369820937871939",
+            "-3774567743925527043971960901589",
+            "-8873736034246294395374537274419",
+            "488289844645088756680442972516",
+            "-2388144175550709792515001316277"
+          ]
+        },
+        {
+          "name": "f29",
+          "count": 10,
+          "VALIDITY": [
+            0,
+            0,
+            1,
+            1,
+            0,
+            0,
+            1,
+            0,
+            1,
+            1
+          ],
+          "DATA": [
+            "-47477340538926222211104914237698",
+            "320639937921939986635581989855928",
+            "861736289842228881011999521004060",
+            "-1205256013177842982651274099236361",
+            "-39643995957385740378372042150003",
+            "-1210790336528949301250817830946444",
+            "1182423445445031739603473987778350",
+            "-640337212803879852231150942968813",
+            "-567587726713873060654848198630762",
+            "280641360776654488558911759135115"
+          ]
+        },
+        {
+          "name": "f30",
+          "count": 10,
+          "VALIDITY": [
+            1,
+            1,
+            1,
+            1,
+            1,
+            1,
+            1,
+            0,
+            1,
+            1
+          ],
+          "DATA": [
+            "-866285575875610221959113146637304",
+            "2344770413948196340883160931087052",
+            "1889857580890564350097047877514790",
+            "-2568726710340227178480991779371723",
+            "-2485213596075291138559974328614959",
+            "-62378076407090361966505761644273",
+            "-2113433394374145039336066697772466",
+            "-2389429054181728925012886258903305",
+            "-2525262726376576108925615177598533",
+            "-604526323558062739370715910785200"
+          ]
+        },
+        {
+          "name": "f31",
+          "count": 10,
+          "VALIDITY": [
+            0,
+            1,
+            1,
+            1,
+            0,
+            1,
+            0,
+            1,
+            1,
+            0
+          ],
+          "DATA": [
+            "-411175611485478784812936027850047826",
+            "457804241550769132910430204506150717",
+            "1549607974636127184352732575182742",
+            "390025168459008888512891394303962448",
+            "-490664276836807094735857019444661312",
+            "-450640457243898505844766529165221166",
+            "-553339316764622659577593048830066531",
+            "19986969431028149966548563329309114",
+            "512190967497020823088167980283378342",
+            "-587244163212803971173675798133411979"
+          ]
+        },
+        {
+          "name": "f32",
+          "count": 10,
+          "VALIDITY": [
+            1,
+            0,
+            1,
+            1,
+            0,
+            0,
+            1,
+            0,
+            1,
+            1
+          ],
+          "DATA": [
+            "246512063483920590600659938977719645",
+            "483324212708049237205831199756205159",
+            "195186088040813738593870463346937474",
+            "-331572691964567402179138728488891845",
+            "466186594735574058231047101478621597",
+            "80512186957271594252263140708656889",
+            "-621018503026533776126752549530840325",
+            "-484705410290628923401085082564299400",
+            "-322852670767952007714303037031011136",
+            "306969564371703225579907932616560177"
+          ]
+        },
+        {
+          "name": "f33",
+          "count": 10,
+          "VALIDITY": [
+            0,
+            0,
+            0,
+            0,
+            0,
+            1,
+            1,
+            1,
+            0,
+            1
+          ],
+          "DATA": [
+            "52513592802194657300546985265901321701",
+            "32396354459564619210286503831104805894",
+            "-152800597997887046528471654246818061800",
+            "16886738551251131147537573820997523099",
+            "-108529890244550965992486427743581407764",
+            "-107805100101953838884357149548341742683",
+            "-152428185005223769908895650096091179800",
+            "38697638049920608411728929261890502602",
+            "-53183835204559125450983905289890638199",
+            "-25198915522830164616311861007733226015"
+          ]
+        },
+        {
+          "name": "f34",
+          "count": 10,
+          "VALIDITY": [
+            0,
+            0,
+            0,
+            1,
+            1,
+            1,
+            1,
+            0,
+            1,
+            1
+          ],
+          "DATA": [
+            "-29473659368749332611063061562898178665",
+            "-64181392425515474831866849063339846755",
+            "90557155313004461723928910484647118568",
+            "69982410088115917703700629187142714952",
+            "137281427883875761429267863353473899975",
+            "66221749426872522877433672036305282198",
+            "139898216716793863940328165403822713329",
+            "-17999019985160863257629297875199464798",
+            "74582888771441381839018581558057879959",
+            "153431389525940565745457493557728519648"
+          ]
+        },
+        {
+          "name": "f35",
+          "count": 10,
+          "VALIDITY": [
+            0,
+            1,
+            0,
+            0,
+            1,
+            0,
+            1,
+            1,
+            1,
+            0
+          ],
+          "DATA": [
+            "132130437221517226508875210221998086214",
+            "-53955540660976668373224815268780394481",
+            "-94671974318719360571824313264981742616",
+            "75940549933612625873217731049581943788",
+            "-58970299111050400318748873232294460623",
+            "106861808580585226388022589188291458328",
+            "151810935532684619700572858784852122078",
+            "147244409300271119764509412340177481716",
+            "139138676242120433411406371456479693081",
+            "58927563101546159870009467050083778558"
+          ]
+        }
+      ]
+    },
+    {
+      "count": 7,
+      "columns": [
+        {
+          "name": "f0",
+          "count": 7,
+          "VALIDITY": [
+            1,
+            0,
+            1,
+            1,
+            0,
+            1,
+            0
+          ],
+          "DATA": [
+            "25114",
+            "-12925",
+            "-1074",
+            "28646",
+            "-16569",
+            "-16770",
+            "-27151"
+          ]
+        },
+        {
+          "name": "f1",
+          "count": 7,
+          "VALIDITY": [
+            1,
+            1,
+            0,
+            0,
+            1,
+            1,
+            1
+          ],
+          "DATA": [
+            "26533",
+            "12085",
+            "25312",
+            "23445",
+            "13356",
+            "-2369",
+            "-29785"
+          ]
+        },
+        {
+          "name": "f2",
+          "count": 7,
+          "VALIDITY": [
+            1,
+            0,
+            0,
+            1,
+            1,
+            0,
+            1
+          ],
+          "DATA": [
+            "-4380859",
+            "-157657",
+            "-4200925",
+            "6061632",
+            "1972423",
+            "-3016747",
+            "2864606"
+          ]
+        },
+        {
+          "name": "f3",
+          "count": 7,
+          "VALIDITY": [
+            0,
+            0,
+            1,
+            0,
+            1,
+            1,
+            1
+          ],
+          "DATA": [
+            "-7045373",
+            "-7371800",
+            "1427921",
+            "-7601068",
+            "2525157",
+            "-6433403",
+            "-2477014"
+          ]
+        },
+        {
+          "name": "f4",
+          "count": 7,
+          "VALIDITY": [
+            0,
+            0,
+            1,
+            0,
+            1,
+            1,
+            0
+          ],
+          "DATA": [
+            "-141345816",
+            "1543059503",
+            "1211501603",
+            "369888761",
+            "894246844",
+            "736166799",
+            "409408642"
+          ]
+        },
+        {
+          "name": "f5",
+          "count": 7,
+          "VALIDITY": [
+            1,
+            1,
+            1,
+            1,
+            1,
+            0,
+            0
+          ],
+          "DATA": [
+            "-1553193945",
+            "1491657644",
+            "376067040",
+            "-1100453955",
+            "1763626627",
+            "1237652218",
+            "-1680624462"
+          ]
+        },
+        {
+          "name": "f6",
+          "count": 7,
+          "VALIDITY": [
+            0,
+            0,
+            1,
+            0,
+            1,
+            1,
+            0
+          ],
+          "DATA": [
+            "102290167",
+            "1044278987",
+            "501196467",
+            "32275276",
+            "-1384353672",
+            "594927697",
+            "-1482240101"
+          ]
+        },
+        {
+          "name": "f7",
+          "count": 7,
+          "VALIDITY": [
+            1,
+            0,
+            1,
+            1,
+            0,
+            1,
+            1
+          ],
+          "DATA": [
+            "431716525885",
+            "-396649752792",
+            "-252538061544",
+            "148825582538",
+            "536082535524",
+            "-135755487371",
+            "425356768774"
+          ]
+        },
+        {
+          "name": "f8",
+          "count": 7,
+          "VALIDITY": [
+            1,
+            1,
+            1,
+            1,
+            0,
+            1,
+            1
+          ],
+          "DATA": [
+            "137916147840",
+            "21393720222",
+            "-325824439981",
+            "278743651518",
+            "-49501503483",
+            "-419736800011",
+            "503854890502"
+          ]
+        },
+        {
+          "name": "f9",
+          "count": 7,
+          "VALIDITY": [
+            1,
+            1,
+            1,
+            1,
+            1,
+            1,
+            1
+          ],
+          "DATA": [
+            "-137642247974694",
+            "-126053382015273",
+            "32896127803947",
+            "17450989064436",
+            "-120690899531858",
+            "109217312768874",
+            "-79259001409277"
+          ]
+        },
+        {
+          "name": "f10",
+          "count": 7,
+          "VALIDITY": [
+            1,
+            1,
+            1,
+            1,
+            1,
... 35772 lines suppressed ...