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

kou pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/arrow-js.git


The following commit(s) were added to refs/heads/main by this push:
     new 572bc96  feat: Add support for BinaryView and Utf8View types (#320)
572bc96 is described below

commit 572bc96048a71e9ab6cee0d0579e423b59068fcb
Author: George <[email protected]>
AuthorDate: Wed Nov 19 03:13:50 2025 -0500

    feat: Add support for BinaryView and Utf8View types (#320)
    
    ## What's Changed
    
    This PR adds read support for BinaryView and Utf8View types (Arrow
    format 1.4.0+), enabling arrow-js to consume IPC data from systems like
    InfluxDB 3.0 and DataFusion that use view types for efficient string
    handling.
    
    ## Implementation Details
    
    ### Core Type Support
    - Added BinaryView and Utf8View type classes with view struct layout
    constants
    - Type enum entries: Type.BinaryView = 23, Type.Utf8View = 24
    - Data class support for variadic buffer management
    
    ### Visitor Pattern
    - Get visitor: Implements proper view semantics (16-byte structs,
    inline/out-of-line data)
    - Set visitor: Marks as immutable (read-only)
    - VectorLoader: Reads from IPC format with variadicBufferCounts
    - TypeComparator, TypeCtor: Type system integration
    - JSON visitors
    - Builders
    
    ### FlatBuffers
    - Generated schema files for BinaryView, Utf8View
    - Introduced `scripts/update_flatbuffers.sh` to regenerate from Arrow
    format definitions
    
    ## What Works
    - Reading BinaryView/Utf8View columns from Arrow IPC as well as JSON
    - Accessing values with proper inline/out-of-line handling
    - Variadic buffer management
    - Type checking and comparison
    - BinaryView and Utf8View Builders
    
    ## Testing
    - [X] Unit tests for BinaryView and Utf8View
    - [X] Tests verify both inline (≤12 bytes) and out-of-line data handling
    - [X] TypeScript compiles without errors
    - [X] All existing tests pass
    - [X] Builders verified
    - [X] Verified against DataFusion 50.0.3 integration, not included in
    this PR (enables native view types, removing need for configuration
    change in DataFusion's SessionConfig)
    
    ## Future Work (Separate PRs)
    - ~~Builders for write operations~~
    - ListView/LargeListView type implementation
    - ~~Additional test coverage~~
    
    Closes #311
    Related to #225
    
    ---------
    
    Co-authored-by: Paul Taylor <[email protected]>
---
 .../enable-binaryview-integration-tests.patch      |  29 +++
 .github/workflows/test.yaml                        |   3 +
 scripts/update_flatbuffers.sh                      |  75 ++++++
 src/Arrow.dom.ts                                   |   8 +-
 src/Arrow.ts                                       |   6 +-
 src/builder/binaryview.ts                          | 183 +++++++++++++++
 src/builder/utf8view.ts                            |  32 +++
 src/data.ts                                        |  92 ++++++--
 src/enum.ts                                        |   2 +
 src/fb/File.ts                                     |  47 ++++
 src/fb/binary-view.ts                              |  47 ++++
 src/fb/large-list-view.ts                          |  42 ++++
 src/fb/list-view.ts                                |  43 ++++
 src/fb/message.ts                                  |   2 +
 src/fb/record-batch.ts                             |  46 +++-
 src/fb/schema.ts                                   |  10 +-
 src/fb/type.ts                                     |  34 ++-
 src/fb/utf8-view.ts                                |  47 ++++
 src/interfaces.ts                                  |   6 +
 src/ipc/message.ts                                 |   2 +
 src/ipc/metadata/json.ts                           |  23 +-
 src/ipc/metadata/message.ts                        |  32 ++-
 src/ipc/reader.ts                                  |   9 +-
 src/ipc/writer.ts                                  |  12 +-
 src/type.ts                                        |  52 +++++
 src/visitor.ts                                     |   6 +
 src/visitor/builderctor.ts                         |   4 +
 src/visitor/get.ts                                 |  51 +++-
 src/visitor/indexof.ts                             |   6 +-
 src/visitor/iterator.ts                            |   6 +-
 src/visitor/jsontypeassembler.ts                   |   6 +
 src/visitor/jsonvectorassembler.ts                 |  56 ++++-
 src/visitor/set.ts                                 |  62 ++++-
 src/visitor/typeassembler.ts                       |  10 +
 src/visitor/typecomparator.ts                      |   6 +-
 src/visitor/vectorassembler.ts                     |  22 +-
 src/visitor/vectorloader.ts                        | 147 ++++++++++--
 test/data/tables.ts                                |   2 +-
 test/generate-test-data.ts                         |  28 ++-
 test/unit/builders/view-builders-tests.ts          | 258 +++++++++++++++++++++
 test/unit/generated-data-tests.ts                  |   2 +
 test/unit/vector/vector-tests.ts                   |  46 +++-
 42 files changed, 1523 insertions(+), 79 deletions(-)

diff --git a/.github/patches/enable-binaryview-integration-tests.patch 
b/.github/patches/enable-binaryview-integration-tests.patch
new file mode 100644
index 0000000..ac5c17e
--- /dev/null
+++ b/.github/patches/enable-binaryview-integration-tests.patch
@@ -0,0 +1,29 @@
+# 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.
+
+diff --git a/dev/archery/archery/integration/datagen.py 
b/dev/archery/archery/integration/datagen.py
+index 83913dc379..7ace28e1be 100644
+--- a/dev/archery/archery/integration/datagen.py
++++ b/dev/archery/archery/integration/datagen.py
+@@ -2003,7 +2003,6 @@ def get_generated_json_files(tempdir=None):
+         .skip_tester('Rust'),
+
+         generate_binary_view_case()
+-        .skip_tester('JS')
+         # TODO(https://github.com/apache/arrow-nanoarrow/issues/618)
+         .skip_tester('nanoarrow')
+         .skip_tester('Rust'),
diff --git a/.github/workflows/test.yaml b/.github/workflows/test.yaml
index a02ebce..85f2e04 100644
--- a/.github/workflows/test.yaml
+++ b/.github/workflows/test.yaml
@@ -193,6 +193,9 @@ jobs:
         uses: actions/setup-python@e797f83bcb11b83ae66e0230d6156d7c80228e7c # 
v6.0.0
         with:
           python-version: 3
+      - name: Patch Archery to enable BinaryView tests
+        run: |
+          patch -p1 < 
js/.github/patches/enable-binaryview-integration-tests.patch
       - name: Setup Archery
         run: pip install -e dev/archery[docker]
       - name: Execute Docker Build
diff --git a/scripts/update_flatbuffers.sh b/scripts/update_flatbuffers.sh
new file mode 100755
index 0000000..a439593
--- /dev/null
+++ b/scripts/update_flatbuffers.sh
@@ -0,0 +1,75 @@
+#!/usr/bin/env bash
+#
+# 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.
+
+# Regenerate the FlatBuffers helper files used by arrow-js. Requires a sibling
+# checkout of apache/arrow (../arrow) if not provided in env and a working 
flatc on PATH.
+
+set -euo pipefail
+
+PROJECT_ROOT="$(cd "$(dirname "${BASH_SOURCE[0]}")/.." && pwd)"
+FORMAT_DIR="${PROJECT_ROOT}/../arrow/format"
+
+if [[ ! -d "${FORMAT_DIR}" ]]; then
+  echo "error: expected FlatBuffers schemas in ${FORMAT_DIR}" >&2
+  exit 1
+fi
+
+if ! command -v flatc >/dev/null 2>&1; then
+  echo "error: flatc not found on PATH" >&2
+  exit 1
+fi
+
+TMPDIR="$(mktemp -d "${PROJECT_ROOT}/.flatc.XXXXXX")"
+cleanup() {
+  rm -rf "${TMPDIR}"
+}
+trap cleanup EXIT
+
+schemas=(File Schema Message Tensor SparseTensor)
+
+for schema in "${schemas[@]}"; do
+  cp "${FORMAT_DIR}/${schema}.fbs" "${TMPDIR}/${schema}.fbs"
+  sed \
+    -e 's/namespace org.apache.arrow.flatbuf;//g' \
+    -e 's/org\.apache\.arrow\.flatbuf\.//g' \
+    "${FORMAT_DIR}/${schema}.fbs" >"${TMPDIR}/${schema}.fbs"
+done
+
+flatc --ts --ts-flat-files --ts-omit-entrypoint \
+  -o "${TMPDIR}" \
+  "${TMPDIR}"/{File,Schema,Message,Tensor,SparseTensor}.fbs
+
+generated_files=(
+  binary-view.ts
+  list-view.ts
+  large-list-view.ts
+  message.ts
+  record-batch.ts
+  schema.ts
+  type.ts
+  utf8-view.ts
+)
+
+for file in "${generated_files[@]}"; do
+  if [[ ! -f "${TMPDIR}/${file}" ]]; then
+    echo "error: expected generated file ${file} not found" >&2
+    exit 1
+  fi
+  install -m 0644 "${TMPDIR}/${file}" "${PROJECT_ROOT}/src/fb/${file}"
+done
diff --git a/src/Arrow.dom.ts b/src/Arrow.dom.ts
index e0cd681..30feeb8 100644
--- a/src/Arrow.dom.ts
+++ b/src/Arrow.dom.ts
@@ -49,8 +49,8 @@ export {
     Bool,
     Int, Int8, Int16, Int32, Int64, Uint8, Uint16, Uint32, Uint64,
     Float, Float16, Float32, Float64,
-    Utf8, LargeUtf8,
-    Binary, LargeBinary,
+    Utf8, LargeUtf8, Utf8View,
+    Binary, LargeBinary, BinaryView,
     FixedSizeBinary,
     Date_, DateDay, DateMillisecond,
     Timestamp, TimestampSecond, TimestampMillisecond, TimestampMicrosecond, 
TimestampNanosecond,
@@ -81,7 +81,7 @@ export {
 } from './Arrow.js';
 
 export {
-    BinaryBuilder, LargeBinaryBuilder,
+    BinaryBuilder, BinaryViewBuilder, LargeBinaryBuilder,
     BoolBuilder,
     DateBuilder, DateDayBuilder, DateMillisecondBuilder,
     DecimalBuilder,
@@ -99,5 +99,5 @@ export {
     TimestampBuilder, TimestampSecondBuilder, TimestampMillisecondBuilder, 
TimestampMicrosecondBuilder, TimestampNanosecondBuilder,
     TimeBuilder, TimeSecondBuilder, TimeMillisecondBuilder, 
TimeMicrosecondBuilder, TimeNanosecondBuilder,
     UnionBuilder, DenseUnionBuilder, SparseUnionBuilder,
-    Utf8Builder, LargeUtf8Builder
+    Utf8Builder, Utf8ViewBuilder, LargeUtf8Builder
 } from './Arrow.js';
diff --git a/src/Arrow.ts b/src/Arrow.ts
index 8321026..2049583 100644
--- a/src/Arrow.ts
+++ b/src/Arrow.ts
@@ -37,8 +37,8 @@ export {
     Bool,
     Int, Int8, Int16, Int32, Int64, Uint8, Uint16, Uint32, Uint64,
     Float, Float16, Float32, Float64,
-    Utf8, LargeUtf8,
-    Binary, LargeBinary,
+    Utf8, LargeUtf8, Utf8View,
+    Binary, LargeBinary, BinaryView,
     FixedSizeBinary,
     Date_, DateDay, DateMillisecond,
     Timestamp, TimestampSecond, TimestampMillisecond, TimestampMicrosecond, 
TimestampNanosecond,
@@ -79,8 +79,10 @@ export { TimestampBuilder, TimestampSecondBuilder, 
TimestampMillisecondBuilder,
 export { IntervalBuilder, IntervalDayTimeBuilder, IntervalYearMonthBuilder, 
IntervalMonthDayNanoBuilder } from './builder/interval.js';
 export { DurationBuilder, DurationSecondBuilder, DurationMillisecondBuilder, 
DurationMicrosecondBuilder, DurationNanosecondBuilder } from 
'./builder/duration.js';
 export { Utf8Builder } from './builder/utf8.js';
+export { Utf8ViewBuilder } from './builder/utf8view.js';
 export { LargeUtf8Builder } from './builder/largeutf8.js';
 export { BinaryBuilder } from './builder/binary.js';
+export { BinaryViewBuilder } from './builder/binaryview.js';
 export { LargeBinaryBuilder } from './builder/largebinary.js';
 export { ListBuilder } from './builder/list.js';
 export { FixedSizeListBuilder } from './builder/fixedsizelist.js';
diff --git a/src/builder/binaryview.ts b/src/builder/binaryview.ts
new file mode 100644
index 0000000..002bfc3
--- /dev/null
+++ b/src/builder/binaryview.ts
@@ -0,0 +1,183 @@
+// 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.
+
+import { BinaryView, Utf8View } from '../type.js';
+import { Builder, BuilderOptions } from '../builder.js';
+import { BufferBuilder } from './buffer.js';
+import { toUint8Array } from '../util/buffer.js';
+import { makeData } from '../data.js';
+import type { DataProps } from '../data.js';
+
+/** @ignore */
+export class BinaryViewBuilder<
+    TType extends BinaryView | Utf8View = BinaryView,
+    TNull = any
+> extends Builder<TType, TNull> {
+    protected _views: BufferBuilder<Uint8Array>;
+    protected _variadicBuffers: Uint8Array[] = [];
+    protected _currentBuffer: BufferBuilder<Uint8Array> | null = null;
+    protected _currentBufferIndex = 0;
+    protected _currentBufferOffset = 0;
+    protected readonly _bufferSize = 32 * 1024 * 1024; // 32MB per buffer as 
per spec recommendation
+
+    constructor(opts: BuilderOptions<TType, TNull>) {
+        super(opts);
+        this._views = new BufferBuilder(Uint8Array);
+    }
+
+    public get byteLength(): number {
+        let size = 0;
+        this._views && (size += this._views.byteLength);
+        this._nulls && (size += this._nulls.byteLength);
+        for (const buffer of this._variadicBuffers) {
+            size += buffer.byteLength;
+        }
+        this._currentBuffer && (size += this._currentBuffer.byteLength);
+        return size;
+    }
+
+    public setValue(index: number, value: TType['TValue']) {
+        return this.writeBinaryValue(index, this.encodeValue(value));
+    }
+
+    protected writeBinaryValue(index: number, data: Uint8Array) {
+        const length = data.length;
+
+        // Ensure views buffer has space up to this index (similar to 
FixedWidthBuilder)
+        const bytesNeeded = (index + 1) * BinaryView.ELEMENT_WIDTH;
+        const currentBytes = this._views.length;
+        if (bytesNeeded > currentBytes) {
+            this._views.reserve(bytesNeeded - currentBytes);
+        }
+
+        const viewBuffer = this._views.buffer;
+        const viewOffset = index * BinaryView.ELEMENT_WIDTH;
+        const view = new DataView(viewBuffer.buffer, viewBuffer.byteOffset + 
viewOffset, BinaryView.ELEMENT_WIDTH);
+
+        // Write length (4 bytes, little-endian)
+        view.setInt32(BinaryView.LENGTH_OFFSET, length, true);
+
+        if (length <= BinaryView.INLINE_CAPACITY) {
+            // Inline: store data directly in view struct (up to 12 bytes)
+            viewBuffer.set(data, viewOffset + BinaryView.INLINE_OFFSET);
+            // Zero out remaining bytes
+            for (let i = length; i < BinaryView.INLINE_CAPACITY; i++) {
+                viewBuffer[viewOffset + BinaryView.INLINE_OFFSET + i] = 0;
+            }
+        } else {
+            // Out-of-line: store in variadic buffer
+            // Write prefix (first 4 bytes of data)
+            const prefix = new DataView(data.buffer, data.byteOffset, 
Math.min(4, length));
+            view.setUint32(BinaryView.INLINE_OFFSET, prefix.getUint32(0, 
true), true);
+
+            // Allocate space in variadic buffer
+            if (!this._currentBuffer || this._currentBufferOffset + length > 
this._bufferSize) {
+                // Start a new buffer
+                if (this._currentBuffer) {
+                    
this._variadicBuffers.push(this._currentBuffer.buffer.slice(0, 
this._currentBufferOffset));
+                }
+                this._currentBuffer = new BufferBuilder(Uint8Array);
+                this._currentBufferIndex = this._variadicBuffers.length;
+                this._currentBufferOffset = 0;
+            }
+
+            // Write data to current buffer
+            const bufferData = this._currentBuffer.reserve(length).buffer;
+            bufferData.set(data, this._currentBufferOffset);
+
+            // Write buffer index and offset to view struct
+            view.setInt32(BinaryView.BUFFER_INDEX_OFFSET, 
this._currentBufferIndex, true);
+            view.setInt32(BinaryView.BUFFER_OFFSET_OFFSET, 
this._currentBufferOffset, true);
+
+            this._currentBufferOffset += length;
+        }
+
+        return this;
+    }
+
+    protected encodeValue(value: TType['TValue']): Uint8Array {
+        return toUint8Array(value as unknown as Uint8Array);
+    }
+
+    public setValid(index: number, isValid: boolean) {
+        // Ensure space is allocated in the views buffer for this index
+        const bytesNeeded = (index + 1) * BinaryView.ELEMENT_WIDTH;
+        const currentBytes = this._views.length;
+        if (bytesNeeded > currentBytes) {
+            this._views.reserve(bytesNeeded - currentBytes);
+        }
+
+        const result = super.setValid(index, isValid);
+
+        if (!result) {
+            // For null values, zero out the view struct
+            const viewBuffer = this._views.buffer;
+            const viewOffset = index * BinaryView.ELEMENT_WIDTH;
+            for (let i = 0; i < BinaryView.ELEMENT_WIDTH; i++) {
+                viewBuffer[viewOffset + i] = 0;
+            }
+        }
+
+        return result;
+    }
+
+    public clear() {
+        this._variadicBuffers = [];
+        this._currentBuffer = null;
+        this._currentBufferIndex = 0;
+        this._currentBufferOffset = 0;
+        this._views.clear();
+        return super.clear();
+    }
+
+    public flush() {
+        const { type, length, nullCount, _views, _nulls } = this;
+
+        // Finalize current buffer if it exists
+        if (this._currentBuffer && this._currentBufferOffset > 0) {
+            this._variadicBuffers.push(this._currentBuffer.buffer.slice(0, 
this._currentBufferOffset));
+            this._currentBuffer = null;
+            this._currentBufferOffset = 0;
+        }
+
+        const views = _views.flush(length * BinaryView.ELEMENT_WIDTH);
+        const nullBitmap = nullCount > 0 ? _nulls.flush(length) : undefined;
+        const variadicBuffers = this._variadicBuffers.slice();
+
+        // Reset variadic buffers for next batch
+        this._variadicBuffers = [];
+        this._currentBufferIndex = 0;
+
+        this.clear();
+
+        const props = {
+            type,
+            length,
+            nullCount,
+            nullBitmap,
+            ['views']: views,
+            ['variadicBuffers']: variadicBuffers
+        };
+
+        return makeData<TType>(props as unknown as DataProps<TType>);
+    }
+
+    public finish() {
+        this.finished = true;
+        return this;
+    }
+}
diff --git a/src/builder/utf8view.ts b/src/builder/utf8view.ts
new file mode 100644
index 0000000..0fea14c
--- /dev/null
+++ b/src/builder/utf8view.ts
@@ -0,0 +1,32 @@
+// 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.
+
+import { Utf8View } from '../type.js';
+import { BuilderOptions } from '../builder.js';
+import { BinaryViewBuilder } from './binaryview.js';
+import { encodeUtf8 } from '../util/utf8.js';
+
+/** @ignore */
+export class Utf8ViewBuilder<TNull = any> extends BinaryViewBuilder<Utf8View, 
TNull> {
+    constructor(opts: BuilderOptions<Utf8View, TNull>) {
+        super(opts);
+    }
+
+    public override setValue(index: number, value: Utf8View['TValue']) {
+        return this.writeBinaryValue(index, encodeUtf8(value));
+    }
+}
diff --git a/src/data.ts b/src/data.ts
index 45fcc35..b5edff8 100644
--- a/src/data.ts
+++ b/src/data.ts
@@ -68,6 +68,7 @@ export class Data<T extends DataType = DataType> {
     declare public readonly typeIds: Buffers<T>[BufferType.TYPE];
     declare public readonly nullBitmap: Buffers<T>[BufferType.VALIDITY];
     declare public readonly valueOffsets: Buffers<T>[BufferType.OFFSET];
+    declare public readonly variadicBuffers: ReadonlyArray<Uint8Array>;
 
     public get typeId(): T['TType'] { return this.type.typeId; }
 
@@ -97,6 +98,7 @@ export class Data<T extends DataType = DataType> {
         values && (byteLength += values.byteLength);
         nullBitmap && (byteLength += nullBitmap.byteLength);
         typeIds && (byteLength += typeIds.byteLength);
+        byteLength += this.variadicBuffers.reduce((size, data) => size + 
(data?.byteLength ?? 0), 0);
         return this.children.reduce((byteLength, child) => byteLength + 
child.byteLength, byteLength);
     }
 
@@ -117,7 +119,16 @@ export class Data<T extends DataType = DataType> {
         return nullCount;
     }
 
-    constructor(type: T, offset: number, length: number, nullCount?: number, 
buffers?: Partial<Buffers<T>> | Data<T>, children: Data[] = [], dictionary?: 
Vector) {
+    constructor(
+        type: T,
+        offset: number,
+        length: number,
+        nullCount?: number,
+        buffers?: Partial<Buffers<T>> | Data<T>,
+        children: Data[] = [],
+        dictionary?: Vector,
+        variadicBuffers: ReadonlyArray<Uint8Array> = []
+    ) {
         this.type = type;
         this.children = children;
         this.dictionary = dictionary;
@@ -131,6 +142,7 @@ export class Data<T extends DataType = DataType> {
             this.typeIds = buffers.typeIds;
             this.nullBitmap = buffers.nullBitmap;
             this.valueOffsets = buffers.valueOffsets;
+            this.variadicBuffers = buffers.variadicBuffers;
         } else {
             this.stride = strideForType(type);
             if (buffers) {
@@ -139,15 +151,22 @@ export class Data<T extends DataType = DataType> {
                 (buffer = (buffers as Buffers<T>)[2]) && (this.nullBitmap = 
buffer);
                 (buffer = (buffers as Buffers<T>)[3]) && (this.typeIds = 
buffer);
             }
+            this.variadicBuffers = variadicBuffers;
         }
+        this.variadicBuffers ??= [];
     }
 
     public getValid(index: number): boolean {
         const { type } = this;
         if (DataType.isUnion(type)) {
             const union = (<unknown>type as Union);
-            const child = 
this.children[union.typeIdToChildIndex[this.typeIds[index]]];
-            const indexInChild = union.mode === UnionMode.Dense ? 
this.valueOffsets[index] : index;
+            const typeId = this.typeIds[index];
+            const childIndex = union.typeIdToChildIndex[typeId];
+            const child = this.children[childIndex];
+            const valueOffsets = this.valueOffsets as Int32Array | 
BigInt64Array | undefined;
+            const indexInChild = union.mode === UnionMode.Dense && valueOffsets
+                ? Number(valueOffsets[index])
+                : index;
             return child.getValid(indexInChild);
         }
         if (this.nullable && this.nullCount > 0) {
@@ -163,8 +182,13 @@ export class Data<T extends DataType = DataType> {
         const { type } = this;
         if (DataType.isUnion(type)) {
             const union = (<unknown>type as Union);
-            const child = 
this.children[union.typeIdToChildIndex[this.typeIds[index]]];
-            const indexInChild = union.mode === UnionMode.Dense ? 
this.valueOffsets[index] : index;
+            const typeId = this.typeIds[index];
+            const childIndex = union.typeIdToChildIndex[typeId];
+            const child = this.children[childIndex];
+            const valueOffsets = this.valueOffsets as Int32Array | 
BigInt64Array | undefined;
+            const indexInChild = union.mode === UnionMode.Dense && valueOffsets
+                ? Number(valueOffsets[index])
+                : index;
             prev = child.getValid(indexInChild);
             child.setValid(indexInChild, value);
         } else {
@@ -200,8 +224,16 @@ export class Data<T extends DataType = DataType> {
         return value;
     }
 
-    public clone<R extends DataType = T>(type: R = this.type as any, offset = 
this.offset, length = this.length, nullCount = this._nullCount, buffers: 
Buffers<R> = <any>this, children: Data[] = this.children) {
-        return new Data(type, offset, length, nullCount, buffers, children, 
this.dictionary);
+    public clone<R extends DataType = T>(
+        type: R = this.type as any,
+        offset = this.offset,
+        length = this.length,
+        nullCount = this._nullCount,
+        buffers: Buffers<R> = <any>this,
+        children: Data[] = this.children,
+        variadicBuffers: ReadonlyArray<Uint8Array> = this.variadicBuffers
+    ) {
+        return new Data(type, offset, length, nullCount, buffers, children, 
this.dictionary, variadicBuffers);
     }
 
     public slice(offset: number, length: number): Data<T> {
@@ -214,12 +246,13 @@ export class Data<T extends DataType = DataType> {
         const buffers = this._sliceBuffers(offset, length, stride, typeId);
         return this.clone<T>(this.type, this.offset + offset, length, 
nullCount, buffers,
             // Don't slice children if we have value offsets (the 
variable-width types)
-            (children.length === 0 || this.valueOffsets) ? children : 
this._sliceChildren(children, childStride * offset, childStride * length));
+            (children.length === 0 || this.valueOffsets) ? children : 
this._sliceChildren(children, childStride * offset, childStride * length),
+            this.variadicBuffers);
     }
 
     public _changeLengthAndBackfillNullBitmap(newLength: number): Data<T> {
         if (this.typeId === Type.Null) {
-            return this.clone(this.type, 0, newLength, 0);
+            return this.clone(this.type, 0, newLength, 0, <any>this.buffers, 
this.children, this.variadicBuffers);
         }
         const { length, nullCount } = this;
         // start initialized with 0s (nulls), then fill from 0 to length with 
1s (not null)
@@ -232,7 +265,7 @@ export class Data<T extends DataType = DataType> {
         }
         const buffers = this.buffers;
         buffers[BufferType.VALIDITY] = bitmap;
-        return this.clone(this.type, 0, newLength, nullCount + (newLength - 
length), buffers);
+        return this.clone(this.type, 0, newLength, nullCount + (newLength - 
length), buffers, this.children, this.variadicBuffers);
     }
 
     protected _sliceBuffers(offset: number, length: number, stride: number, 
typeId: T['TType']): Buffers<T> {
@@ -240,10 +273,15 @@ export class Data<T extends DataType = DataType> {
         const { buffers } = this;
         // If typeIds exist, slice the typeIds buffer
         (arr = buffers[BufferType.TYPE]) && (buffers[BufferType.TYPE] = 
arr.subarray(offset, offset + length));
-        // If offsets exist, only slice the offsets buffer
-        (arr = buffers[BufferType.OFFSET]) && (buffers[BufferType.OFFSET] = 
arr.subarray(offset, offset + length + 1)) ||
-            // Otherwise if no offsets, slice the data buffer. Don't slice the 
data vector for Booleans, since the offset goes by bits not bytes
-            (arr = buffers[BufferType.DATA]) && (buffers[BufferType.DATA] = 
typeId === 6 ? arr : arr.subarray(stride * offset, stride * (offset + length)));
+        if (DataType.isBinaryView(this.type) || 
DataType.isUtf8View(this.type)) {
+            const width = BinaryView.ELEMENT_WIDTH;
+            (arr = buffers[BufferType.DATA]) && (buffers[BufferType.DATA] = 
arr.subarray(offset * width, (offset + length) * width));
+        } else {
+            // If offsets exist, only slice the offsets buffer
+            (arr = buffers[BufferType.OFFSET]) && (buffers[BufferType.OFFSET] 
= arr.subarray(offset, offset + length + 1)) ||
+                // Otherwise if no offsets, slice the data buffer. Don't slice 
the data vector for Booleans, since the offset goes by bits not bytes
+                (arr = buffers[BufferType.DATA]) && (buffers[BufferType.DATA] 
= typeId === 6 ? arr : arr.subarray(stride * offset, stride * (offset + 
length)));
+        }
         return buffers;
     }
 
@@ -256,7 +294,7 @@ export class Data<T extends DataType = DataType> {
 
 import {
     Dictionary,
-    Bool, Null, Utf8, LargeUtf8, Binary, LargeBinary, Decimal, 
FixedSizeBinary, List, FixedSizeList, Map_, Struct,
+    Bool, Null, Utf8, Utf8View, LargeUtf8, Binary, BinaryView, LargeBinary, 
Decimal, FixedSizeBinary, List, FixedSizeList, Map_, Struct,
     Float,
     Int,
     Date_,
@@ -311,6 +349,15 @@ class MakeDataVisitor extends Visitor {
         const { ['length']: length = valueOffsets.length - 1, ['nullCount']: 
nullCount = props['nullBitmap'] ? -1 : 0 } = props;
         return new Data(type, offset, length, nullCount, [valueOffsets, data, 
nullBitmap]);
     }
+    public visitUtf8View<T extends Utf8View>(props: Utf8ViewDataProps<T>) {
+        const { ['type']: type, ['offset']: offset = 0 } = props;
+        const views = toArrayBufferView(type.ArrayType, props['views']);
+        const nullBitmap = toUint8Array(props['nullBitmap']);
+        const variadicBuffers = (props['variadicBuffers'] || []).map((buffer) 
=> toUint8Array(buffer));
+        const length = props['length'] ?? Math.trunc(views.length / 
Utf8View.ELEMENT_WIDTH);
+        const nullCount = props['nullBitmap'] ? -1 : 0;
+        return new Data(type, offset, length, nullCount, [undefined, views, 
nullBitmap], [], undefined, variadicBuffers);
+    }
     public visitLargeUtf8<T extends LargeUtf8>(props: LargeUtf8DataProps<T>) {
         const { ['type']: type, ['offset']: offset = 0 } = props;
         const data = toUint8Array(props['data']);
@@ -327,6 +374,15 @@ class MakeDataVisitor extends Visitor {
         const { ['length']: length = valueOffsets.length - 1, ['nullCount']: 
nullCount = props['nullBitmap'] ? -1 : 0 } = props;
         return new Data(type, offset, length, nullCount, [valueOffsets, data, 
nullBitmap]);
     }
+    public visitBinaryView<T extends BinaryView>(props: 
BinaryViewDataProps<T>) {
+        const { ['type']: type, ['offset']: offset = 0 } = props;
+        const views = toArrayBufferView(type.ArrayType, props['views']);
+        const nullBitmap = toUint8Array(props['nullBitmap']);
+        const variadicBuffers = (props['variadicBuffers'] || []).map((buffer) 
=> toUint8Array(buffer));
+        const length = props['length'] ?? Math.trunc(views.length / 
BinaryView.ELEMENT_WIDTH);
+        const nullCount = props['nullBitmap'] ? -1 : 0;
+        return new Data(type, offset, length, nullCount, [undefined, views, 
nullBitmap], [], undefined, variadicBuffers);
+    }
     public visitLargeBinary<T extends LargeBinary>(props: 
LargeBinaryDataProps<T>) {
         const { ['type']: type, ['offset']: offset = 0 } = props;
         const data = toUint8Array(props['data']);
@@ -455,8 +511,10 @@ interface IntervalDataProps<T extends Interval> extends 
DataProps_<T> { data?: D
 interface DurationDataProps<T extends Duration> extends DataProps_<T> { data?: 
DataBuffer<T> }
 interface FixedSizeBinaryDataProps<T extends FixedSizeBinary> extends 
DataProps_<T> { data?: DataBuffer<T> }
 interface BinaryDataProps<T extends Binary> extends DataProps_<T> { 
valueOffsets: ValueOffsetsBuffer; data?: DataBuffer<T> }
+interface BinaryViewDataProps<T extends BinaryView> extends DataProps_<T> { 
views: DataBuffer<T>; variadicBuffers?: ReadonlyArray<ArrayLike<number> | 
Iterable<number> | Uint8Array>; data?: DataBuffer<T> }
 interface LargeBinaryDataProps<T extends LargeBinary> extends DataProps_<T> { 
valueOffsets: LargeValueOffsetsBuffer | ValueOffsetsBuffer; data?: 
DataBuffer<T> }
 interface Utf8DataProps<T extends Utf8> extends DataProps_<T> { valueOffsets: 
ValueOffsetsBuffer; data?: DataBuffer<T> }
+interface Utf8ViewDataProps<T extends Utf8View> extends DataProps_<T> { views: 
DataBuffer<T>; variadicBuffers?: ReadonlyArray<ArrayLike<number> | 
Iterable<number> | Uint8Array>; data?: DataBuffer<T> }
 interface LargeUtf8DataProps<T extends LargeUtf8> extends DataProps_<T> { 
valueOffsets: LargeValueOffsetsBuffer | ValueOffsetsBuffer; data?: 
DataBuffer<T> }
 interface ListDataProps<T extends List> extends DataProps_<T> { valueOffsets: 
ValueOffsetsBuffer; child: Data<T['valueType']> }
 interface FixedSizeListDataProps<T extends FixedSizeList> extends 
DataProps_<T> { child: Data<T['valueType']> }
@@ -481,8 +539,10 @@ export type DataProps<T extends DataType> = (
     T extends FixedSizeBinary /* */ ? FixedSizeBinaryDataProps<T> :
     T extends Binary /*          */ ? BinaryDataProps<T> :
     T extends LargeBinary /*     */ ? LargeBinaryDataProps<T> :
+    T extends BinaryView /*      */ ? BinaryViewDataProps<T> :
     T extends Utf8 /*            */ ? Utf8DataProps<T> :
     T extends LargeUtf8 /*       */ ? LargeUtf8DataProps<T> :
+    T extends Utf8View /*        */ ? Utf8ViewDataProps<T> :
     T extends List /*            */ ? ListDataProps<T> :
     T extends FixedSizeList /*   */ ? FixedSizeListDataProps<T> :
     T extends Struct /*          */ ? StructDataProps<T> :
@@ -507,10 +567,12 @@ export function makeData<T extends Timestamp>(props: 
TimestampDataProps<T>): Dat
 export function makeData<T extends Interval>(props: IntervalDataProps<T>): 
Data<T>;
 export function makeData<T extends Duration>(props: DurationDataProps<T>): 
Data<T>;
 export function makeData<T extends FixedSizeBinary>(props: 
FixedSizeBinaryDataProps<T>): Data<T>;
+export function makeData<T extends BinaryView>(props: BinaryViewDataProps<T>): 
Data<T>;
 export function makeData<T extends Binary>(props: BinaryDataProps<T>): Data<T>;
 export function makeData<T extends LargeBinary>(props: 
LargeBinaryDataProps<T>): Data<T>;
 export function makeData<T extends Utf8>(props: Utf8DataProps<T>): Data<T>;
 export function makeData<T extends LargeUtf8>(props: LargeUtf8DataProps<T>): 
Data<T>;
+export function makeData<T extends Utf8View>(props: Utf8ViewDataProps<T>): 
Data<T>;
 export function makeData<T extends List>(props: ListDataProps<T>): Data<T>;
 export function makeData<T extends FixedSizeList>(props: 
FixedSizeListDataProps<T>): Data<T>;
 export function makeData<T extends Struct>(props: StructDataProps<T>): Data<T>;
diff --git a/src/enum.ts b/src/enum.ts
index 73d9553..514a816 100644
--- a/src/enum.ts
+++ b/src/enum.ts
@@ -70,6 +70,8 @@ export enum Type {
     Duration = 18, /** Measure of elapsed time in either seconds, 
milliseconds, microseconds or nanoseconds */
     LargeBinary = 19, /** Large variable-length bytes (no guarantee of 
UTF8-ness) */
     LargeUtf8 = 20, /** Large variable-length string as List<Char> */
+    BinaryView = 23, /** Variable-length binary values backed by 
inline-or-referenced views */
+    Utf8View = 24, /** Variable-length UTF8 string values backed by 
inline-or-referenced views */
 
     Dictionary = -1, /** Dictionary aka Category type */
     Int8 = -2,
diff --git a/src/fb/File.ts b/src/fb/File.ts
new file mode 100644
index 0000000..12c6f82
--- /dev/null
+++ b/src/fb/File.ts
@@ -0,0 +1,47 @@
+// automatically generated by the FlatBuffers compiler, do not modify
+
+/* eslint-disable @typescript-eslint/no-unused-vars, 
@typescript-eslint/no-explicit-any, @typescript-eslint/no-non-null-assertion */
+
+export { Binary } from './binary.js';
+export { BinaryView } from './binary-view.js';
+export { Block } from './block.js';
+export { Bool } from './bool.js';
+export { Buffer } from './buffer.js';
+export { Date } from './date.js';
+export { DateUnit } from './date-unit.js';
+export { Decimal } from './decimal.js';
+export { DictionaryEncoding } from './dictionary-encoding.js';
+export { DictionaryKind } from './dictionary-kind.js';
+export { Duration } from './duration.js';
+export { Endianness } from './endianness.js';
+export { Feature } from './feature.js';
+export { Field } from './field.js';
+export { FixedSizeBinary } from './fixed-size-binary.js';
+export { FixedSizeList } from './fixed-size-list.js';
+export { FloatingPoint } from './floating-point.js';
+export { Footer } from './footer.js';
+export { Int } from './int.js';
+export { Interval } from './interval.js';
+export { IntervalUnit } from './interval-unit.js';
+export { KeyValue } from './key-value.js';
+export { LargeBinary } from './large-binary.js';
+export { LargeList } from './large-list.js';
+export { LargeListView } from './large-list-view.js';
+export { LargeUtf8 } from './large-utf8.js';
+export { List } from './list.js';
+export { ListView } from './list-view.js';
+export { Map } from './map.js';
+export { MetadataVersion } from './metadata-version.js';
+export { Null } from './null.js';
+export { Precision } from './precision.js';
+export { RunEndEncoded } from './run-end-encoded.js';
+export { Schema } from './schema.js';
+export { Struct_ } from './struct-.js';
+export { Time } from './time.js';
+export { TimeUnit } from './time-unit.js';
+export { Timestamp } from './timestamp.js';
+export { Type } from './type.js';
+export { Union } from './union.js';
+export { UnionMode } from './union-mode.js';
+export { Utf8 } from './utf8.js';
+export { Utf8View } from './utf8-view.js';
diff --git a/src/fb/binary-view.ts b/src/fb/binary-view.ts
new file mode 100644
index 0000000..f91f910
--- /dev/null
+++ b/src/fb/binary-view.ts
@@ -0,0 +1,47 @@
+// automatically generated by the FlatBuffers compiler, do not modify
+
+/* eslint-disable @typescript-eslint/no-unused-vars, 
@typescript-eslint/no-explicit-any, @typescript-eslint/no-non-null-assertion */
+
+import * as flatbuffers from 'flatbuffers';
+
+/**
+ * Logically the same as Binary, but the internal representation uses a view
+ * struct that contains the string length and either the string's entire data
+ * inline (for small strings) or an inlined prefix, an index of another buffer,
+ * and an offset pointing to a slice in that buffer (for non-small strings).
+ *
+ * Since it uses a variable number of data buffers, each Field with this type
+ * must have a corresponding entry in `variadicBufferCounts`.
+ */
+export class BinaryView {
+  bb: flatbuffers.ByteBuffer|null = null;
+  bb_pos = 0;
+  __init(i:number, bb:flatbuffers.ByteBuffer):BinaryView {
+  this.bb_pos = i;
+  this.bb = bb;
+  return this;
+}
+
+static getRootAsBinaryView(bb:flatbuffers.ByteBuffer, 
obj?:BinaryView):BinaryView {
+  return (obj || new BinaryView()).__init(bb.readInt32(bb.position()) + 
bb.position(), bb);
+}
+
+static getSizePrefixedRootAsBinaryView(bb:flatbuffers.ByteBuffer, 
obj?:BinaryView):BinaryView {
+  bb.setPosition(bb.position() + flatbuffers.SIZE_PREFIX_LENGTH);
+  return (obj || new BinaryView()).__init(bb.readInt32(bb.position()) + 
bb.position(), bb);
+}
+
+static startBinaryView(builder:flatbuffers.Builder) {
+  builder.startObject(0);
+}
+
+static endBinaryView(builder:flatbuffers.Builder):flatbuffers.Offset {
+  const offset = builder.endObject();
+  return offset;
+}
+
+static createBinaryView(builder:flatbuffers.Builder):flatbuffers.Offset {
+  BinaryView.startBinaryView(builder);
+  return BinaryView.endBinaryView(builder);
+}
+}
diff --git a/src/fb/large-list-view.ts b/src/fb/large-list-view.ts
new file mode 100644
index 0000000..5785cd3
--- /dev/null
+++ b/src/fb/large-list-view.ts
@@ -0,0 +1,42 @@
+// automatically generated by the FlatBuffers compiler, do not modify
+
+/* eslint-disable @typescript-eslint/no-unused-vars, 
@typescript-eslint/no-explicit-any, @typescript-eslint/no-non-null-assertion */
+
+import * as flatbuffers from 'flatbuffers';
+
+/**
+ * Same as ListView, but with 64-bit offsets and sizes, allowing to represent
+ * extremely large data values.
+ */
+export class LargeListView {
+  bb: flatbuffers.ByteBuffer|null = null;
+  bb_pos = 0;
+  __init(i:number, bb:flatbuffers.ByteBuffer):LargeListView {
+  this.bb_pos = i;
+  this.bb = bb;
+  return this;
+}
+
+static getRootAsLargeListView(bb:flatbuffers.ByteBuffer, 
obj?:LargeListView):LargeListView {
+  return (obj || new LargeListView()).__init(bb.readInt32(bb.position()) + 
bb.position(), bb);
+}
+
+static getSizePrefixedRootAsLargeListView(bb:flatbuffers.ByteBuffer, 
obj?:LargeListView):LargeListView {
+  bb.setPosition(bb.position() + flatbuffers.SIZE_PREFIX_LENGTH);
+  return (obj || new LargeListView()).__init(bb.readInt32(bb.position()) + 
bb.position(), bb);
+}
+
+static startLargeListView(builder:flatbuffers.Builder) {
+  builder.startObject(0);
+}
+
+static endLargeListView(builder:flatbuffers.Builder):flatbuffers.Offset {
+  const offset = builder.endObject();
+  return offset;
+}
+
+static createLargeListView(builder:flatbuffers.Builder):flatbuffers.Offset {
+  LargeListView.startLargeListView(builder);
+  return LargeListView.endLargeListView(builder);
+}
+}
diff --git a/src/fb/list-view.ts b/src/fb/list-view.ts
new file mode 100644
index 0000000..f9afae0
--- /dev/null
+++ b/src/fb/list-view.ts
@@ -0,0 +1,43 @@
+// automatically generated by the FlatBuffers compiler, do not modify
+
+/* eslint-disable @typescript-eslint/no-unused-vars, 
@typescript-eslint/no-explicit-any, @typescript-eslint/no-non-null-assertion */
+
+import * as flatbuffers from 'flatbuffers';
+
+/**
+ * Represents the same logical types that List can, but contains offsets and
+ * sizes allowing for writes in any order and sharing of child values among
+ * list values.
+ */
+export class ListView {
+  bb: flatbuffers.ByteBuffer|null = null;
+  bb_pos = 0;
+  __init(i:number, bb:flatbuffers.ByteBuffer):ListView {
+  this.bb_pos = i;
+  this.bb = bb;
+  return this;
+}
+
+static getRootAsListView(bb:flatbuffers.ByteBuffer, obj?:ListView):ListView {
+  return (obj || new ListView()).__init(bb.readInt32(bb.position()) + 
bb.position(), bb);
+}
+
+static getSizePrefixedRootAsListView(bb:flatbuffers.ByteBuffer, 
obj?:ListView):ListView {
+  bb.setPosition(bb.position() + flatbuffers.SIZE_PREFIX_LENGTH);
+  return (obj || new ListView()).__init(bb.readInt32(bb.position()) + 
bb.position(), bb);
+}
+
+static startListView(builder:flatbuffers.Builder) {
+  builder.startObject(0);
+}
+
+static endListView(builder:flatbuffers.Builder):flatbuffers.Offset {
+  const offset = builder.endObject();
+  return offset;
+}
+
+static createListView(builder:flatbuffers.Builder):flatbuffers.Offset {
+  ListView.startListView(builder);
+  return ListView.endListView(builder);
+}
+}
diff --git a/src/fb/message.ts b/src/fb/message.ts
index d752b91..d351859 100644
--- a/src/fb/message.ts
+++ b/src/fb/message.ts
@@ -1,5 +1,7 @@
 // automatically generated by the FlatBuffers compiler, do not modify
 
+/* eslint-disable @typescript-eslint/no-unused-vars, 
@typescript-eslint/no-explicit-any, @typescript-eslint/no-non-null-assertion */
+
 import * as flatbuffers from 'flatbuffers';
 
 import { KeyValue } from './key-value.js';
diff --git a/src/fb/record-batch.ts b/src/fb/record-batch.ts
index 0068199..e6f41d0 100644
--- a/src/fb/record-batch.ts
+++ b/src/fb/record-batch.ts
@@ -1,5 +1,7 @@
 // automatically generated by the FlatBuffers compiler, do not modify
 
+/* eslint-disable @typescript-eslint/no-unused-vars, 
@typescript-eslint/no-explicit-any, @typescript-eslint/no-non-null-assertion */
+
 import * as flatbuffers from 'flatbuffers';
 
 import { BodyCompression } from './body-compression.js';
@@ -78,8 +80,34 @@ compression(obj?:BodyCompression):BodyCompression|null {
   return offset ? (obj || new 
BodyCompression()).__init(this.bb!.__indirect(this.bb_pos + offset), this.bb!) 
: null;
 }
 
+/**
+ * Some types such as Utf8View are represented using a variable number of 
buffers.
+ * For each such Field in the pre-ordered flattened logical schema, there will 
be
+ * an entry in variadicBufferCounts to indicate the number of number of 
variadic
+ * buffers which belong to that Field in the current RecordBatch.
+ *
+ * For example, the schema
+ *     col1: Struct<alpha: Int32, beta: BinaryView, gamma: Float64>
+ *     col2: Utf8View
+ * contains two Fields with variadic buffers so variadicBufferCounts will have
+ * two entries, the first counting the variadic buffers of `col1.beta` and the
+ * second counting `col2`'s.
+ *
+ * This field may be omitted if and only if the schema contains no Fields with
+ * a variable number of buffers, such as BinaryView and Utf8View.
+ */
+variadicBufferCounts(index: number):bigint|null {
+  const offset = this.bb!.__offset(this.bb_pos, 12);
+  return offset ? this.bb!.readInt64(this.bb!.__vector(this.bb_pos + offset) + 
index * 8) : BigInt(0);
+}
+
+variadicBufferCountsLength():number {
+  const offset = this.bb!.__offset(this.bb_pos, 12);
+  return offset ? this.bb!.__vector_len(this.bb_pos + offset) : 0;
+}
+
 static startRecordBatch(builder:flatbuffers.Builder) {
-  builder.startObject(4);
+  builder.startObject(5);
 }
 
 static addLength(builder:flatbuffers.Builder, length:bigint) {
@@ -106,6 +134,22 @@ static addCompression(builder:flatbuffers.Builder, 
compressionOffset:flatbuffers
   builder.addFieldOffset(3, compressionOffset, 0);
 }
 
+static addVariadicBufferCounts(builder:flatbuffers.Builder, 
variadicBufferCountsOffset:flatbuffers.Offset) {
+  builder.addFieldOffset(4, variadicBufferCountsOffset, 0);
+}
+
+static createVariadicBufferCountsVector(builder:flatbuffers.Builder, 
data:bigint[]):flatbuffers.Offset {
+  builder.startVector(8, data.length, 8);
+  for (let i = data.length - 1; i >= 0; i--) {
+    builder.addInt64(data[i]!);
+  }
+  return builder.endVector();
+}
+
+static startVariadicBufferCountsVector(builder:flatbuffers.Builder, 
numElems:number) {
+  builder.startVector(8, numElems, 8);
+}
+
 static endRecordBatch(builder:flatbuffers.Builder):flatbuffers.Offset {
   const offset = builder.endObject();
   return offset;
diff --git a/src/fb/schema.ts b/src/fb/schema.ts
index 394883e..daae447 100644
--- a/src/fb/schema.ts
+++ b/src/fb/schema.ts
@@ -1,5 +1,7 @@
 // automatically generated by the FlatBuffers compiler, do not modify
 
+/* eslint-disable @typescript-eslint/no-unused-vars, 
@typescript-eslint/no-explicit-any, @typescript-eslint/no-non-null-assertion */
+
 import * as flatbuffers from 'flatbuffers';
 
 import { Endianness } from './endianness.js';
@@ -133,14 +135,6 @@ static 
endSchema(builder:flatbuffers.Builder):flatbuffers.Offset {
   return offset;
 }
 
-static finishSchemaBuffer(builder:flatbuffers.Builder, 
offset:flatbuffers.Offset) {
-  builder.finish(offset);
-}
-
-static finishSizePrefixedSchemaBuffer(builder:flatbuffers.Builder, 
offset:flatbuffers.Offset) {
-  builder.finish(offset, undefined, true);
-}
-
 static createSchema(builder:flatbuffers.Builder, endianness:Endianness, 
fieldsOffset:flatbuffers.Offset, customMetadataOffset:flatbuffers.Offset, 
featuresOffset:flatbuffers.Offset):flatbuffers.Offset {
   Schema.startSchema(builder);
   Schema.addEndianness(builder, endianness);
diff --git a/src/fb/type.ts b/src/fb/type.ts
index 8eb8704..8f913d0 100644
--- a/src/fb/type.ts
+++ b/src/fb/type.ts
@@ -1,6 +1,9 @@
 // automatically generated by the FlatBuffers compiler, do not modify
 
+/* eslint-disable @typescript-eslint/no-unused-vars, 
@typescript-eslint/no-explicit-any, @typescript-eslint/no-non-null-assertion */
+
 import { Binary } from './binary.js';
+import { BinaryView } from './binary-view.js';
 import { Bool } from './bool.js';
 import { Date } from './date.js';
 import { Decimal } from './decimal.js';
@@ -12,8 +15,10 @@ import { Int } from './int.js';
 import { Interval } from './interval.js';
 import { LargeBinary } from './large-binary.js';
 import { LargeList } from './large-list.js';
+import { LargeListView } from './large-list-view.js';
 import { LargeUtf8 } from './large-utf8.js';
 import { List } from './list.js';
+import { ListView } from './list-view.js';
 import { Map } from './map.js';
 import { Null } from './null.js';
 import { RunEndEncoded } from './run-end-encoded.js';
@@ -22,6 +27,7 @@ import { Time } from './time.js';
 import { Timestamp } from './timestamp.js';
 import { Union } from './union.js';
 import { Utf8 } from './utf8.js';
+import { Utf8View } from './utf8-view.js';
 
 
 /**
@@ -52,15 +58,19 @@ export enum Type {
   LargeBinary = 19,
   LargeUtf8 = 20,
   LargeList = 21,
-  RunEndEncoded = 22
+  RunEndEncoded = 22,
+  BinaryView = 23,
+  Utf8View = 24,
+  ListView = 25,
+  LargeListView = 26
 }
 
 export function unionToType(
   type: Type,
-  accessor: 
(obj:Binary|Bool|Date|Decimal|Duration|FixedSizeBinary|FixedSizeList|FloatingPoint|Int|Interval|LargeBinary|LargeList|LargeUtf8|List|Map|Null|RunEndEncoded|Struct_|Time|Timestamp|Union|Utf8)
 => 
Binary|Bool|Date|Decimal|Duration|FixedSizeBinary|FixedSizeList|FloatingPoint|Int|Interval|LargeBinary|LargeList|LargeUtf8|List|Map|Null|RunEndEncoded|Struct_|Time|Timestamp|Union|Utf8|null
-): 
Binary|Bool|Date|Decimal|Duration|FixedSizeBinary|FixedSizeList|FloatingPoint|Int|Interval|LargeBinary|LargeList|LargeUtf8|List|Map|Null|RunEndEncoded|Struct_|Time|Timestamp|Union|Utf8|null
 {
+  accessor: 
(obj:Binary|BinaryView|Bool|Date|Decimal|Duration|FixedSizeBinary|FixedSizeList|FloatingPoint|Int|Interval|LargeBinary|LargeList|LargeListView|LargeUtf8|List|ListView|Map|Null|RunEndEncoded|Struct_|Time|Timestamp|Union|Utf8|Utf8View)
 => 
Binary|BinaryView|Bool|Date|Decimal|Duration|FixedSizeBinary|FixedSizeList|FloatingPoint|Int|Interval|LargeBinary|LargeList|LargeListView|LargeUtf8|List|ListView|Map|Null|RunEndEncoded|Struct_|Time|Timestamp|Union|Utf8|Utf8View|null
+): 
Binary|BinaryView|Bool|Date|Decimal|Duration|FixedSizeBinary|FixedSizeList|FloatingPoint|Int|Interval|LargeBinary|LargeList|LargeListView|LargeUtf8|List|ListView|Map|Null|RunEndEncoded|Struct_|Time|Timestamp|Union|Utf8|Utf8View|null
 {
   switch(Type[type]) {
-    case 'NONE': return null;
+    case 'NONE': return null; 
     case 'Null': return accessor(new Null())! as Null;
     case 'Int': return accessor(new Int())! as Int;
     case 'FloatingPoint': return accessor(new FloatingPoint())! as 
FloatingPoint;
@@ -83,17 +93,21 @@ export function unionToType(
     case 'LargeUtf8': return accessor(new LargeUtf8())! as LargeUtf8;
     case 'LargeList': return accessor(new LargeList())! as LargeList;
     case 'RunEndEncoded': return accessor(new RunEndEncoded())! as 
RunEndEncoded;
+    case 'BinaryView': return accessor(new BinaryView())! as BinaryView;
+    case 'Utf8View': return accessor(new Utf8View())! as Utf8View;
+    case 'ListView': return accessor(new ListView())! as ListView;
+    case 'LargeListView': return accessor(new LargeListView())! as 
LargeListView;
     default: return null;
   }
 }
 
 export function unionListToType(
-  type: Type,
-  accessor: (index: number, 
obj:Binary|Bool|Date|Decimal|Duration|FixedSizeBinary|FixedSizeList|FloatingPoint|Int|Interval|LargeBinary|LargeList|LargeUtf8|List|Map|Null|RunEndEncoded|Struct_|Time|Timestamp|Union|Utf8)
 => 
Binary|Bool|Date|Decimal|Duration|FixedSizeBinary|FixedSizeList|FloatingPoint|Int|Interval|LargeBinary|LargeList|LargeUtf8|List|Map|Null|RunEndEncoded|Struct_|Time|Timestamp|Union|Utf8|null,
+  type: Type, 
+  accessor: (index: number, 
obj:Binary|BinaryView|Bool|Date|Decimal|Duration|FixedSizeBinary|FixedSizeList|FloatingPoint|Int|Interval|LargeBinary|LargeList|LargeListView|LargeUtf8|List|ListView|Map|Null|RunEndEncoded|Struct_|Time|Timestamp|Union|Utf8|Utf8View)
 => 
Binary|BinaryView|Bool|Date|Decimal|Duration|FixedSizeBinary|FixedSizeList|FloatingPoint|Int|Interval|LargeBinary|LargeList|LargeListView|LargeUtf8|List|ListView|Map|Null|RunEndEncoded|Struct_|Time|Timestamp|Union|Utf8|Utf8View|null,
 
   index: number
-): 
Binary|Bool|Date|Decimal|Duration|FixedSizeBinary|FixedSizeList|FloatingPoint|Int|Interval|LargeBinary|LargeList|LargeUtf8|List|Map|Null|RunEndEncoded|Struct_|Time|Timestamp|Union|Utf8|null
 {
+): 
Binary|BinaryView|Bool|Date|Decimal|Duration|FixedSizeBinary|FixedSizeList|FloatingPoint|Int|Interval|LargeBinary|LargeList|LargeListView|LargeUtf8|List|ListView|Map|Null|RunEndEncoded|Struct_|Time|Timestamp|Union|Utf8|Utf8View|null
 {
   switch(Type[type]) {
-    case 'NONE': return null;
+    case 'NONE': return null; 
     case 'Null': return accessor(index, new Null())! as Null;
     case 'Int': return accessor(index, new Int())! as Int;
     case 'FloatingPoint': return accessor(index, new FloatingPoint())! as 
FloatingPoint;
@@ -116,6 +130,10 @@ export function unionListToType(
     case 'LargeUtf8': return accessor(index, new LargeUtf8())! as LargeUtf8;
     case 'LargeList': return accessor(index, new LargeList())! as LargeList;
     case 'RunEndEncoded': return accessor(index, new RunEndEncoded())! as 
RunEndEncoded;
+    case 'BinaryView': return accessor(index, new BinaryView())! as BinaryView;
+    case 'Utf8View': return accessor(index, new Utf8View())! as Utf8View;
+    case 'ListView': return accessor(index, new ListView())! as ListView;
+    case 'LargeListView': return accessor(index, new LargeListView())! as 
LargeListView;
     default: return null;
   }
 }
diff --git a/src/fb/utf8-view.ts b/src/fb/utf8-view.ts
new file mode 100644
index 0000000..886a9df
--- /dev/null
+++ b/src/fb/utf8-view.ts
@@ -0,0 +1,47 @@
+// automatically generated by the FlatBuffers compiler, do not modify
+
+/* eslint-disable @typescript-eslint/no-unused-vars, 
@typescript-eslint/no-explicit-any, @typescript-eslint/no-non-null-assertion */
+
+import * as flatbuffers from 'flatbuffers';
+
+/**
+ * Logically the same as Utf8, but the internal representation uses a view
+ * struct that contains the string length and either the string's entire data
+ * inline (for small strings) or an inlined prefix, an index of another buffer,
+ * and an offset pointing to a slice in that buffer (for non-small strings).
+ *
+ * Since it uses a variable number of data buffers, each Field with this type
+ * must have a corresponding entry in `variadicBufferCounts`.
+ */
+export class Utf8View {
+  bb: flatbuffers.ByteBuffer|null = null;
+  bb_pos = 0;
+  __init(i:number, bb:flatbuffers.ByteBuffer):Utf8View {
+  this.bb_pos = i;
+  this.bb = bb;
+  return this;
+}
+
+static getRootAsUtf8View(bb:flatbuffers.ByteBuffer, obj?:Utf8View):Utf8View {
+  return (obj || new Utf8View()).__init(bb.readInt32(bb.position()) + 
bb.position(), bb);
+}
+
+static getSizePrefixedRootAsUtf8View(bb:flatbuffers.ByteBuffer, 
obj?:Utf8View):Utf8View {
+  bb.setPosition(bb.position() + flatbuffers.SIZE_PREFIX_LENGTH);
+  return (obj || new Utf8View()).__init(bb.readInt32(bb.position()) + 
bb.position(), bb);
+}
+
+static startUtf8View(builder:flatbuffers.Builder) {
+  builder.startObject(0);
+}
+
+static endUtf8View(builder:flatbuffers.Builder):flatbuffers.Offset {
+  const offset = builder.endObject();
+  return offset;
+}
+
+static createUtf8View(builder:flatbuffers.Builder):flatbuffers.Offset {
+  Utf8View.startUtf8View(builder);
+  return Utf8View.endUtf8View(builder);
+}
+}
diff --git a/src/interfaces.ts b/src/interfaces.ts
index 0645753..eea88bd 100644
--- a/src/interfaces.ts
+++ b/src/interfaces.ts
@@ -212,6 +212,7 @@ export type TypeToDataType<T extends Type> = {
     [Type.LargeUtf8]: type.LargeUtf8;
     [Type.Binary]: type.Binary;
     [Type.LargeBinary]: type.LargeBinary;
+    [Type.BinaryView]: type.BinaryView;
     [Type.FixedSizeBinary]: type.FixedSizeBinary;
     [Type.Date]: type.Date_;
     [Type.DateDay]: type.DateDay;
@@ -244,6 +245,7 @@ export type TypeToDataType<T extends Type> = {
     [Type.Struct]: type.Struct;
     [Type.Dictionary]: type.Dictionary;
     [Type.FixedSizeList]: type.FixedSizeList;
+    [Type.Utf8View]: type.Utf8View;
 }[T];
 
 /** @ignore */
@@ -268,6 +270,7 @@ type TypeToBuilder<T extends Type = any, TNull = any> = {
     [Type.LargeUtf8]: LargeUtf8Builder<TNull>;
     [Type.Binary]: BinaryBuilder<TNull>;
     [Type.LargeBinary]: LargeBinaryBuilder<TNull>;
+    [Type.BinaryView]: Builder<any, TNull>;
     [Type.FixedSizeBinary]: FixedSizeBinaryBuilder<TNull>;
     [Type.Date]: DateBuilder<any, TNull>;
     [Type.DateDay]: DateDayBuilder<TNull>;
@@ -300,6 +303,7 @@ type TypeToBuilder<T extends Type = any, TNull = any> = {
     [Type.Struct]: StructBuilder<any, TNull>;
     [Type.Dictionary]: DictionaryBuilder<any, TNull>;
     [Type.FixedSizeList]: FixedSizeListBuilder<any, TNull>;
+    [Type.Utf8View]: Builder<any, TNull>;
 }[T];
 
 /** @ignore */
@@ -324,6 +328,7 @@ type DataTypeToBuilder<T extends DataType = any, TNull = 
any> = {
     [Type.LargeUtf8]: T extends type.LargeUtf8 ? LargeUtf8Builder<TNull> : 
never;
     [Type.Binary]: T extends type.Binary ? BinaryBuilder<TNull> : never;
     [Type.LargeBinary]: T extends type.LargeBinary ? LargeBinaryBuilder<TNull> 
: never;
+    [Type.BinaryView]: T extends type.BinaryView ? Builder<any, TNull> : never;
     [Type.FixedSizeBinary]: T extends type.FixedSizeBinary ? 
FixedSizeBinaryBuilder<TNull> : never;
     [Type.Date]: T extends type.Date_ ? DateBuilder<T, TNull> : never;
     [Type.DateDay]: T extends type.DateDay ? DateDayBuilder<TNull> : never;
@@ -356,4 +361,5 @@ type DataTypeToBuilder<T extends DataType = any, TNull = 
any> = {
     [Type.Struct]: T extends type.Struct ? StructBuilder<T['dataTypes'], 
TNull> : never;
     [Type.Dictionary]: T extends type.Dictionary ? DictionaryBuilder<T, TNull> 
: never;
     [Type.FixedSizeList]: T extends type.FixedSizeList ? 
FixedSizeListBuilder<T['valueType'], TNull> : never;
+    [Type.Utf8View]: T extends type.Utf8View ? Builder<any, TNull> : never;
 }[T['TType']];
diff --git a/src/ipc/message.ts b/src/ipc/message.ts
index 3dc8625..40a6543 100644
--- a/src/ipc/message.ts
+++ b/src/ipc/message.ts
@@ -204,6 +204,8 @@ export class JSONMessageReader extends MessageReader {
                 ...(column['TYPE_ID'] && [column['TYPE_ID']] || []),
                 ...(column['OFFSET'] && [column['OFFSET']] || []),
                 ...(column['DATA'] && [column['DATA']] || []),
+                ...(column['VIEWS'] && [column['VIEWS']] || []),
+                ...(column['VARIADIC_DATA_BUFFERS'] || []),
                 ...flattenDataSources(column['children'])
             ], [] as any[][]);
         }
diff --git a/src/ipc/metadata/json.ts b/src/ipc/metadata/json.ts
index 15f8718..8aed54e 100644
--- a/src/ipc/metadata/json.ts
+++ b/src/ipc/metadata/json.ts
@@ -18,7 +18,7 @@
 import { Schema, Field } from '../../schema.js';
 import {
     DataType, Dictionary, TimeBitWidth,
-    Utf8, LargeUtf8, Binary, LargeBinary, Decimal, FixedSizeBinary,
+    Utf8, LargeUtf8, Binary, LargeBinary, BinaryView, Utf8View, Decimal, 
FixedSizeBinary,
     List, FixedSizeList, Map_, Struct, Union,
     Bool, Null, Int, Float, Date_, Time, Interval, Timestamp, IntBitWidth, 
Int32, TKeys, Duration,
 } from '../../type.js';
@@ -41,7 +41,8 @@ export function recordBatchFromJSON(b: any) {
         b['count'],
         fieldNodesFromJSON(b['columns']),
         buffersFromJSON(b['columns']),
-        null
+        null,
+        variadicBufferCountsFromJSON(b['columns'])
     );
 }
 
@@ -83,6 +84,13 @@ function buffersFromJSON(xs: any[], buffers: BufferRegion[] 
= []): BufferRegion[
         column['TYPE_ID'] && buffers.push(new BufferRegion(buffers.length, 
column['TYPE_ID'].length));
         column['OFFSET'] && buffers.push(new BufferRegion(buffers.length, 
column['OFFSET'].length));
         column['DATA'] && buffers.push(new BufferRegion(buffers.length, 
column['DATA'].length));
+        column['VIEWS'] && buffers.push(new BufferRegion(buffers.length, 
column['VIEWS'].length));
+        // Handle variadic buffers for view types (BinaryView, Utf8View)
+        if (column['VARIADIC_DATA_BUFFERS']) {
+            for (const buf of column['VARIADIC_DATA_BUFFERS']) {
+                buffers.push(new BufferRegion(buffers.length, buf.length));
+            }
+        }
         buffers = buffersFromJSON(column['children'], buffers);
     }
     return buffers;
@@ -93,6 +101,15 @@ function nullCountFromJSON(validity: number[]) {
     return (validity || []).reduce((sum, val) => sum + +(val === 0), 0);
 }
 
+/** @ignore */
+function variadicBufferCountsFromJSON(xs: any[]): number[] {
+    return (xs || []).reduce<number[]>((counts, column: any) => [
+        ...counts,
+        ...(column['VARIADIC_DATA_BUFFERS'] ? 
[column['VARIADIC_DATA_BUFFERS'].length] : []),
+        ...variadicBufferCountsFromJSON(column['children'])
+    ], [] as number[]);
+}
+
 /** @ignore */
 export function fieldFromJSON(_field: any, dictionaries?: Map<number, 
DataType>) {
 
@@ -149,8 +166,10 @@ function typeFromJSON(f: any, children?: Field[]): 
DataType<any> {
         case 'null': return new Null();
         case 'binary': return new Binary();
         case 'largebinary': return new LargeBinary();
+        case 'binaryview': return new BinaryView();
         case 'utf8': return new Utf8();
         case 'largeutf8': return new LargeUtf8();
+        case 'utf8view': return new Utf8View();
         case 'bool': return new Bool();
         case 'list': return new List((children || [])[0]);
         case 'struct': return new Struct(children || []);
diff --git a/src/ipc/metadata/message.ts b/src/ipc/metadata/message.ts
index 17e8897..b41ec4a 100644
--- a/src/ipc/metadata/message.ts
+++ b/src/ipc/metadata/message.ts
@@ -57,7 +57,7 @@ import ByteBuffer = flatbuffers.ByteBuffer;
 
 import {
     DataType, Dictionary, TimeBitWidth,
-    Utf8, LargeUtf8, Binary, LargeBinary, Decimal, FixedSizeBinary,
+    Utf8, LargeUtf8, Binary, LargeBinary, BinaryView, Utf8View, Decimal, 
FixedSizeBinary,
     List, FixedSizeList, Map_, Struct, Union,
     Bool, Null, Int, Float, Date_, Time, Interval, Timestamp, IntBitWidth, 
Int32, TKeys, Duration,
 } from '../../type.js';
@@ -156,20 +156,24 @@ export class RecordBatch {
     protected _nodes: FieldNode[];
     protected _buffers: BufferRegion[];
     protected _compression: BodyCompression | null;
+    protected _variadicBufferCounts: number[];
     public get nodes() { return this._nodes; }
     public get length() { return this._length; }
     public get buffers() { return this._buffers; }
     public get compression() { return this._compression; }
+    public get variadicBufferCounts() { return this._variadicBufferCounts; }
     constructor(
         length: bigint | number,
         nodes: FieldNode[],
         buffers: BufferRegion[],
-        compression: BodyCompression | null
+        compression: BodyCompression | null,
+        variadicBufferCounts: number[] = []
     ) {
         this._nodes = nodes;
         this._buffers = buffers;
         this._length = bigIntToNumber(length);
         this._compression = compression;
+        this._variadicBufferCounts = variadicBufferCounts;
     }
 }
 
@@ -334,7 +338,8 @@ function decodeRecordBatch(batch: _RecordBatch, version = 
MetadataVersion.V5) {
         batch.length(),
         decodeFieldNodes(batch),
         decodeBuffers(batch, version),
-        decodeBodyCompression(batch.compression())
+        decodeBodyCompression(batch.compression()),
+        decodeVariadicBufferCounts(batch)
     );
     return recordBatch;
 }
@@ -382,6 +387,16 @@ function decodeBuffers(batch: _RecordBatch, version: 
MetadataVersion) {
     return bufferRegions;
 }
 
+/** @ignore */
+function decodeVariadicBufferCounts(batch: _RecordBatch) {
+    const counts = [] as number[];
+    const length = Math.trunc(batch.variadicBufferCountsLength());
+    for (let i = 0; i < length; ++i) {
+        counts.push(bigIntToNumber(batch.variadicBufferCounts(i)!));
+    }
+    return counts;
+}
+
 /** @ignore */
 function decodeSchemaFields(schema: _Schema, dictionaries?: Map<number, 
DataType>) {
     const fields = [] as Field[];
@@ -468,8 +483,10 @@ function decodeFieldType(f: _Field, children?: Field[]): 
DataType<any> {
         case Type['Null']: return new Null();
         case Type['Binary']: return new Binary();
         case Type['LargeBinary']: return new LargeBinary();
+        case Type['BinaryView']: return new BinaryView();
         case Type['Utf8']: return new Utf8();
         case Type['LargeUtf8']: return new LargeUtf8();
+        case Type['Utf8View']: return new Utf8View();
         case Type['Bool']: return new Bool();
         case Type['List']: return new List((children || [])[0]);
         case Type['Struct_']: return new Struct(children || []);
@@ -614,6 +631,7 @@ function encodeRecordBatch(b: Builder, recordBatch: 
RecordBatch) {
 
     const nodes = recordBatch.nodes || [];
     const buffers = recordBatch.buffers || [];
+    const variadicBufferCounts = recordBatch.variadicBufferCounts || [];
 
     _RecordBatch.startNodesVector(b, nodes.length);
     for (const n of nodes.slice().reverse()) FieldNode.encode(b, n);
@@ -630,6 +648,11 @@ function encodeRecordBatch(b: Builder, recordBatch: 
RecordBatch) {
         bodyCompressionOffset = encodeBodyCompression(b, 
recordBatch.compression);
     }
 
+    let variadicBufferCountsOffset = -1;
+    if (variadicBufferCounts.length > 0) {
+        variadicBufferCountsOffset = 
_RecordBatch.createVariadicBufferCountsVector(b, 
variadicBufferCounts.map(BigInt));
+    }
+
     _RecordBatch.startRecordBatch(b);
     _RecordBatch.addLength(b, BigInt(recordBatch.length));
     _RecordBatch.addNodes(b, nodesVectorOffset);
@@ -637,6 +660,9 @@ function encodeRecordBatch(b: Builder, recordBatch: 
RecordBatch) {
     if (recordBatch.compression !== null && bodyCompressionOffset) {
         _RecordBatch.addCompression(b, bodyCompressionOffset);
     }
+    if (variadicBufferCountsOffset !== -1) {
+        _RecordBatch.addVariadicBufferCounts(b, variadicBufferCountsOffset);
+    }
     return _RecordBatch.endRecordBatch(b);
 }
 
diff --git a/src/ipc/reader.ts b/src/ipc/reader.ts
index e36eeb5..af49f37 100644
--- a/src/ipc/reader.ts
+++ b/src/ipc/reader.ts
@@ -397,7 +397,8 @@ abstract class RecordBatchReaderImpl<T extends TypeMap = 
any> implements RecordB
                     header.data.length,
                     header.data.nodes,
                     buffers,
-                    null
+                    null,
+                    header.data.variadicBufferCounts
                 ), id, isDelta)
             } else {
                 throw new Error('Dictionary batch is compressed but codec not 
found');
@@ -412,11 +413,11 @@ abstract class RecordBatchReaderImpl<T extends TypeMap = 
any> implements RecordB
     }
 
     protected _loadVectors(header: metadata.RecordBatch, body: Uint8Array, 
types: (Field | DataType)[]) {
-        return new VectorLoader(body, header.nodes, header.buffers, 
this.dictionaries, this.schema.metadataVersion).visitMany(types);
+        return new VectorLoader(body, header.nodes, header.buffers, 
this.dictionaries, this.schema.metadataVersion, 
header.variadicBufferCounts).visitMany(types);
     }
 
     protected _loadCompressedVectors(header: metadata.RecordBatch, body: 
Uint8Array[], types: (Field | DataType)[]) {
-        return new CompressedVectorLoader(body, header.nodes, header.buffers, 
this.dictionaries, this.schema.metadataVersion).visitMany(types);
+        return new CompressedVectorLoader(body, header.nodes, header.buffers, 
this.dictionaries, this.schema.metadataVersion, 
header.variadicBufferCounts).visitMany(types);
     }
 
     private _decompressBuffers(header: metadata.RecordBatch, body: Uint8Array, 
codec: Codec): { decommpressedBody: Uint8Array[]; buffers: 
metadata.BufferRegion[] } {
@@ -757,7 +758,7 @@ class RecordBatchJSONReaderImpl<T extends TypeMap = any> 
extends RecordBatchStre
         super(source, dictionaries);
     }
     protected _loadVectors(header: metadata.RecordBatch, body: any, types: 
(Field | DataType)[]) {
-        return new JSONVectorLoader(body, header.nodes, header.buffers, 
this.dictionaries, this.schema.metadataVersion).visitMany(types);
+        return new JSONVectorLoader(body, header.nodes, header.buffers, 
this.dictionaries, this.schema.metadataVersion, 
header.variadicBufferCounts).visitMany(types);
     }
 }
 
diff --git a/src/ipc/writer.ts b/src/ipc/writer.ts
index 17c8f0b..0b13fdf 100644
--- a/src/ipc/writer.ts
+++ b/src/ipc/writer.ts
@@ -274,8 +274,8 @@ export class RecordBatchWriter<T extends TypeMap = any> 
extends ReadableInterop<
     }
 
     protected _writeRecordBatch(batch: RecordBatch<T>) {
-        const { byteLength, nodes, bufferRegions, buffers } = 
this._assembleRecordBatch(batch);
-        const recordBatch = new metadata.RecordBatch(batch.numRows, nodes, 
bufferRegions, this._compression);
+        const { byteLength, nodes, bufferRegions, buffers, 
variadicBufferCounts } = this._assembleRecordBatch(batch);
+        const recordBatch = new metadata.RecordBatch(batch.numRows, nodes, 
bufferRegions, this._compression, variadicBufferCounts);
         const message = Message.from(recordBatch, byteLength);
         return this
             ._writeDictionaries(batch)
@@ -284,11 +284,11 @@ export class RecordBatchWriter<T extends TypeMap = any> 
extends ReadableInterop<
     }
 
     protected _assembleRecordBatch(batch: RecordBatch<T> | Vector) {
-        let { byteLength, nodes, bufferRegions, buffers } = 
VectorAssembler.assemble(batch);
+        let { byteLength, nodes, bufferRegions, buffers, variadicBufferCounts 
} = VectorAssembler.assemble(batch);
         if (this._compression != null) {
             ({ byteLength, bufferRegions, buffers } = 
this._compressBodyBuffers(buffers));
         }
-        return { byteLength, nodes, bufferRegions, buffers };
+        return { byteLength, nodes, bufferRegions, buffers, 
variadicBufferCounts };
     }
 
     protected _compressBodyBuffers(buffers: ArrayBufferView[]) {
@@ -337,8 +337,8 @@ export class RecordBatchWriter<T extends TypeMap = any> 
extends ReadableInterop<
     }
 
     protected _writeDictionaryBatch(dictionary: Data, id: number, isDelta = 
false) {
-        const { byteLength, nodes, bufferRegions, buffers } = 
this._assembleRecordBatch(new Vector([dictionary]));
-        const recordBatch = new metadata.RecordBatch(dictionary.length, nodes, 
bufferRegions, this._compression);
+        const { byteLength, nodes, bufferRegions, buffers, 
variadicBufferCounts } = this._assembleRecordBatch(new Vector([dictionary]));
+        const recordBatch = new metadata.RecordBatch(dictionary.length, nodes, 
bufferRegions, this._compression, variadicBufferCounts);
         const dictionaryBatch = new metadata.DictionaryBatch(recordBatch, id, 
isDelta);
         const message = Message.from(dictionaryBatch, byteLength);
         return this
diff --git a/src/type.ts b/src/type.ts
index ea5e24f..f1fc3fc 100644
--- a/src/type.ts
+++ b/src/type.ts
@@ -58,8 +58,10 @@ export abstract class DataType<TType extends Type = Type, 
TChildren extends Type
     /** @nocollapse */ static isInt(x: any): x is Int_ { return x?.typeId === 
Type.Int; }
     /** @nocollapse */ static isFloat(x: any): x is Float { return x?.typeId 
=== Type.Float; }
     /** @nocollapse */ static isBinary(x: any): x is Binary { return x?.typeId 
=== Type.Binary; }
+    /** @nocollapse */ static isBinaryView(x: any): x is BinaryView { return 
x?.typeId === Type.BinaryView; }
     /** @nocollapse */ static isLargeBinary(x: any): x is LargeBinary { return 
x?.typeId === Type.LargeBinary; }
     /** @nocollapse */ static isUtf8(x: any): x is Utf8 { return x?.typeId === 
Type.Utf8; }
+    /** @nocollapse */ static isUtf8View(x: any): x is Utf8View { return 
x?.typeId === Type.Utf8View; }
     /** @nocollapse */ static isLargeUtf8(x: any): x is LargeUtf8 { return 
x?.typeId === Type.LargeUtf8; }
     /** @nocollapse */ static isBool(x: any): x is Bool { return x?.typeId === 
Type.Bool; }
     /** @nocollapse */ static isDecimal(x: any): x is Decimal { return 
x?.typeId === Type.Decimal; }
@@ -69,6 +71,8 @@ export abstract class DataType<TType extends Type = Type, 
TChildren extends Type
     /** @nocollapse */ static isInterval(x: any): x is Interval_ { return 
x?.typeId === Type.Interval; }
     /** @nocollapse */ static isDuration(x: any): x is Duration { return 
x?.typeId === Type.Duration; }
     /** @nocollapse */ static isList(x: any): x is List { return x?.typeId === 
Type.List; }
+    // TODO: Implement ListView type
+    //     /** @nocollapse */ static isListView(x: any): x is ListView { 
return x?.typeId === Type.ListView; }
     /** @nocollapse */ static isStruct(x: any): x is Struct { return x?.typeId 
=== Type.Struct; }
     /** @nocollapse */ static isUnion(x: any): x is Union_ { return x?.typeId 
=== Type.Union; }
     /** @nocollapse */ static isFixedSizeBinary(x: any): x is FixedSizeBinary 
{ return x?.typeId === Type.FixedSizeBinary; }
@@ -254,6 +258,30 @@ export class Binary extends DataType<Type.Binary> {
     })(Binary.prototype);
 }
 
+/** @ignore */
+export interface BinaryView extends DataType<Type.BinaryView> {
+    TArray: Uint8Array;
+    TValue: Uint8Array;
+    ArrayType: TypedArrayConstructor<Uint8Array>;
+}
+/** @ignore */
+export class BinaryView extends DataType<Type.BinaryView> {
+    public static readonly ELEMENT_WIDTH = 16;
+    public static readonly INLINE_CAPACITY = 12;
+    public static readonly LENGTH_OFFSET = 0;
+    public static readonly INLINE_OFFSET = 4;
+    public static readonly BUFFER_INDEX_OFFSET = 8;
+    public static readonly BUFFER_OFFSET_OFFSET = 12;
+    constructor() {
+        super(Type.BinaryView);
+    }
+    public toString() { return `BinaryView`; }
+    protected static [Symbol.toStringTag] = ((proto: BinaryView) => {
+        (<any>proto).ArrayType = Uint8Array;
+        return proto[Symbol.toStringTag] = 'BinaryView';
+    })(BinaryView.prototype);
+}
+
 /** @ignore */
 export interface LargeBinary extends DataType<Type.LargeBinary> { TArray: 
Uint8Array; TOffsetArray: BigInt64Array; TValue: Uint8Array; ArrayType: 
TypedArrayConstructor<Uint8Array>; OffsetArrayType: 
BigIntArrayConstructor<BigInt64Array> }
 /** @ignore */
@@ -269,6 +297,7 @@ export class LargeBinary extends DataType<Type.LargeBinary> 
{
     })(LargeBinary.prototype);
 }
 
+/** @ignore */
 /** @ignore */
 export interface Utf8 extends DataType<Type.Utf8> { TArray: Uint8Array; 
TOffsetArray: Int32Array; TValue: string; ArrayType: 
TypedArrayConstructor<Uint8Array>; OffsetArrayType: 
TypedArrayConstructor<Int32Array> }
 /** @ignore */
@@ -283,6 +312,26 @@ export class Utf8 extends DataType<Type.Utf8> {
     })(Utf8.prototype);
 }
 
+/** @ignore */
+export interface Utf8View extends DataType<Type.Utf8View> {
+    TArray: Uint8Array;
+    TValue: string;
+    ArrayType: TypedArrayConstructor<Uint8Array>;
+}
+/** @ignore */
+export class Utf8View extends DataType<Type.Utf8View> {
+    public static readonly ELEMENT_WIDTH = BinaryView.ELEMENT_WIDTH;
+    public static readonly INLINE_CAPACITY = BinaryView.INLINE_CAPACITY;
+    constructor() {
+        super(Type.Utf8View);
+    }
+    public toString() { return `Utf8View`; }
+    protected static [Symbol.toStringTag] = ((proto: Utf8View) => {
+        (<any>proto).ArrayType = Uint8Array;
+        return proto[Symbol.toStringTag] = 'Utf8View';
+    })(Utf8View.prototype);
+}
+
 /** @ignore */
 export interface LargeUtf8 extends DataType<Type.LargeUtf8> { TArray: 
Uint8Array; TOffsetArray: BigInt64Array; TValue: string; ArrayType: 
TypedArrayConstructor<Uint8Array>; OffsetArrayType: 
BigIntArrayConstructor<BigInt64Array> }
 /** @ignore */
@@ -298,6 +347,7 @@ export class LargeUtf8 extends DataType<Type.LargeUtf8> {
     })(LargeUtf8.prototype);
 }
 
+/** @ignore */
 /** @ignore */
 export interface Bool extends DataType<Type.Bool> { TArray: Uint8Array; 
TValue: boolean; ArrayType: TypedArrayConstructor<Uint8Array> }
 /** @ignore */
@@ -759,6 +809,8 @@ export function strideForType(type: DataType) {
         }
         // case Type.Int: return 1 + +((t as Int_).bitWidth > 32);
         // case Type.Time: return 1 + +((t as Time_).bitWidth > 32);
+        case Type.BinaryView:
+        case Type.Utf8View: return 16;
         case Type.FixedSizeList: return (t as FixedSizeList).listSize;
         case Type.FixedSizeBinary: return (t as FixedSizeBinary).byteWidth;
         default: return 1;
diff --git a/src/visitor.ts b/src/visitor.ts
index 977e0a4..a6d27a7 100644
--- a/src/visitor.ts
+++ b/src/visitor.ts
@@ -37,8 +37,10 @@ export abstract class Visitor {
     public visitFloat(_node: any, ..._args: any[]): any { return null; }
     public visitUtf8(_node: any, ..._args: any[]): any { return null; }
     public visitLargeUtf8(_node: any, ..._args: any[]): any { return null; }
+    public visitUtf8View(_node: any, ..._args: any[]): any { return null; }
     public visitBinary(_node: any, ..._args: any[]): any { return null; }
     public visitLargeBinary(_node: any, ..._args: any[]): any { return null; }
+    public visitBinaryView(_node: any, ..._args: any[]): any { return null; }
     public visitFixedSizeBinary(_node: any, ..._args: any[]): any { return 
null; }
     public visitDate(_node: any, ..._args: any[]): any { return null; }
     public visitTimestamp(_node: any, ..._args: any[]): any { return null; }
@@ -92,8 +94,10 @@ function getVisitFnByTypeId(visitor: Visitor, dtype: Type, 
throwIfNotFound = tru
         case Type.Float64: fn = visitor.visitFloat64 || visitor.visitFloat; 
break;
         case Type.Utf8: fn = visitor.visitUtf8; break;
         case Type.LargeUtf8: fn = visitor.visitLargeUtf8; break;
+        case Type.Utf8View: fn = visitor.visitUtf8View || visitor.visitUtf8; 
break;
         case Type.Binary: fn = visitor.visitBinary; break;
         case Type.LargeBinary: fn = visitor.visitLargeBinary; break;
+        case Type.BinaryView: fn = visitor.visitBinaryView || 
visitor.visitBinary; break;
         case Type.FixedSizeBinary: fn = visitor.visitFixedSizeBinary; break;
         case Type.Date: fn = visitor.visitDate; break;
         case Type.DateDay: fn = visitor.visitDateDay || visitor.visitDate; 
break;
@@ -157,8 +161,10 @@ function inferDType<T extends DataType>(type: T): Type {
             return Type.Float;
         case Type.Binary: return Type.Binary;
         case Type.LargeBinary: return Type.LargeBinary;
+        case Type.BinaryView: return Type.BinaryView;
         case Type.Utf8: return Type.Utf8;
         case Type.LargeUtf8: return Type.LargeUtf8;
+        case Type.Utf8View: return Type.Utf8View;
         case Type.Bool: return Type.Bool;
         case Type.Decimal: return Type.Decimal;
         case Type.Time:
diff --git a/src/visitor/builderctor.ts b/src/visitor/builderctor.ts
index 791576b..ca7669a 100644
--- a/src/visitor/builderctor.ts
+++ b/src/visitor/builderctor.ts
@@ -42,6 +42,8 @@ import { TimeBuilder, TimeSecondBuilder, 
TimeMillisecondBuilder, TimeMicrosecond
 import { UnionBuilder, DenseUnionBuilder, SparseUnionBuilder } from 
'../builder/union.js';
 import { Utf8Builder } from '../builder/utf8.js';
 import { LargeUtf8Builder } from '../builder/largeutf8.js';
+import { BinaryViewBuilder } from '../builder/binaryview.js';
+import { Utf8ViewBuilder } from '../builder/utf8view.js';
 
 /** @ignore */
 export interface GetBuilderCtor extends Visitor {
@@ -104,6 +106,8 @@ export class GetBuilderCtor extends Visitor {
     public visitDurationNanosecond() { return DurationNanosecondBuilder; }
     public visitFixedSizeList() { return FixedSizeListBuilder; }
     public visitMap() { return MapBuilder; }
+    public visitBinaryView() { return BinaryViewBuilder; }
+    public visitUtf8View() { return Utf8ViewBuilder; }
 }
 
 /** @ignore */
diff --git a/src/visitor/get.ts b/src/visitor/get.ts
index a5502dd..b914624 100644
--- a/src/visitor/get.ts
+++ b/src/visitor/get.ts
@@ -28,7 +28,7 @@ import { uint16ToFloat64 } from '../util/math.js';
 import { Type, UnionMode, Precision, DateUnit, TimeUnit, IntervalUnit } from 
'../enum.js';
 import {
     DataType, Dictionary,
-    Bool, Null, Utf8, LargeUtf8, Binary, LargeBinary, Decimal, 
FixedSizeBinary, List, FixedSizeList, Map_, Struct,
+    Bool, Null, Utf8, Utf8View, LargeUtf8, Binary, BinaryView, LargeBinary, 
Decimal, FixedSizeBinary, List, FixedSizeList, Map_, Struct,
     Float, Float16, Float32, Float64,
     Int, Uint8, Uint16, Uint32, Uint64, Int8, Int16, Int32, Int64,
     Date_, DateDay, DateMillisecond,
@@ -63,8 +63,10 @@ export interface GetVisitor extends Visitor {
     visitFloat64<T extends Float64>(data: Data<T>, index: number): T['TValue'] 
| null;
     visitUtf8<T extends Utf8>(data: Data<T>, index: number): T['TValue'] | 
null;
     visitLargeUtf8<T extends LargeUtf8>(data: Data<T>, index: number): 
T['TValue'] | null;
+    visitUtf8View<T extends Utf8View>(data: Data<T>, index: number): 
T['TValue'] | null;
     visitBinary<T extends Binary>(data: Data<T>, index: number): T['TValue'] | 
null;
     visitLargeBinary<T extends LargeBinary>(data: Data<T>, index: number): 
T['TValue'] | null;
+    visitBinaryView<T extends BinaryView>(data: Data<T>, index: number): 
T['TValue'] | null;
     visitFixedSizeBinary<T extends FixedSizeBinary>(data: Data<T>, index: 
number): T['TValue'] | null;
     visitDate<T extends Date_>(data: Data<T>, index: number): T['TValue'] | 
null;
     visitDateDay<T extends DateDay>(data: Data<T>, index: number): T['TValue'] 
| null;
@@ -109,6 +111,9 @@ function wrapGet<T extends DataType>(fn: (data: Data<T>, 
_1: any) => any) {
 
 /** @ignore */const epochDaysToMs = (data: Int32Array, index: number) => 
86400000 * data[index];
 
+const BINARY_VIEW_SIZE = 16;
+const BINARY_VIEW_INLINE_CAPACITY = 12;
+
 /** @ignore */
 const getNull = <T extends Null>(_data: Data<T>, _index: number): T['TValue'] 
=> null;
 /** @ignore */
@@ -149,10 +154,52 @@ const getFixedSizeBinary = <T extends FixedSizeBinary>({ 
stride, values }: Data<
 /** @ignore */
 const getBinary = <T extends Binary | LargeBinary>({ values, valueOffsets }: 
Data<T>, index: number): T['TValue'] => getVariableWidthBytes(values, 
valueOffsets, index);
 /** @ignore */
+const getBinaryViewBytes = (data: Data<BinaryView | Utf8View>, index: number): 
Uint8Array => {
+    const values = data.values as Uint8Array;
+    if (!values) {
+        throw new Error('BinaryView data is missing view buffer');
+    }
+    const viewOffset = index * BINARY_VIEW_SIZE;
+    const end = viewOffset + BINARY_VIEW_SIZE;
+    if (viewOffset < 0 || end > values.length) {
+        throw new Error(`BinaryView data buffer is too short: expected 
${BINARY_VIEW_SIZE} bytes, got ${Math.max(0, values.length - viewOffset)}`);
+    }
+    // Get the 16-byte view struct from the values array
+    const viewStruct = values.subarray(viewOffset, end);
+    if (viewStruct.length < BINARY_VIEW_SIZE) {
+        throw new Error(`BinaryView data buffer is too short: expected 
${BINARY_VIEW_SIZE} bytes, got ${viewStruct.length}`);
+    }
+    const view = new DataView(values.buffer, viewStruct.byteOffset, 
BINARY_VIEW_SIZE);
+    const size = view.getInt32(0, true);
+    if (size <= 0) {
+        return new Uint8Array(0);
+    }
+    if (size <= BINARY_VIEW_INLINE_CAPACITY) {
+        // Inline data is in bytes 4-15 of the view struct
+        return viewStruct.subarray(4, 4 + size);
+    }
+    const bufferIndex = view.getInt32(8, true);
+    const offset = view.getInt32(12, true);
+    const variadicBuffer = data.variadicBuffers?.[bufferIndex];
+    if (!variadicBuffer) {
+        throw new Error(`BinaryView variadic buffer ${bufferIndex} is 
missing`);
+    }
+    return variadicBuffer.subarray(offset, offset + size);
+};
+/** @ignore */
+const getBinaryViewValue = <T extends BinaryView>(data: Data<T>, index: 
number): T['TValue'] => {
+    return getBinaryViewBytes(data, index) as T['TValue'];
+};
+/** @ignore */
 const getUtf8 = <T extends Utf8 | LargeUtf8>({ values, valueOffsets }: 
Data<T>, index: number): T['TValue'] => {
     const bytes = getVariableWidthBytes(values, valueOffsets, index);
     return bytes !== null ? decodeUtf8(bytes) : null as any;
 };
+/** @ignore */
+const getUtf8ViewValue = <T extends Utf8View>(data: Data<T>, index: number): 
T['TValue'] => {
+    const bytes = getBinaryViewBytes(data, index);
+    return decodeUtf8(bytes);
+};
 
 /* istanbul ignore next */
 /** @ignore */
@@ -332,8 +379,10 @@ GetVisitor.prototype.visitFloat32 = wrapGet(getNumeric);
 GetVisitor.prototype.visitFloat64 = wrapGet(getNumeric);
 GetVisitor.prototype.visitUtf8 = wrapGet(getUtf8);
 GetVisitor.prototype.visitLargeUtf8 = wrapGet(getUtf8);
+GetVisitor.prototype.visitUtf8View = wrapGet(getUtf8ViewValue);
 GetVisitor.prototype.visitBinary = wrapGet(getBinary);
 GetVisitor.prototype.visitLargeBinary = wrapGet(getBinary);
+GetVisitor.prototype.visitBinaryView = wrapGet(getBinaryViewValue);
 GetVisitor.prototype.visitFixedSizeBinary = wrapGet(getFixedSizeBinary);
 GetVisitor.prototype.visitDate = wrapGet(getDate);
 GetVisitor.prototype.visitDateDay = wrapGet(getDateDay);
diff --git a/src/visitor/indexof.ts b/src/visitor/indexof.ts
index 3a4d117..6881f99 100644
--- a/src/visitor/indexof.ts
+++ b/src/visitor/indexof.ts
@@ -24,7 +24,7 @@ import { getBool, BitIterator } from '../util/bit.js';
 import { createElementComparator } from '../util/vector.js';
 import {
     DataType, Dictionary,
-    Bool, Null, Utf8, LargeUtf8, Binary, LargeBinary, Decimal, 
FixedSizeBinary, List, FixedSizeList, Map_, Struct,
+    Bool, Null, Utf8, Utf8View, LargeUtf8, Binary, BinaryView, LargeBinary, 
Decimal, FixedSizeBinary, List, FixedSizeList, Map_, Struct,
     Float, Float16, Float32, Float64,
     Int, Uint8, Uint16, Uint32, Uint64, Int8, Int16, Int32, Int64,
     Date_, DateDay, DateMillisecond,
@@ -59,8 +59,10 @@ export interface IndexOfVisitor extends Visitor {
     visitFloat64<T extends Float64>(data: Data<T>, value: T['TValue'] | null, 
index?: number): number;
     visitUtf8<T extends Utf8>(data: Data<T>, value: T['TValue'] | null, 
index?: number): number;
     visitLargeUtf8<T extends LargeUtf8>(data: Data<T>, value: T['TValue'] | 
null, index?: number): number;
+    visitUtf8View<T extends Utf8View>(data: Data<T>, value: T['TValue'] | 
null, index?: number): number;
     visitBinary<T extends Binary>(data: Data<T>, value: T['TValue'] | null, 
index?: number): number;
     visitLargeBinary<T extends LargeBinary>(data: Data<T>, value: T['TValue'] 
| null, index?: number): number;
+    visitBinaryView<T extends BinaryView>(data: Data<T>, value: T['TValue'] | 
null, index?: number): number;
     visitFixedSizeBinary<T extends FixedSizeBinary>(data: Data<T>, value: 
T['TValue'] | null, index?: number): number;
     visitDate<T extends Date_>(data: Data<T>, value: T['TValue'] | null, 
index?: number): number;
     visitDateDay<T extends DateDay>(data: Data<T>, value: T['TValue'] | null, 
index?: number): number;
@@ -177,8 +179,10 @@ IndexOfVisitor.prototype.visitFloat32 = indexOfValue;
 IndexOfVisitor.prototype.visitFloat64 = indexOfValue;
 IndexOfVisitor.prototype.visitUtf8 = indexOfValue;
 IndexOfVisitor.prototype.visitLargeUtf8 = indexOfValue;
+IndexOfVisitor.prototype.visitUtf8View = indexOfValue;
 IndexOfVisitor.prototype.visitBinary = indexOfValue;
 IndexOfVisitor.prototype.visitLargeBinary = indexOfValue;
+IndexOfVisitor.prototype.visitBinaryView = indexOfValue;
 IndexOfVisitor.prototype.visitFixedSizeBinary = indexOfValue;
 IndexOfVisitor.prototype.visitDate = indexOfValue;
 IndexOfVisitor.prototype.visitDateDay = indexOfValue;
diff --git a/src/visitor/iterator.ts b/src/visitor/iterator.ts
index 9f2844b..ef54504 100644
--- a/src/visitor/iterator.ts
+++ b/src/visitor/iterator.ts
@@ -21,7 +21,7 @@ import { Type, Precision } from '../enum.js';
 import { TypeToDataType } from '../interfaces.js';
 import {
     DataType, Dictionary,
-    Bool, Null, Utf8, LargeUtf8, Binary, LargeBinary, Decimal, 
FixedSizeBinary, List, FixedSizeList, Map_, Struct,
+    Bool, Null, Utf8, Utf8View, LargeUtf8, Binary, BinaryView, LargeBinary, 
Decimal, FixedSizeBinary, List, FixedSizeList, Map_, Struct,
     Float, Float16, Float32, Float64,
     Int, Uint8, Uint16, Uint32, Uint64, Int8, Int16, Int32, Int64,
     Date_, DateDay, DateMillisecond,
@@ -57,8 +57,10 @@ export interface IteratorVisitor extends Visitor {
     visitFloat64<T extends Float64>(vector: Vector<T>): 
IterableIterator<T['TValue'] | null>;
     visitUtf8<T extends Utf8>(vector: Vector<T>): IterableIterator<T['TValue'] 
| null>;
     visitLargeUtf8<T extends LargeUtf8>(vector: Vector<T>): 
IterableIterator<T['TValue'] | null>;
+    visitUtf8View<T extends Utf8View>(vector: Vector<T>): 
IterableIterator<T['TValue'] | null>;
     visitBinary<T extends Binary>(vector: Vector<T>): 
IterableIterator<T['TValue'] | null>;
     visitLargeBinary<T extends LargeBinary>(vector: Vector<T>): 
IterableIterator<T['TValue'] | null>;
+    visitBinaryView<T extends BinaryView>(vector: Vector<T>): 
IterableIterator<T['TValue'] | null>;
     visitFixedSizeBinary<T extends FixedSizeBinary>(vector: Vector<T>): 
IterableIterator<T['TValue'] | null>;
     visitDate<T extends Date_>(vector: Vector<T>): 
IterableIterator<T['TValue'] | null>;
     visitDateDay<T extends DateDay>(vector: Vector<T>): 
IterableIterator<T['TValue'] | null>;
@@ -164,8 +166,10 @@ IteratorVisitor.prototype.visitFloat32 = vectorIterator;
 IteratorVisitor.prototype.visitFloat64 = vectorIterator;
 IteratorVisitor.prototype.visitUtf8 = vectorIterator;
 IteratorVisitor.prototype.visitLargeUtf8 = vectorIterator;
+IteratorVisitor.prototype.visitUtf8View = vectorIterator;
 IteratorVisitor.prototype.visitBinary = vectorIterator;
 IteratorVisitor.prototype.visitLargeBinary = vectorIterator;
+IteratorVisitor.prototype.visitBinaryView = vectorIterator;
 IteratorVisitor.prototype.visitFixedSizeBinary = vectorIterator;
 IteratorVisitor.prototype.visitDate = vectorIterator;
 IteratorVisitor.prototype.visitDateDay = vectorIterator;
diff --git a/src/visitor/jsontypeassembler.ts b/src/visitor/jsontypeassembler.ts
index 823b1de..cf11003 100644
--- a/src/visitor/jsontypeassembler.ts
+++ b/src/visitor/jsontypeassembler.ts
@@ -45,6 +45,9 @@ export class JSONTypeAssembler extends Visitor {
     public visitLargeBinary<T extends type.LargeBinary>({ typeId }: T) {
         return { 'name': ArrowType[typeId].toLowerCase() };
     }
+    public visitBinaryView<T extends type.BinaryView>({ typeId }: T) {
+        return { 'name': ArrowType[typeId].toLowerCase() };
+    }
     public visitBool<T extends type.Bool>({ typeId }: T) {
         return { 'name': ArrowType[typeId].toLowerCase() };
     }
@@ -54,6 +57,9 @@ export class JSONTypeAssembler extends Visitor {
     public visitLargeUtf8<T extends type.LargeUtf8>({ typeId }: T) {
         return { 'name': ArrowType[typeId].toLowerCase() };
     }
+    public visitUtf8View<T extends type.Utf8View>({ typeId }: T) {
+        return { 'name': ArrowType[typeId].toLowerCase() };
+    }
     public visitDecimal<T extends type.Decimal>({ typeId, scale, precision, 
bitWidth }: T) {
         return { 'name': ArrowType[typeId].toLowerCase(), 'scale': scale, 
'precision': precision, 'bitWidth': bitWidth };
     }
diff --git a/src/visitor/jsonvectorassembler.ts 
b/src/visitor/jsonvectorassembler.ts
index 6841b39..2f4973a 100644
--- a/src/visitor/jsonvectorassembler.ts
+++ b/src/visitor/jsonvectorassembler.ts
@@ -28,7 +28,7 @@ import { toIntervalDayTimeObjects, 
toIntervalMonthDayNanoObjects } from '../util
 import {
     DataType,
     Float, Int, Date_, Interval, Time, Timestamp, Union, Duration,
-    Bool, Null, Utf8, LargeUtf8, Binary, LargeBinary, Decimal, 
FixedSizeBinary, List, FixedSizeList, Map_, Struct, IntArray,
+    Bool, Null, Utf8, LargeUtf8, Binary, LargeBinary, BinaryView, Utf8View, 
Decimal, FixedSizeBinary, List, FixedSizeList, Map_, Struct, IntArray,
 } from '../type.js';
 
 /** @ignore */
@@ -46,6 +46,8 @@ export interface JSONVectorAssembler extends Visitor {
     visitLargeUtf8<T extends LargeUtf8>(data: Data<T>): { DATA: string[]; 
OFFSET: string[] };
     visitBinary<T extends Binary>(data: Data<T>): { DATA: string[]; OFFSET: 
number[] };
     visitLargeBinary<T extends LargeBinary>(data: Data<T>): { DATA: string[]; 
OFFSET: string[] };
+    visitBinaryView<T extends BinaryView>(data: Data<T>): { VIEWS: any[]; 
VARIADIC_DATA_BUFFERS: string[] };
+    visitUtf8View<T extends Utf8View>(data: Data<T>): { VIEWS: any[]; 
VARIADIC_DATA_BUFFERS: string[] };
     visitFixedSizeBinary<T extends FixedSizeBinary>(data: Data<T>): { DATA: 
string[] };
     visitDate<T extends Date_>(data: Data<T>): { DATA: number[] };
     visitTimestamp<T extends Timestamp>(data: Data<T>): { DATA: string[] };
@@ -112,6 +114,15 @@ export class JSONVectorAssembler extends Visitor {
     public visitLargeBinary<T extends LargeBinary>(data: Data<T>) {
         return { 'DATA': [...binaryToString(new Vector([data]))], 'OFFSET': 
[...bigNumsToStrings(data.valueOffsets, 2)] };
     }
+    public visitBinaryView<T extends BinaryView>(data: Data<T>) {
+        return binaryViewDataToJSON(data, (bytes) => Array.from(bytes)
+                .map(b => ('0' + (b & 0xFF).toString(16)).slice(-2))
+                .join('')
+                .toUpperCase());
+    }
+    public visitUtf8View<T extends Utf8View>(data: Data<T>) {
+        return binaryViewDataToJSON(data, (bytes) => Array.from(bytes).map(b 
=> String.fromCodePoint(b)).join(''));
+    }
     public visitFixedSizeBinary<T extends FixedSizeBinary>(data: Data<T>) {
         return { 'DATA': [...binaryToString(new Vector([data]))] };
     }
@@ -195,3 +206,46 @@ function* bigNumsToStrings(values: BigUint64Array | 
BigInt64Array | Uint32Array
         yield `${BN.new(u32s.subarray((i + 0) * stride, (i + 1) * stride), 
false)}`;
     }
 }
+
+/** @ignore */
+function binaryViewDataToJSON(data: Data<BinaryView> | Data<Utf8View>, 
formatInlined: (bytes: Uint8Array) => string) {
+    const INLINE_SIZE = 12;
+    const viewsData = data.values;
+    const dataView = new DataView(viewsData.buffer, viewsData.byteOffset, 
viewsData.byteLength);
+    const numViews = viewsData.byteLength / 16;
+    const bytesToHex = (bytes: Uint8Array) =>
+        Array.from(bytes)
+            .map(b => ('0' + (b & 0xFF).toString(16)).slice(-2))
+            .join('')
+            .toUpperCase();
+    const parsedViews = Array.from({ length: numViews }, (_, i) => {
+      const offset = i * 16;
+      const size = dataView.getInt32(offset, true);
+      return [offset, size];
+    }).map(([offset, size]) => (size > INLINE_SIZE) ? {
+      'SIZE': size,
+      'PREFIX_HEX': bytesToHex(viewsData.subarray(offset + 4, offset + 8)),
+      'BUFFER_INDEX': dataView.getInt32(offset + 8, true),
+      'OFFSET': dataView.getInt32(offset + 12, true)
+    } : {
+      'SIZE': size,
+      'INLINED': formatInlined(viewsData.subarray(offset + 4, offset + 4 + 
size))
+    });
+    const uniqueBufferIndices = [...new Set(
+        parsedViews
+            .map(v => v['BUFFER_INDEX'])
+            .filter((idx): idx is number => idx !== undefined)
+    )];
+    const variadicBuffers = uniqueBufferIndices.map(bufferIndex =>
+        bytesToHex(data.variadicBuffers[bufferIndex])
+    );
+    const bufferIndexMap = new Map(
+        uniqueBufferIndices.map((bufferIndex, outputIndex) => [bufferIndex, 
outputIndex])
+    );
+    // Remap buffer indices in views
+    const views = parsedViews.map(v => v['BUFFER_INDEX'] !== undefined
+      ? { ...v, 'BUFFER_INDEX': bufferIndexMap.get(v['BUFFER_INDEX']) }
+      : v
+    );
+    return { 'VIEWS': views, 'VARIADIC_DATA_BUFFERS': variadicBuffers };
+}
diff --git a/src/visitor/set.ts b/src/visitor/set.ts
index 4bf632b..9050193 100644
--- a/src/visitor/set.ts
+++ b/src/visitor/set.ts
@@ -26,7 +26,7 @@ import { float64ToUint16 } from '../util/math.js';
 import { Type, UnionMode, Precision, DateUnit, TimeUnit, IntervalUnit } from 
'../enum.js';
 import {
     DataType, Dictionary,
-    Bool, Null, Utf8, LargeUtf8, Binary, LargeBinary, Decimal, 
FixedSizeBinary, List, FixedSizeList, Map_, Struct,
+    Bool, Null, Utf8, Utf8View, LargeUtf8, Binary, BinaryView, LargeBinary, 
Decimal, FixedSizeBinary, List, FixedSizeList, Map_, Struct,
     Float, Float16, Float32, Float64,
     Int, Uint8, Uint16, Uint32, Uint64, Int8, Int16, Int32, Int64,
     Date_, DateDay, DateMillisecond,
@@ -61,8 +61,10 @@ export interface SetVisitor extends Visitor {
     visitFloat64<T extends Float64>(data: Data<T>, index: number, value: 
T['TValue']): void;
     visitUtf8<T extends Utf8>(data: Data<T>, index: number, value: 
T['TValue']): void;
     visitLargeUtf8<T extends LargeUtf8>(data: Data<T>, index: number, value: 
T['TValue']): void;
+    visitUtf8View<T extends Utf8View>(data: Data<T>, index: number, value: 
T['TValue']): void;
     visitBinary<T extends Binary>(data: Data<T>, index: number, value: 
T['TValue']): void;
     visitLargeBinary<T extends LargeBinary>(data: Data<T>, index: number, 
value: T['TValue']): void;
+    visitBinaryView<T extends BinaryView>(data: Data<T>, index: number, value: 
T['TValue']): void;
     visitFixedSizeBinary<T extends FixedSizeBinary>(data: Data<T>, index: 
number, value: T['TValue']): void;
     visitDate<T extends Date_>(data: Data<T>, index: number, value: 
T['TValue']): void;
     visitDateDay<T extends DateDay>(data: Data<T>, index: number, value: 
T['TValue']): void;
@@ -155,7 +157,63 @@ export const setFixedSizeBinary = <T extends 
FixedSizeBinary>({ stride, values }
 /** @ignore */
 const setBinary = <T extends Binary | LargeBinary>({ values, valueOffsets }: 
Data<T>, index: number, value: T['TValue']) => setVariableWidthBytes(values, 
valueOffsets, index, value);
 /** @ignore */
+const ensureWritableVariadicBuffers = (data: Data<BinaryView | Utf8View>): 
Uint8Array[] => {
+    let buffers = data.variadicBuffers as unknown as Uint8Array[];
+    if (!Array.isArray(buffers) || Object.isFrozen(buffers)) {
+        buffers = Array.from(buffers) as Uint8Array[];
+        (data as any).variadicBuffers = buffers;
+    }
+    return buffers;
+};
+/** @ignore */
+const setBinaryViewBytes = (data: Data<BinaryView | Utf8View>, index: number, 
bytes: Uint8Array) => {
+    const views = data.values as Uint8Array | undefined;
+    if (!views) {
+        throw new Error('BinaryView data is missing view buffer');
+    }
+    const elementWidth = BinaryView.ELEMENT_WIDTH;
+    const viewOffset = index * elementWidth;
+    const end = viewOffset + elementWidth;
+    if (viewOffset < 0 || end > views.length) {
+        throw new RangeError(`BinaryView index ${index} out of bounds`);
+    }
+
+    views.fill(0, viewOffset, end);
+
+    const view = new DataView(views.buffer, views.byteOffset + viewOffset, 
elementWidth);
+    const length = bytes.length;
+    view.setInt32(BinaryView.LENGTH_OFFSET, length, true);
+
+    if (length <= BinaryView.INLINE_CAPACITY) {
+        views.set(bytes, viewOffset + BinaryView.INLINE_OFFSET);
+        return;
+    }
+
+    const prefix =
+        (bytes[0] ?? 0) |
+        ((bytes[1] ?? 0) << 8) |
+        ((bytes[2] ?? 0) << 16) |
+        ((bytes[3] ?? 0) << 24);
+    view.setUint32(BinaryView.INLINE_OFFSET, prefix >>> 0, true);
+
+    const buffers = ensureWritableVariadicBuffers(data);
+    const copy = bytes.slice();
+    const bufferIndex = buffers.push(copy) - 1;
+    view.setInt32(BinaryView.BUFFER_INDEX_OFFSET, bufferIndex, true);
+    view.setInt32(BinaryView.BUFFER_OFFSET_OFFSET, 0, true);
+};
+/** @ignore */
+const setBinaryView = <T extends BinaryView>(data: Data<T>, index: number, 
value: T['TValue']) => {
+    const bytes = value instanceof Uint8Array ? value : new Uint8Array(value);
+    setBinaryViewBytes(data as unknown as Data<BinaryView | Utf8View>, index, 
bytes);
+};
+/** @ignore */
 const setUtf8 = <T extends Utf8 | LargeUtf8>({ values, valueOffsets }: 
Data<T>, index: number, value: T['TValue']) => setVariableWidthBytes(values, 
valueOffsets, index, encodeUtf8(value));
+/** @ignore */
+const setUtf8View = <T extends Utf8View>(data: Data<T>, index: number, value: 
T['TValue']) => {
+    const bytes = encodeUtf8(value);
+    setBinaryViewBytes(data as unknown as Data<BinaryView | Utf8View>, index, 
bytes);
+};
 
 /* istanbul ignore next */
 export const setDate = <T extends Date_>(data: Data<T>, index: number, value: 
T['TValue']): void => {
@@ -359,8 +417,10 @@ SetVisitor.prototype.visitFloat32 = wrapSet(setFloat);
 SetVisitor.prototype.visitFloat64 = wrapSet(setFloat);
 SetVisitor.prototype.visitUtf8 = wrapSet(setUtf8);
 SetVisitor.prototype.visitLargeUtf8 = wrapSet(setUtf8);
+SetVisitor.prototype.visitUtf8View = wrapSet(setUtf8View);
 SetVisitor.prototype.visitBinary = wrapSet(setBinary);
 SetVisitor.prototype.visitLargeBinary = wrapSet(setBinary);
+SetVisitor.prototype.visitBinaryView = wrapSet(setBinaryView);
 SetVisitor.prototype.visitFixedSizeBinary = wrapSet(setFixedSizeBinary);
 SetVisitor.prototype.visitDate = wrapSet(setDate);
 SetVisitor.prototype.visitDateDay = wrapSet(setDateDay);
diff --git a/src/visitor/typeassembler.ts b/src/visitor/typeassembler.ts
index 169f362..d997f6c 100644
--- a/src/visitor/typeassembler.ts
+++ b/src/visitor/typeassembler.ts
@@ -25,9 +25,11 @@ import { Null } from '../fb/null.js';
 import { Int } from '../fb/int.js';
 import { FloatingPoint } from '../fb/floating-point.js';
 import { Binary } from '../fb/binary.js';
+import { BinaryView } from '../fb/binary-view.js';
 import { LargeBinary } from '../fb/large-binary.js';
 import { Bool } from '../fb/bool.js';
 import { Utf8 } from '../fb/utf8.js';
+import { Utf8View } from '../fb/utf8-view.js';
 import { LargeUtf8 } from '../fb/large-utf8.js';
 import { Decimal } from '../fb/decimal.js';
 import { Date } from '../fb/date.js';
@@ -72,6 +74,10 @@ export class TypeAssembler extends Visitor {
         Binary.startBinary(b);
         return Binary.endBinary(b);
     }
+    public visitBinaryView<T extends type.BinaryView>(_node: T, b: Builder) {
+        BinaryView.startBinaryView(b);
+        return BinaryView.endBinaryView(b);
+    }
     public visitLargeBinary<T extends type.LargeBinary>(_node: T, b: Builder) {
         LargeBinary.startLargeBinary(b);
         return LargeBinary.endLargeBinary(b);
@@ -84,6 +90,10 @@ export class TypeAssembler extends Visitor {
         Utf8.startUtf8(b);
         return Utf8.endUtf8(b);
     }
+    public visitUtf8View<T extends type.Utf8View>(_node: T, b: Builder) {
+        Utf8View.startUtf8View(b);
+        return Utf8View.endUtf8View(b);
+    }
     public visitLargeUtf8<T extends type.LargeUtf8>(_node: T, b: Builder) {
         LargeUtf8.startLargeUtf8(b);
         return LargeUtf8.endLargeUtf8(b);
diff --git a/src/visitor/typecomparator.ts b/src/visitor/typecomparator.ts
index 65413cc..5c1d60a 100644
--- a/src/visitor/typecomparator.ts
+++ b/src/visitor/typecomparator.ts
@@ -21,7 +21,7 @@ import { Visitor } from '../visitor.js';
 import { Schema, Field } from '../schema.js';
 import {
     DataType, TypeMap, Dictionary,
-    Bool, Null, Utf8, LargeUtf8, Binary, LargeBinary, Decimal, 
FixedSizeBinary, List, FixedSizeList, Map_, Struct,
+    Bool, Null, Utf8, Utf8View, LargeUtf8, Binary, BinaryView, LargeBinary, 
Decimal, FixedSizeBinary, List, FixedSizeList, Map_, Struct,
     Float, Float16, Float32, Float64,
     Int, Uint8, Uint16, Uint32, Uint64, Int8, Int16, Int32, Int64,
     Date_, DateDay, DateMillisecond,
@@ -55,8 +55,10 @@ export interface TypeComparator extends Visitor {
     visitFloat64<T extends Float64>(type: T, other?: DataType | null): other 
is T;
     visitUtf8<T extends Utf8>(type: T, other?: DataType | null): other is T;
     visitLargeUtf8<T extends LargeUtf8>(type: T, other?: DataType | null): 
other is T;
+    visitUtf8View<T extends Utf8View>(type: T, other?: DataType | null): other 
is T;
     visitBinary<T extends Binary>(type: T, other?: DataType | null): other is 
T;
     visitLargeBinary<T extends LargeBinary>(type: T, other?: DataType | null): 
other is T;
+    visitBinaryView<T extends BinaryView>(type: T, other?: DataType | null): 
other is T;
     visitFixedSizeBinary<T extends FixedSizeBinary>(type: T, other?: DataType 
| null): other is T;
     visitDate<T extends Date_>(type: T, other?: DataType | null): other is T;
     visitDateDay<T extends DateDay>(type: T, other?: DataType | null): other 
is T;
@@ -254,8 +256,10 @@ TypeComparator.prototype.visitFloat32 = compareFloat;
 TypeComparator.prototype.visitFloat64 = compareFloat;
 TypeComparator.prototype.visitUtf8 = compareAny;
 TypeComparator.prototype.visitLargeUtf8 = compareAny;
+TypeComparator.prototype.visitUtf8View = compareAny;
 TypeComparator.prototype.visitBinary = compareAny;
 TypeComparator.prototype.visitLargeBinary = compareAny;
+TypeComparator.prototype.visitBinaryView = compareAny;
 TypeComparator.prototype.visitFixedSizeBinary = compareFixedSizeBinary;
 TypeComparator.prototype.visitDate = compareDate;
 TypeComparator.prototype.visitDateDay = compareDate;
diff --git a/src/visitor/vectorassembler.ts b/src/visitor/vectorassembler.ts
index 7dc3695..2ac6f8f 100644
--- a/src/visitor/vectorassembler.ts
+++ b/src/visitor/vectorassembler.ts
@@ -27,7 +27,7 @@ import { BufferRegion, FieldNode } from 
'../ipc/metadata/message.js';
 import {
     DataType, Dictionary,
     Float, Int, Date_, Interval, Time, Timestamp, Union, Duration,
-    Bool, Null, Utf8, LargeUtf8, Binary, LargeBinary, Decimal, 
FixedSizeBinary, List, FixedSizeList, Map_, Struct,
+    Bool, Null, Utf8, Utf8View, LargeUtf8, Binary, BinaryView, LargeBinary, 
Decimal, FixedSizeBinary, List, FixedSizeList, Map_, Struct,
 } from '../type.js';
 import { bigIntToNumber } from '../util/bigint.js';
 
@@ -115,11 +115,13 @@ export class VectorAssembler extends Visitor {
     public get buffers() { return this._buffers; }
     public get byteLength() { return this._byteLength; }
     public get bufferRegions() { return this._bufferRegions; }
+    public get variadicBufferCounts() { return this._variadicBufferCounts; }
 
     protected _byteLength = 0;
     protected _nodes: FieldNode[] = [];
     protected _buffers: ArrayBufferView[] = [];
     protected _bufferRegions: BufferRegion[] = [];
+    protected _variadicBufferCounts: number[] = [];
 }
 
 /** @ignore */
@@ -215,6 +217,22 @@ function assembleFlatListVector<T extends Utf8 | LargeUtf8 
| Binary | LargeBinar
     return this;
 }
 
+/** @ignore */
+function assembleBinaryViewVector<T extends BinaryView | Utf8View>(this: 
VectorAssembler, data: Data<T>) {
+    const { offset, length, stride, values, variadicBuffers = [] } = data;
+    if (!values) {
+        throw new Error('BinaryView data is missing view buffer');
+    }
+    const start = offset * stride;
+    const end = start + length * stride;
+    addBuffer.call(this, values.subarray(start, end));
+    for (const buffer of variadicBuffers) {
+        addBuffer.call(this, buffer);
+    }
+    this._variadicBufferCounts.push(variadicBuffers.length);
+    return this;
+}
+
 /** @ignore */
 function assembleListVector<T extends Map_ | List | FixedSizeList>(this: 
VectorAssembler, data: Data<T>) {
     const { length, valueOffsets } = data;
@@ -239,8 +257,10 @@ VectorAssembler.prototype.visitInt = assembleFlatVector;
 VectorAssembler.prototype.visitFloat = assembleFlatVector;
 VectorAssembler.prototype.visitUtf8 = assembleFlatListVector;
 VectorAssembler.prototype.visitLargeUtf8 = assembleFlatListVector;
+VectorAssembler.prototype.visitUtf8View = assembleBinaryViewVector;
 VectorAssembler.prototype.visitBinary = assembleFlatListVector;
 VectorAssembler.prototype.visitLargeBinary = assembleFlatListVector;
+VectorAssembler.prototype.visitBinaryView = assembleBinaryViewVector;
 VectorAssembler.prototype.visitFixedSizeBinary = assembleFlatVector;
 VectorAssembler.prototype.visitDate = assembleFlatVector;
 VectorAssembler.prototype.visitTimestamp = assembleFlatVector;
diff --git a/src/visitor/vectorloader.ts b/src/visitor/vectorloader.ts
index 7c82e7a..a6892b2 100644
--- a/src/visitor/vectorloader.ts
+++ b/src/visitor/vectorloader.ts
@@ -44,13 +44,16 @@ export class VectorLoader extends Visitor {
     protected buffersIndex = -1;
     private dictionaries: Map<number, Vector<any>>;
     private readonly metadataVersion: MetadataVersion;
-    constructor(bytes: Uint8Array, nodes: FieldNode[], buffers: 
BufferRegion[], dictionaries: Map<number, Vector<any>>, metadataVersion = 
MetadataVersion.V5) {
+    private variadicBufferCounts: number[];
+    private variadicBufferIndex = -1;
+    constructor(bytes: Uint8Array, nodes: FieldNode[], buffers: 
BufferRegion[], dictionaries: Map<number, Vector<any>>, metadataVersion = 
MetadataVersion.V5, variadicBufferCounts: number[] = []) {
         super();
         this.bytes = bytes;
         this.nodes = nodes;
         this.buffers = buffers;
         this.dictionaries = dictionaries;
         this.metadataVersion = metadataVersion;
+        this.variadicBufferCounts = variadicBufferCounts;
     }
 
     public visit<T extends DataType>(node: Field<T> | T): Data<T> {
@@ -75,12 +78,38 @@ export class VectorLoader extends Visitor {
     public visitLargeUtf8<T extends type.LargeUtf8>(type: T, { length, 
nullCount } = this.nextFieldNode()) {
         return makeData({ type, length, nullCount, nullBitmap: 
this.readNullBitmap(type, nullCount), valueOffsets: this.readOffsets(type), 
data: this.readData(type) });
     }
+    public visitUtf8View<T extends type.Utf8View>(type: T, { length, nullCount 
} = this.nextFieldNode()) {
+        const nullBitmap = this.readNullBitmap(type, nullCount);
+        const views = this.readData(type);
+        const variadicBuffers = 
this.readVariadicBuffers(this.nextVariadicBufferCount());
+        return makeData({
+            type,
+            length,
+            nullCount,
+            nullBitmap,
+            ['views']: views,
+            ['variadicBuffers']: variadicBuffers
+        });
+    }
     public visitBinary<T extends type.Binary>(type: T, { length, nullCount } = 
this.nextFieldNode()) {
         return makeData({ type, length, nullCount, nullBitmap: 
this.readNullBitmap(type, nullCount), valueOffsets: this.readOffsets(type), 
data: this.readData(type) });
     }
     public visitLargeBinary<T extends type.LargeBinary>(type: T, { length, 
nullCount } = this.nextFieldNode()) {
         return makeData({ type, length, nullCount, nullBitmap: 
this.readNullBitmap(type, nullCount), valueOffsets: this.readOffsets(type), 
data: this.readData(type) });
     }
+    public visitBinaryView<T extends type.BinaryView>(type: T, { length, 
nullCount } = this.nextFieldNode()) {
+        const nullBitmap = this.readNullBitmap(type, nullCount);
+        const views = this.readData(type);
+        const variadicBuffers = 
this.readVariadicBuffers(this.nextVariadicBufferCount());
+        return makeData({
+            type,
+            length,
+            nullCount,
+            nullBitmap,
+            ['views']: views,
+            ['variadicBuffers']: variadicBuffers
+        });
+    }
     public visitFixedSizeBinary<T extends type.FixedSizeBinary>(type: T, { 
length, nullCount } = this.nextFieldNode()) {
         return makeData({ type, length, nullCount, nullBitmap: 
this.readNullBitmap(type, nullCount), data: this.readData(type) });
     }
@@ -142,6 +171,12 @@ export class VectorLoader extends Visitor {
     protected readData<T extends DataType>(_type: T, { length, offset } = 
this.nextBufferRange()) {
         return this.bytes.subarray(offset, offset + length);
     }
+    protected readVariadicBuffers(length: number) {
+        return Array.from({ length }, () => this.readData(null as any));
+    }
+    protected nextVariadicBufferCount() {
+        return this.variadicBufferCounts[++this.variadicBufferIndex] ?? 0;
+    }
     protected readDictionary<T extends type.Dictionary>(type: T): 
Vector<T['dictionary']> {
         return this.dictionaries.get(type.id)!;
     }
@@ -150,8 +185,8 @@ export class VectorLoader extends Visitor {
 /** @ignore */
 export class JSONVectorLoader extends VectorLoader {
     private sources: any[][];
-    constructor(sources: any[][], nodes: FieldNode[], buffers: BufferRegion[], 
dictionaries: Map<number, Vector<any>>, metadataVersion: MetadataVersion) {
-        super(new Uint8Array(0), nodes, buffers, dictionaries, 
metadataVersion);
+    constructor(sources: any[][], nodes: FieldNode[], buffers: BufferRegion[], 
dictionaries: Map<number, Vector<any>>, metadataVersion: MetadataVersion, 
variadicBufferCounts: number[] = []) {
+        super(new Uint8Array(0), nodes, buffers, dictionaries, 
metadataVersion, variadicBufferCounts);
         this.sources = sources;
     }
     protected readNullBitmap<T extends DataType>(_type: T, nullCount: number, 
{ offset } = this.nextBufferRange()) {
@@ -175,6 +210,10 @@ export class JSONVectorLoader extends VectorLoader {
             return toArrayBufferView(Uint8Array, 
Int128.convertArray(sources[offset] as string[]));
         } else if (DataType.isBinary(type) || DataType.isLargeBinary(type) || 
DataType.isFixedSizeBinary(type)) {
             return binaryDataFromJSON(sources[offset] as string[]);
+        } else if (DataType.isBinaryView(type)) {
+            return binaryViewDataFromJSON(sources[offset] as any[]);
+        } else if (DataType.isUtf8View(type)) {
+            return utf8ViewDataFromJSON(sources[offset] as any[]);
         } else if (DataType.isBool(type)) {
             return packBools(sources[offset] as number[]);
         } else if (DataType.isUtf8(type) || DataType.isLargeUtf8(type)) {
@@ -191,25 +230,107 @@ export class JSONVectorLoader extends VectorLoader {
         }
         return toArrayBufferView(Uint8Array, toArrayBufferView(type.ArrayType, 
sources[offset].map((x) => +x)));
     }
+    protected readVariadicBuffers(length: number) {
+        // Per Arrow C++ reference implementation 
(cpp/src/arrow/ipc/reader.cc),
+        // each variadic buffer is stored as a separate buffer region, matching
+        // the IPC format where each is accessed via separate GetBuffer() 
calls.
+        // VARIADIC_DATA_BUFFERS in JSON is an array, but flattenDataSources 
spreads
+        // it so each hex string gets its own sources entry, maintaining 1:1
+        // correspondence with BufferRegion entries.
+        const buffers: Uint8Array[] = [];
+        for (let i = 0; i < length; i++) {
+            const { offset } = this.nextBufferRange();
+            // sources[offset] is 'any[]' but for variadic buffers it's 
actually a string
+            // after spreading in flattenDataSources. Cast necessary due to 
heterogeneous
+            // sources array structure (most fields are arrays, variadic 
elements are strings).
+            const hexString = this.sources[offset] as unknown as string;
+            buffers.push(hexStringToBytes(hexString));
+        }
+        return buffers;
+    }
+}
+
+/** @ignore */
+function hexStringToBytes(hexString: string): Uint8Array {
+    // Parse hex string per Arrow JSON integration format (uppercase hex 
encoding).
+    // Used for: VARIADIC_DATA_BUFFERS elements, Binary DATA (after join),
+    // BinaryView PREFIX_HEX and INLINED fields.
+    const data = new Uint8Array(hexString.length / 2);
+    for (let i = 0; i < hexString.length; i += 2) {
+        data[i >> 1] = Number.parseInt(hexString.slice(i, i + 2), 16);
+    }
+    return data;
+}
+
+/** @ignore */
+function binaryDataFromJSON(values: string[]): Uint8Array {
+    // Arrow JSON Binary/LargeBinary/FixedSizeBinary format:
+    // "DATA": ["49BC7D5B6C47D2","3F5FB6D9322026"] (array of hex strings, one 
per value)
+    // Join all values into one continuous hex string, then parse to bytes.
+    return hexStringToBytes(values.join(''));
 }
 
 /** @ignore */
-function binaryDataFromJSON(values: string[]) {
-    // "DATA": ["49BC7D5B6C47D2","3F5FB6D9322026"]
-    // There are definitely more efficient ways to do this... but it gets the
-    // job done.
-    const joined = values.join('');
-    const data = new Uint8Array(joined.length / 2);
-    for (let i = 0; i < joined.length; i += 2) {
-        data[i >> 1] = Number.parseInt(joined.slice(i, i + 2), 16);
+function parseViewDataFromJSON(views: any[], parseInlined: (inlined: string) 
=> Uint8Array) {
+    // Each view is a 16-byte struct: [length: i32, prefix/inlined: 12 bytes, 
buffer_index: i32, offset: i32]
+    const data = new Uint8Array(views.length * 16);
+    const dataView = new DataView(data.buffer);
+
+    for (const [i, view] of views.entries()) {
+        const offset = i * 16;
+        const size = view['SIZE'];
+
+        // Write size (int32 at byte 0)
+        dataView.setInt32(offset, size, true);
+
+        if (view['INLINED'] !== undefined) {
+            // Inline view: parse INLINED field using provided callback
+            const bytes = parseInlined(view['INLINED']);
+            for (let j = 0; j < bytes.length && j < 12; j++) {
+                data[offset + 4 + j] = bytes[j];
+            }
+        } else {
+            // Out-of-line view: write prefix, buffer_index, offset
+            const prefix = view['PREFIX_HEX'];
+            // Write 4-byte prefix at bytes 4-7
+            for (let j = 0; j < 8 && j < prefix.length; j += 2) {
+                data[offset + 4 + (j >> 1)] = Number.parseInt(prefix.slice(j, 
j + 2), 16);
+            }
+            // Write buffer_index (int32 at byte 8)
+            dataView.setInt32(offset + 8, view['BUFFER_INDEX'], true);
+            // Write offset (int32 at byte 12)
+            dataView.setInt32(offset + 12, view['OFFSET'], true);
+        }
     }
+
     return data;
 }
 
+/** @ignore */
+function binaryViewDataFromJSON(views: any[]) {
+    return parseViewDataFromJSON(views, (inlined: string) => {
+        // BinaryView: INLINED is hex-encoded string
+        const bytes = new Uint8Array(inlined.length / 2);
+        for (let i = 0; i < inlined.length; i += 2) {
+            bytes[i >> 1] = Number.parseInt(inlined.slice(i, i + 2), 16);
+        }
+        return bytes;
+    });
+}
+
+/** @ignore */
+function utf8ViewDataFromJSON(views: any[]) {
+    return parseViewDataFromJSON(views, (inlined: string) => {
+        // Utf8View: INLINED is UTF-8 string - encode to bytes
+        const encoder = new TextEncoder();
+        return encoder.encode(inlined);
+    });
+}
+
 export class CompressedVectorLoader extends VectorLoader {
     private bodyChunks: Uint8Array[];
-    constructor(bodyChunks: Uint8Array[], nodes: FieldNode[], buffers: 
BufferRegion[], dictionaries: Map<number, Vector<any>>, metadataVersion: 
MetadataVersion) {
-        super(new Uint8Array(0), nodes, buffers, dictionaries, 
metadataVersion);
+    constructor(bodyChunks: Uint8Array[], nodes: FieldNode[], buffers: 
BufferRegion[], dictionaries: Map<number, Vector<any>>, metadataVersion: 
MetadataVersion, variadicBufferCounts: number[] = []) {
+        super(new Uint8Array(0), nodes, buffers, dictionaries, 
metadataVersion, variadicBufferCounts);
         this.bodyChunks = bodyChunks;
     }
     protected readData<T extends DataType>(_type: T, _buffer = 
this.nextBufferRange()) {
diff --git a/test/data/tables.ts b/test/data/tables.ts
index e9674d9..80950a5 100644
--- a/test/data/tables.ts
+++ b/test/data/tables.ts
@@ -27,7 +27,7 @@ const nestedVectorGeneratorNames = ['struct', 'denseUnion', 
'sparseUnion', 'map'
 const dictionaryKeyGeneratorNames = ['int8', 'int16', 'int32', 'uint8', 
'uint16', 'uint32'];
 const valueVectorGeneratorNames = [
     'null_', 'bool', 'int8', 'int16', 'int32', 'int64', 'uint8', 'uint16', 
'uint32', 'uint64',
-    'float16', 'float32', 'float64', 'utf8', 'largeUtf8', 'binary', 
'largeBinary', 'fixedSizeBinary', 'dateDay', 'dateMillisecond',
+    'float16', 'float32', 'float64', 'utf8', 'largeUtf8', 'utf8View', 
'binary', 'largeBinary', 'binaryView', 'fixedSizeBinary', 'dateDay', 
'dateMillisecond',
     'timestampSecond', 'timestampMillisecond', 'timestampMicrosecond', 
'timestampNanosecond',
     'timeSecond', 'timeMillisecond', 'timeMicrosecond', 'timeNanosecond', 
'decimal',
     'dictionary', 'intervalDayTime', 'intervalYearMonth', 
'intervalMonthDayNano',
diff --git a/test/generate-test-data.ts b/test/generate-test-data.ts
index de4a826..f173633 100644
--- a/test/generate-test-data.ts
+++ b/test/generate-test-data.ts
@@ -16,14 +16,14 @@
 // under the License.
 
 import {
-    makeData, Vector, Visitor, DataType, TypeMap,
+    makeData, Vector, vectorFromArray, Visitor, DataType, TypeMap,
     Table, Schema, Field, RecordBatch,
     Null,
     Bool,
     Int, Int8, Int16, Int32, Int64, Uint8, Uint16, Uint32, Uint64,
     Float, Float16, Float32, Float64,
-    Utf8, LargeUtf8,
-    Binary, LargeBinary,
+    Utf8, LargeUtf8, Utf8View,
+    Binary, LargeBinary, BinaryView,
     FixedSizeBinary,
     Date_, DateDay, DateMillisecond,
     Timestamp, TimestampSecond, TimestampMillisecond, TimestampMicrosecond, 
TimestampNanosecond,
@@ -79,8 +79,10 @@ interface TestDataVectorGenerator extends Visitor {
     visitFloat: typeof generateFloat;
     visitUtf8: typeof generateUtf8;
     visitLargeUtf8: typeof generateLargeUtf8;
+    visitUtf8View: typeof generateUtf8View;
     visitBinary: typeof generateBinary;
     visitLargeBinary: typeof generateLargeBinary;
+    visitBinaryView: typeof generateBinaryView;
     visitFixedSizeBinary: typeof generateFixedSizeBinary;
     visitDate: typeof generateDate;
     visitTimestamp: typeof generateTimestamp;
@@ -106,8 +108,10 @@ TestDataVectorGenerator.prototype.visitUint64 = 
generateBigInt;
 TestDataVectorGenerator.prototype.visitFloat = generateFloat;
 TestDataVectorGenerator.prototype.visitUtf8 = generateUtf8;
 TestDataVectorGenerator.prototype.visitLargeUtf8 = generateLargeUtf8;
+TestDataVectorGenerator.prototype.visitUtf8View = generateUtf8View;
 TestDataVectorGenerator.prototype.visitBinary = generateBinary;
 TestDataVectorGenerator.prototype.visitLargeBinary = generateLargeBinary;
+TestDataVectorGenerator.prototype.visitBinaryView = generateBinaryView;
 TestDataVectorGenerator.prototype.visitFixedSizeBinary = 
generateFixedSizeBinary;
 TestDataVectorGenerator.prototype.visitDate = generateDate;
 TestDataVectorGenerator.prototype.visitTimestamp = generateTimestamp;
@@ -222,8 +226,10 @@ export const float32 = (length = 100, nullCount = 
Math.trunc(length * 0.2)) => v
 export const float64 = (length = 100, nullCount = Math.trunc(length * 0.2)) => 
vectorGenerator.visit(new Float64(), length, nullCount);
 export const utf8 = (length = 100, nullCount = Math.trunc(length * 0.2)) => 
vectorGenerator.visit(new Utf8(), length, nullCount);
 export const largeUtf8 = (length = 100, nullCount = Math.trunc(length * 0.2)) 
=> vectorGenerator.visit(new LargeUtf8(), length, nullCount);
+export const utf8View = (length = 100, nullCount = Math.trunc(length * 0.2)) 
=> vectorGenerator.visit(new Utf8View(), length, nullCount);
 export const binary = (length = 100, nullCount = Math.trunc(length * 0.2)) => 
vectorGenerator.visit(new Binary(), length, nullCount);
 export const largeBinary = (length = 100, nullCount = Math.trunc(length * 
0.2)) => vectorGenerator.visit(new LargeBinary(), length, nullCount);
+export const binaryView = (length = 100, nullCount = Math.trunc(length * 0.2)) 
=> vectorGenerator.visit(new BinaryView(), length, nullCount);
 export const fixedSizeBinary = (length = 100, nullCount = Math.trunc(length * 
0.2), byteWidth = 8) => vectorGenerator.visit(new FixedSizeBinary(byteWidth), 
length, nullCount);
 export const dateDay = (length = 100, nullCount = Math.trunc(length * 0.2)) => 
vectorGenerator.visit(new DateDay(), length, nullCount);
 export const dateMillisecond = (length = 100, nullCount = Math.trunc(length * 
0.2)) => vectorGenerator.visit(new DateMillisecond(), length, nullCount);
@@ -252,7 +258,7 @@ export const fixedSizeList = (length = 100, nullCount = 
Math.trunc(length * 0.2)
 export const map = <TKey extends DataType = any, TValue extends DataType = 
any>(length = 100, nullCount = Math.trunc(length * 0.2), child: Field<Struct<{ 
key: TKey; value: TValue }>> = <any>defaultMapChild()) => 
vectorGenerator.visit(new Map_<TKey, TValue>(child), length, nullCount);
 
 export const vecs = {
-    null_, bool, int8, int16, int32, int64, uint8, uint16, uint32, uint64, 
float16, float32, float64, utf8, largeUtf8, binary, largeBinary, 
fixedSizeBinary, dateDay, dateMillisecond, timestampSecond, 
timestampMillisecond, timestampMicrosecond, timestampNanosecond, timeSecond, 
timeMillisecond, timeMicrosecond, timeNanosecond, decimal, list, struct, 
denseUnion, sparseUnion, dictionary, intervalDayTime, intervalYearMonth, 
intervalMonthDayNano, fixedSizeList, map, durationSecond, durationMil [...]
+    null_, bool, int8, int16, int32, int64, uint8, uint16, uint32, uint64, 
float16, float32, float64, utf8, largeUtf8, utf8View, binary, largeBinary, 
binaryView, fixedSizeBinary, dateDay, dateMillisecond, timestampSecond, 
timestampMillisecond, timestampMicrosecond, timestampNanosecond, timeSecond, 
timeMillisecond, timeMicrosecond, timeNanosecond, decimal, list, struct, 
denseUnion, sparseUnion, dictionary, intervalDayTime, intervalYearMonth, 
intervalMonthDayNano, fixedSizeList, map, durat [...]
 } as { [k: string]: (...args: any[]) => any };
 
 function generateNull<T extends Null>(this: TestDataVectorGenerator, type: T, 
length = 100): GeneratedVector<T> {
@@ -364,6 +370,13 @@ function generateLargeUtf8<T extends LargeUtf8>(this: 
TestDataVectorGenerator, t
     return { values: () => values, vector: new Vector([makeData({ type, 
length, nullCount, nullBitmap, valueOffsets, data })]) };
 }
 
+function generateUtf8View<T extends Utf8View>(this: TestDataVectorGenerator, 
type: T, length = 100, nullCount = Math.trunc(length * 0.2)): 
GeneratedVector<T> {
+    const nullBitmap = createBitmap(length, nullCount);
+    const values = Array.from({ length }, (_, i) => isValid(nullBitmap, i) ? 
randomString(Math.trunc(Math.random() * 20)) : null);
+    const vector = vectorFromArray(values, type);
+    return { values: () => values, vector };
+}
+
 function generateBinary<T extends Binary>(this: TestDataVectorGenerator, type: 
T, length = 100, nullCount = Math.trunc(length * 0.2)): GeneratedVector<T> {
     const nullBitmap = createBitmap(length, nullCount);
     const valueOffsets = createVariableWidthOffsets32(length, nullBitmap, 10, 
20, nullCount != 0);
@@ -384,6 +397,13 @@ function generateLargeBinary<T extends LargeBinary>(this: 
TestDataVectorGenerato
     return { values: () => values, vector: new Vector([makeData({ type, 
length, nullCount, nullBitmap, valueOffsets, data })]) };
 }
 
+function generateBinaryView<T extends BinaryView>(this: 
TestDataVectorGenerator, type: T, length = 100, nullCount = Math.trunc(length * 
0.2)): GeneratedVector<T> {
+    const nullBitmap = createBitmap(length, nullCount);
+    const values = Array.from({ length }, (_, i) => isValid(nullBitmap, i) ? 
randomBytes(Math.trunc(Math.random() * 20)) : null);
+    const vector = vectorFromArray(values, type);
+    return { values: () => values, vector };
+}
+
 function generateFixedSizeBinary<T extends FixedSizeBinary>(this: 
TestDataVectorGenerator, type: T, length = 100, nullCount = Math.trunc(length * 
0.2)): GeneratedVector<T> {
     const nullBitmap = createBitmap(length, nullCount);
     const data = fillRandom(Uint8Array, length * type.byteWidth);
diff --git a/test/unit/builders/view-builders-tests.ts 
b/test/unit/builders/view-builders-tests.ts
new file mode 100644
index 0000000..88ee28f
--- /dev/null
+++ b/test/unit/builders/view-builders-tests.ts
@@ -0,0 +1,258 @@
+// 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.
+
+import { BinaryView, Utf8View } from '../../../src/type.js';
+import { makeBuilder, vectorFromArray } from '../../../src/factories.js';
+
+describe('BinaryViewBuilder', () => {
+    it('should build inline binary values (≤12 bytes)', () => {
+        const builder = makeBuilder({ type: new BinaryView() });
+        const values = [
+            new Uint8Array([1, 2, 3]),
+            new Uint8Array([4, 5, 6, 7, 8, 9, 10, 11, 12]),
+            new Uint8Array([13])
+        ];
+
+        for (const value of values) {
+            builder.append(value);
+        }
+
+        const vector = builder.finish().toVector();
+        expect(vector).toHaveLength(3);
+        expect(vector.get(0)).toEqual(values[0]);
+        expect(vector.get(1)).toEqual(values[1]);
+        expect(vector.get(2)).toEqual(values[2]);
+    });
+
+    it('should build out-of-line binary values (>12 bytes)', () => {
+        const builder = makeBuilder({ type: new BinaryView() });
+        const value = new Uint8Array(100);
+        for (let i = 0; i < 100; i++) {
+            value[i] = i % 256;
+        }
+
+        builder.append(value);
+
+        const vector = builder.finish().toVector();
+        expect(vector).toHaveLength(1);
+        expect(vector.get(0)).toEqual(value);
+    });
+
+    it('should build mixed inline and out-of-line values', () => {
+        const builder = makeBuilder({ type: new BinaryView() });
+        const small = new Uint8Array([1, 2, 3]);
+        const large = new Uint8Array(50);
+        for (let i = 0; i < 50; i++) {
+            large[i] = i % 256;
+        }
+
+        builder.append(small);
+        builder.append(large);
+        builder.append(small);
+
+        const vector = builder.finish().toVector();
+        expect(vector).toHaveLength(3);
+        expect(vector.get(0)).toEqual(small);
+        expect(vector.get(1)).toEqual(large);
+        expect(vector.get(2)).toEqual(small);
+    });
+
+    it('should handle null values', () => {
+        const builder = makeBuilder({ type: new BinaryView(), nullValues: 
[null] });
+
+        builder.append(new Uint8Array([1, 2, 3]));
+        builder.append(null);
+        builder.append(new Uint8Array([4, 5, 6]));
+
+        const vector = builder.finish().toVector();
+        expect(vector).toHaveLength(3);
+        expect(vector.get(0)).toEqual(new Uint8Array([1, 2, 3]));
+        expect(vector.get(1)).toBeNull();
+        expect(vector.get(2)).toEqual(new Uint8Array([4, 5, 6]));
+    });
+
+    it('should handle empty values', () => {
+        const builder = makeBuilder({ type: new BinaryView() });
+
+        builder.append(new Uint8Array([]));
+        builder.append(new Uint8Array([1]));
+
+        const vector = builder.finish().toVector();
+        expect(vector).toHaveLength(2);
+        expect(vector.get(0)).toEqual(new Uint8Array([]));
+        expect(vector.get(1)).toEqual(new Uint8Array([1]));
+    });
+
+    it('should handle exactly 12-byte boundary values', () => {
+        const builder = makeBuilder({ type: new BinaryView() });
+        const exactly12 = new Uint8Array([1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 
12]);
+        const exactly13 = new Uint8Array([1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 
12, 13]);
+
+        builder.append(exactly12);
+        builder.append(exactly13);
+
+        const vector = builder.finish().toVector();
+        expect(vector).toHaveLength(2);
+        expect(vector.get(0)).toEqual(exactly12);
+        expect(vector.get(1)).toEqual(exactly13);
+    });
+
+    it('should handle multiple flushes', () => {
+        const builder = makeBuilder({ type: new BinaryView() });
+
+        builder.append(new Uint8Array([1, 2]));
+        const data1 = builder.flush();
+        expect(data1).toHaveLength(1);
+
+        builder.append(new Uint8Array([3, 4]));
+        builder.append(new Uint8Array([5, 6]));
+        const data2 = builder.flush();
+        expect(data2).toHaveLength(2);
+    });
+});
+
+describe('Utf8ViewBuilder', () => {
+    it('should build inline string values (≤12 bytes)', () => {
+        const builder = makeBuilder({ type: new Utf8View() });
+        const values = ['hello', 'world', 'foo'];
+
+        for (const value of values) {
+            builder.append(value);
+        }
+
+        const vector = builder.finish().toVector();
+        expect(vector).toHaveLength(3);
+        expect(vector.get(0)).toBe('hello');
+        expect(vector.get(1)).toBe('world');
+        expect(vector.get(2)).toBe('foo');
+    });
+
+    it('should build out-of-line string values (>12 bytes)', () => {
+        const builder = makeBuilder({ type: new Utf8View() });
+        const longString = 'This is a long string that exceeds 12 bytes';
+
+        builder.append(longString);
+
+        const vector = builder.finish().toVector();
+        expect(vector).toHaveLength(1);
+        expect(vector.get(0)).toBe(longString);
+    });
+
+    it('should build mixed inline and out-of-line strings', () => {
+        const builder = makeBuilder({ type: new Utf8View() });
+        const short = 'hi';
+        const long = 'This is a very long string that definitely exceeds the 
12 byte inline capacity';
+
+        builder.append(short);
+        builder.append(long);
+        builder.append(short);
+
+        const vector = builder.finish().toVector();
+        expect(vector).toHaveLength(3);
+        expect(vector.get(0)).toBe(short);
+        expect(vector.get(1)).toBe(long);
+        expect(vector.get(2)).toBe(short);
+    });
+
+    it('should handle null values', () => {
+        const builder = makeBuilder({ type: new Utf8View(), nullValues: [null] 
});
+
+        builder.append('hello');
+        builder.append(null);
+        builder.append('world');
+
+        const vector = builder.finish().toVector();
+        expect(vector).toHaveLength(3);
+        expect(vector.get(0)).toBe('hello');
+        expect(vector.get(1)).toBeNull();
+        expect(vector.get(2)).toBe('world');
+    });
+
+    it('should handle empty strings', () => {
+        const builder = makeBuilder({ type: new Utf8View() });
+
+        builder.append('');
+        builder.append('a');
+
+        const vector = builder.finish().toVector();
+        expect(vector).toHaveLength(2);
+        expect(vector.get(0)).toBe('');
+        expect(vector.get(1)).toBe('a');
+    });
+
+    it('should handle UTF-8 multibyte characters', () => {
+        const builder = makeBuilder({ type: new Utf8View() });
+        const values = ['🚀', '你好', 'Ñoño', 'emoji: 🎉'];
+
+        for (const value of values) {
+            builder.append(value);
+        }
+
+        const vector = builder.finish().toVector();
+        expect(vector).toHaveLength(4);
+        expect(vector.get(0)).toBe('🚀');
+        expect(vector.get(1)).toBe('你好');
+        expect(vector.get(2)).toBe('Ñoño');
+        expect(vector.get(3)).toBe('emoji: 🎉');
+    });
+
+    it('should handle exactly 12-byte boundary strings', () => {
+        const builder = makeBuilder({ type: new Utf8View() });
+        const exactly12 = 'twelve bytes'; // ASCII: 12 bytes
+        const exactly13 = 'thirteen byte'; // ASCII: 13 bytes
+
+        builder.append(exactly12);
+        builder.append(exactly13);
+
+        const vector = builder.finish().toVector();
+        expect(vector).toHaveLength(2);
+        expect(vector.get(0)).toBe(exactly12);
+        expect(vector.get(1)).toBe(exactly13);
+    });
+
+    it('should build from vectorFromArray', () => {
+        const values = ['hello', 'world', null, 'foo'];
+        const vector = vectorFromArray(values, new Utf8View());
+
+        expect(vector).toHaveLength(4);
+        expect(vector.get(0)).toBe('hello');
+        expect(vector.get(1)).toBe('world');
+        expect(vector.get(2)).toBeNull();
+        expect(vector.get(3)).toBe('foo');
+    });
+
+    it('should handle large batch of values', () => {
+        const builder = makeBuilder({ type: new Utf8View() });
+        const count = 1000;
+        const values: string[] = [];
+
+        for (let i = 0; i < count; i++) {
+            const value = i % 2 === 0
+                ? `short_${i}` // inline
+                : `this_is_a_long_string_that_goes_out_of_line_${i}`; // 
out-of-line
+            values.push(value);
+            builder.append(value);
+        }
+
+        const vector = builder.finish().toVector();
+        expect(vector).toHaveLength(count);
+
+        for (let i = 0; i < count; i++) {
+            expect(vector.get(i)).toBe(values[i]);
+        }
+    });
+});
diff --git a/test/unit/generated-data-tests.ts 
b/test/unit/generated-data-tests.ts
index 9affe5f..0d3a760 100644
--- a/test/unit/generated-data-tests.ts
+++ b/test/unit/generated-data-tests.ts
@@ -39,8 +39,10 @@ describe('Generated Test Data', () => {
     describe('Float64', () => { validateVector(generate.float64()); });
     describe('Utf8', () => { validateVector(generate.utf8()); });
     describe('LargeUtf8', () => { validateVector(generate.largeUtf8()); });
+    describe('Utf8View', () => { validateVector(generate.utf8View()); });
     describe('Binary', () => { validateVector(generate.binary()); });
     describe('LargeBinary', () => { validateVector(generate.largeBinary()); });
+    describe('BinaryView', () => { validateVector(generate.binaryView()); });
     describe('FixedSizeBinary', () => { 
validateVector(generate.fixedSizeBinary()); });
     describe('DateDay', () => { validateVector(generate.dateDay()); });
     describe('DateMillisecond', () => { 
validateVector(generate.dateMillisecond()); });
diff --git a/test/unit/vector/vector-tests.ts b/test/unit/vector/vector-tests.ts
index 73c9cdb..d8199e5 100644
--- a/test/unit/vector/vector-tests.ts
+++ b/test/unit/vector/vector-tests.ts
@@ -16,7 +16,7 @@
 // under the License.
 
 import {
-    Bool, DateDay, DateMillisecond, Dictionary, Float64, Int32, List, 
makeVector, Struct, Utf8, LargeUtf8, util, Vector, vectorFromArray, makeData, 
FixedSizeList, Field,
+    Bool, DateDay, DateMillisecond, Dictionary, Float64, Int32, List, 
makeVector, Struct, Utf8, LargeUtf8, Utf8View, BinaryView, util, Vector, 
vectorFromArray, makeData, FixedSizeList, Field,
 } from 'apache-arrow';
 
 describe(`makeVectorFromArray`, () => {
@@ -256,6 +256,50 @@ describe(`LargeUtf8Vector`, () => {
     });
 });
 
+describe(`Utf8ViewVector`, () => {
+    const values = ['foo', 'bar', 'baz', 'foo bar', 'bar'];
+    const vector = vectorFromArray(values, new Utf8View);
+
+    test(`has utf8View type`, () => {
+        expect(vector.type).toBeInstanceOf(Utf8View);
+    });
+
+    test(`is not memoized`, () => {
+        expect(vector.isMemoized).toBe(false);
+        const memoizedVector = vector.memoize();
+        expect(memoizedVector.isMemoized).toBe(true);
+        const unMemoizedVector = vector.unmemoize();
+        expect(unMemoizedVector.isMemoized).toBe(false);
+    });
+
+    basicVectorTests(vector, values, ['abc', '123']);
+    describe(`sliced`, () => {
+        basicVectorTests(vector.slice(1, 3), values.slice(1, 3), ['foo', 
'abc']);
+    });
+});
+
+describe(`BinaryViewVector`, () => {
+    const values = [new Uint8Array([1, 2, 3]), new Uint8Array([4, 5]), new 
Uint8Array([6, 7, 8, 9])];
+    const vector = vectorFromArray(values, new BinaryView);
+
+    test(`has binaryView type`, () => {
+        expect(vector.type).toBeInstanceOf(BinaryView);
+    });
+
+    test(`is not memoized`, () => {
+        expect(vector.isMemoized).toBe(false);
+        const memoizedVector = vector.memoize();
+        expect(memoizedVector.isMemoized).toBe(true);
+        const unMemoizedVector = vector.unmemoize();
+        expect(unMemoizedVector.isMemoized).toBe(false);
+    });
+
+    basicVectorTests(vector, values, [new Uint8Array([10, 11]), new 
Uint8Array([12])]);
+    describe(`sliced`, () => {
+        basicVectorTests(vector.slice(1, 3), values.slice(1, 3), [new 
Uint8Array([1, 2]), new Uint8Array([3, 4])]);
+    });
+});
+
 describe(`ListVector`, () => {
     const values = [[1, 2], [1, 2, 3]];
     const vector = vectorFromArray(values);


Reply via email to