You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by bh...@apache.org on 2019/02/23 02:24:51 UTC

[arrow] branch master updated: ARROW-4578: [JS] Ensure Float16 is zero-copy, add more native BigInt support

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

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


The following commit(s) were added to refs/heads/master by this push:
     new a26cf7a  ARROW-4578: [JS] Ensure Float16 is zero-copy, add more native BigInt support
a26cf7a is described below

commit a26cf7a2588fe4f1372816f962906939848da6f6
Author: ptaylor <pa...@me.com>
AuthorDate: Fri Feb 22 18:23:33 2019 -0800

    ARROW-4578: [JS] Ensure Float16 is zero-copy, add more native BigInt support
    
    This started as a continuation of https://github.com/apache/arrow/pull/3634, but grew enough to deserve its own PR. I've made a PR to my own fork that highlights just the changes here: https://github.com/trxcllnt/arrow/pull/8. I'll rebase this PR after https://github.com/apache/arrow/pull/3634 is merged so only these changes are included.
    
    This PR reverts the behavior of `Float16Vector#toArray()` back to returning a zero-copy slice of the underlying `Uint16Array` data, and exposes the copying behavior via new `toFloat32Array()` and `toFloat64Array()` methods. `Float16Array.from()` will also convert any incoming 32 or 64-bit floats to Uint16s if necessary.
    
    It also adds tighter integration with the new `BigInt`, `BigInt64Array`, and `BigUint64Array` primitives (if available):
    1. Use the native `BigInt` to convert/stringify i64s/u64s
    2. Support the `BigInt` type in element comparator and `indexOf()`
    3. Add zero-copy `toBigInt64Array()` and `toBigUint64Array()` methods to `Int64Vector` and `Uint64Vector`, respectively
    
    0.4.0 added support for basic conversion to the native `BigInt` when available, but would only create positive `BigInts`, and was slower than necessary. This PR uses the native Arrays to create the BigInts, so we should see some speed ups there. Ex:
    
    ```ts
    const vec = Int64Vector.from(new Int32Array([-1, 2147483647]))
    const big = vec.get(0)
    assert(big[0] === -1) // true
    assert(big[1] === 2147483647) // true
    const num = 0n + big // or BigInt(big)
    assert(num === (2n ** 63n - 1n)) // true
    ```
    
    JIRAs associated with this PR are:
    * [ARROW-4578](https://issues.apache.org/jira/browse/ARROW-4578) - Float16Vector toArray should be zero-copy
    * [ARROW-4579](https://issues.apache.org/jira/browse/ARROW-4579) - Add more interop with BigInt/BigInt64Array/BigUint64Array
    * [ARROW-4580](https://issues.apache.org/jira/browse/ARROW-4580) - Accept Iterables in IntVector/FloatVector from() signatures
    
    Author: ptaylor <pa...@me.com>
    
    Closes #3653 from trxcllnt/js/int-and-float-fixes and squashes the following commits:
    
    69ee6f77 <ptaylor> cleanup after rebase
    f44e97b3 <ptaylor> ensure truncated bitmap size isn't larger than it should be
    7ac081ad <ptaylor> fix lint
    6046e660 <ptaylor> remove more getters in favor of readonly direct property accesses
    94d56334 <ptaylor> support BigInt in comparitor/indexOf
    760a2199 <ptaylor> update BN to use BigIntArrays for signed/unsigned 64bit integers if possible
    77fcd402 <ptaylor> add initial BigInt64Array and BigUint64Array support
    d561204e <ptaylor> ensure Float16Vector.toArray() is zero-copy again, add toFloat32Array() and toFloat64Array() methods instead
    854ae66f <ptaylor> ensure Int/FloatVector.from return signatures are as specific as possible, and accept Iterable<number>
    4656ea55 <ptaylor> cleanup/rename Table + Schema + RecordBatch from -> new, cleanup argument extraction util fns
    69abf406 <ptaylor> add initial RecordBatch.new and select tests
    9c7ed3d4 <ptaylor> guard against out-of-bounds selections
    a4222f81 <ptaylor> clean up: eliminate more getters in favor of read-only properties
    8eabb1c0 <ptaylor> clean up/speed up: move common argument flattening methods into a utility file
    b3b4f1fd <ptaylor> add Table and Schema assign() impls
    79f9db1c <ptaylor> add selectAt() method to Table, Schema, and RecordBatch for selecting columns by index
---
 js/.vscode/launch.json                      |   1 -
 js/src/interfaces.ts                        |  41 ++-
 js/src/type.ts                              | 221 +++++++-------
 js/src/util/bit.ts                          |  17 +-
 js/src/util/bn.ts                           |  49 +--
 js/src/util/buffer.ts                       |  36 ++-
 js/src/util/compat.ts                       |  37 +++
 js/src/util/vector.ts                       |  10 +-
 js/src/vector/float.ts                      |  42 ++-
 js/src/vector/int.ts                        |  92 ++++--
 js/src/visitor/jsonvectorassembler.ts       |   2 +-
 js/src/visitor/toarray.ts                   |   9 +-
 js/test/unit/vector/float16-vector-tests.ts |  73 -----
 js/test/unit/vector/numeric-vector-tests.ts | 442 +++++++++++++++++++++++-----
 14 files changed, 749 insertions(+), 323 deletions(-)

diff --git a/js/.vscode/launch.json b/js/.vscode/launch.json
index fc99c1b..103d047 100644
--- a/js/.vscode/launch.json
+++ b/js/.vscode/launch.json
@@ -52,7 +52,6 @@
                 // "test/unit/vector/vector-tests.ts",
                 // "test/unit/vector/bool-vector-tests.ts",
                 // "test/unit/vector/date-vector-tests.ts",
-                // "test/unit/vector/float16-vector-tests.ts",
                 // "test/unit/vector/numeric-vector-tests.ts",
 
                 // "test/unit/visitor-tests.ts",
diff --git a/js/src/interfaces.ts b/js/src/interfaces.ts
index f908eff..21407af 100644
--- a/js/src/interfaces.ts
+++ b/js/src/interfaces.ts
@@ -21,12 +21,19 @@ import * as type from './type';
 import { DataType } from './type';
 import * as vecs from './vector/index';
 
+/** @ignore */ type FloatArray = Float32Array | Float64Array;
+/** @ignore */ type IntArray = Int8Array | Int16Array | Int32Array;
+/** @ignore */ type UintArray = Uint8Array | Uint16Array | Uint32Array | Uint8ClampedArray;
 /** @ignore */
-export interface ArrayBufferViewConstructor<T extends ArrayBufferView> {
+export type TypedArray = FloatArray | IntArray | UintArray;
+export type BigIntArray = BigInt64Array | BigUint64Array;
+
+/** @ignore */
+export interface TypedArrayConstructor<T extends TypedArray> {
     readonly prototype: T;
-    new(length: number): T;
-    new(arrayOrArrayBuffer: ArrayLike<number> | ArrayBufferLike): T;
-    new(buffer: ArrayBufferLike, byteOffset: number, length?: number): T;
+    new(length?: number): T;
+    new(array: Iterable<number>): T;
+    new(buffer: ArrayBufferLike, byteOffset?: number, length?: number): T;
     /**
       * The size in bytes of each element in the array.
       */
@@ -43,6 +50,32 @@ export interface ArrayBufferViewConstructor<T extends ArrayBufferView> {
       * @param thisArg Value of 'this' used to invoke the mapfn.
       */
     from(arrayLike: ArrayLike<number>, mapfn?: (v: number, k: number) => number, thisArg?: any): T;
+    from<U>(arrayLike: ArrayLike<U>, mapfn: (v: U, k: number) => number, thisArg?: any): T;
+}
+
+/** @ignore */
+export interface BigIntArrayConstructor<T extends BigIntArray> {
+    readonly prototype: T;
+    new(length?: number): T;
+    new(array: Iterable<bigint>): T;
+    new(buffer: ArrayBufferLike, byteOffset?: number, length?: number): T;
+    /**
+      * The size in bytes of each element in the array.
+      */
+    readonly BYTES_PER_ELEMENT: number;
+    /**
+      * Returns a new array from a set of elements.
+      * @param items A set of elements to include in the new array object.
+      */
+    of(...items: bigint[]): T;
+    /**
+      * Creates an array from an array-like or iterable object.
+      * @param arrayLike An array-like or iterable object to convert to an array.
+      * @param mapfn A mapping function to call on every element of the array.
+      * @param thisArg Value of 'this' used to invoke the mapfn.
+      */
+    from(arrayLike: ArrayLike<bigint>, mapfn?: (v: bigint, k: number) => bigint, thisArg?: any): T;
+    from<U>(arrayLike: ArrayLike<U>, mapfn: (v: U, k: number) => bigint, thisArg?: any): T;
 }
 
 /** @ignore */
diff --git a/js/src/type.ts b/js/src/type.ts
index 55e4814..9ad15bc 100644
--- a/js/src/type.ts
+++ b/js/src/type.ts
@@ -20,7 +20,7 @@
 import { Field } from './schema';
 import { Vector } from './vector';
 import { flatbuffers } from 'flatbuffers';
-import { ArrayBufferViewConstructor } from './interfaces';
+import { TypedArrayConstructor } from './interfaces';
 import { Vector as VType, TypeToDataType } from './interfaces';
 import { instance as comparer } from './visitor/typecomparator';
 
@@ -45,11 +45,12 @@ export type RowLike<T extends { [key: string]: DataType }> =
     & { get<K extends keyof T>(key: K): T[K]['TValue']; }
     ;
 
-export interface DataType<TType extends Type = Type> {
+export interface DataType<TType extends Type = Type, TChildren extends { [key: string]: DataType } = any> {
     readonly TType: TType;
     readonly TArray: any;
     readonly TValue: any;
     readonly ArrayType: any;
+    readonly children: Field<TChildren[keyof TChildren]>[];
 }
 
 export class DataType<TType extends Type = Type, TChildren extends { [key: string]: DataType } = any> {
@@ -76,16 +77,13 @@ export class DataType<TType extends Type = Type, TChildren extends { [key: strin
     /** @nocollapse */ static             isMap (x: any): x is Map_            { return x && x.typeId === Type.Map;             }
     /** @nocollapse */ static      isDictionary (x: any): x is Dictionary      { return x && x.typeId === Type.Dictionary;      }
 
-    public get children() { return this._children; }
     public get typeId(): TType { return <any> Type.NONE; }
-
-    constructor(protected _children?: Field<TChildren[keyof TChildren]>[]) {}
-
     public compareTo(other: DataType): other is TypeToDataType<TType> {
         return comparer.visit(this, other);
     }
 
     protected static [Symbol.toStringTag] = ((proto: DataType) => {
+        (<any> proto).children = null;
         (<any> proto).ArrayType = Array;
         return proto[Symbol.toStringTag] = 'DataType';
     })(DataType.prototype);
@@ -105,36 +103,36 @@ type Ints = Type.Int | Type.Int8 | Type.Int16 | Type.Int32 | Type.Int64 | Type.U
 /** @ignore */
 type IType = {
     [Type.Int   ]: { bitWidth: IntBitWidth; isSigned: true | false; TArray: IntArray;    TValue: number | Int32Array | Uint32Array; };
-    [Type.Int8  ]: { bitWidth:           8; isSigned: true;         TArray: Int8Array;   TValue: number;            };
-    [Type.Int16 ]: { bitWidth:          16; isSigned: true;         TArray: Int16Array;  TValue: number;            };
-    [Type.Int32 ]: { bitWidth:          32; isSigned: true;         TArray: Int32Array;  TValue: number;            };
-    [Type.Int64 ]: { bitWidth:          64; isSigned: true;         TArray: Int32Array;  TValue: Int32Array;        };
-    [Type.Uint8 ]: { bitWidth:           8; isSigned: false;        TArray: Uint8Array;  TValue: number;            };
-    [Type.Uint16]: { bitWidth:          16; isSigned: false;        TArray: Uint16Array; TValue: number;            };
-    [Type.Uint32]: { bitWidth:          32; isSigned: false;        TArray: Uint32Array; TValue: number;            };
-    [Type.Uint64]: { bitWidth:          64; isSigned: false;        TArray: Uint32Array; TValue: Uint32Array;       };
+    [Type.Int8  ]: { bitWidth:           8; isSigned: true;         TArray: Int8Array;   TValue: number;      };
+    [Type.Int16 ]: { bitWidth:          16; isSigned: true;         TArray: Int16Array;  TValue: number;      };
+    [Type.Int32 ]: { bitWidth:          32; isSigned: true;         TArray: Int32Array;  TValue: number;      };
+    [Type.Int64 ]: { bitWidth:          64; isSigned: true;         TArray: Int32Array;  TValue: Int32Array;  };
+    [Type.Uint8 ]: { bitWidth:           8; isSigned: false;        TArray: Uint8Array;  TValue: number;      };
+    [Type.Uint16]: { bitWidth:          16; isSigned: false;        TArray: Uint16Array; TValue: number;      };
+    [Type.Uint32]: { bitWidth:          32; isSigned: false;        TArray: Uint32Array; TValue: number;      };
+    [Type.Uint64]: { bitWidth:          64; isSigned: false;        TArray: Uint32Array; TValue: Uint32Array; };
 };
 
 interface Int_<T extends Ints = Ints> extends DataType<T> { TArray: IType[T]['TArray']; TValue: IType[T]['TValue']; }
 class Int_<T extends Ints = Ints> extends DataType<T> {
-    constructor(protected _isSigned: IType[T]['isSigned'],
-                protected _bitWidth: IType[T]['bitWidth']) {
+    constructor(public readonly isSigned: IType[T]['isSigned'],
+                public readonly bitWidth: IType[T]['bitWidth']) {
         super();
     }
     public get typeId() { return Type.Int as T; }
-    public get isSigned() { return this._isSigned; }
-    public get bitWidth() { return this._bitWidth; }
-    public get ArrayType(): ArrayBufferViewConstructor<IType[T]['TArray']> {
-        switch (this._bitWidth) {
-            case  8: return (this._isSigned ?  Int8Array :  Uint8Array) as any;
-            case 16: return (this._isSigned ? Int16Array : Uint16Array) as any;
-            case 32: return (this._isSigned ? Int32Array : Uint32Array) as any;
-            case 64: return (this._isSigned ? Int32Array : Uint32Array) as any;
+    public get ArrayType(): TypedArrayConstructor<IType[T]['TArray']> {
+        switch (this.bitWidth) {
+            case  8: return this.isSigned ?  Int8Array :  Uint8Array;
+            case 16: return this.isSigned ? Int16Array : Uint16Array;
+            case 32: return this.isSigned ? Int32Array : Uint32Array;
+            case 64: return this.isSigned ? Int32Array : Uint32Array;
         }
         throw new Error(`Unrecognized ${this[Symbol.toStringTag]} type`);
     }
-    public toString() { return `${this._isSigned ? `I` : `Ui`}nt${this._bitWidth}`; }
+    public toString() { return `${this.isSigned ? `I` : `Ui`}nt${this.bitWidth}`; }
     protected static [Symbol.toStringTag] = ((proto: Int_) => {
+        (<any> proto).isSigned = null;
+        (<any> proto).bitWidth = null;
         return proto[Symbol.toStringTag] = 'Int';
     })(Int_.prototype);
 }
@@ -150,6 +148,15 @@ export class Uint16 extends Int_<Type.Uint16> { constructor() { super(false, 16)
 export class Uint32 extends Int_<Type.Uint32> { constructor() { super(false, 32); } }
 export class Uint64 extends Int_<Type.Uint64> { constructor() { super(false, 64); } }
 
+Object.defineProperty(Int8.prototype, 'ArrayType', { value: Int8Array });
+Object.defineProperty(Int16.prototype, 'ArrayType', { value: Int16Array });
+Object.defineProperty(Int32.prototype, 'ArrayType', { value: Int32Array });
+Object.defineProperty(Int64.prototype, 'ArrayType', { value: Int32Array });
+Object.defineProperty(Uint8.prototype, 'ArrayType', { value: Uint8Array });
+Object.defineProperty(Uint16.prototype, 'ArrayType', { value: Uint16Array });
+Object.defineProperty(Uint32.prototype, 'ArrayType', { value: Uint32Array });
+Object.defineProperty(Uint64.prototype, 'ArrayType', { value: Uint32Array });
+
 /** @ignore */
 type Floats = Type.Float | Type.Float16 | Type.Float32 | Type.Float64;
 /** @ignore */
@@ -162,12 +169,11 @@ type FType = {
 
 export interface Float<T extends Floats = Floats> extends DataType<T> { TArray: FType[T]['TArray']; TValue: number; }
 export class Float<T extends Floats = Floats> extends DataType<T> {
-    constructor(protected _precision: Precision) {
+    constructor(public readonly precision: Precision) {
         super();
     }
     public get typeId() { return Type.Float as T; }
-    public get precision() { return this._precision; }
-    public get ArrayType(): ArrayBufferViewConstructor<FType[T]['TArray']> {
+    public get ArrayType(): TypedArrayConstructor<FType[T]['TArray']> {
         switch (this.precision) {
             case Precision.HALF: return Uint16Array;
             case Precision.SINGLE: return Float32Array;
@@ -177,6 +183,7 @@ export class Float<T extends Floats = Floats> extends DataType<T> {
     }
     public toString() { return `Float${(this.precision << 5) || 16}`; }
     protected static [Symbol.toStringTag] = ((proto: Float) => {
+        (<any> proto).precision = null;
         return proto[Symbol.toStringTag] = 'Float';
     })(Float.prototype);
 }
@@ -185,6 +192,10 @@ export class Float16 extends Float<Type.Float16> { constructor() { super(Precisi
 export class Float32 extends Float<Type.Float32> { constructor() { super(Precision.SINGLE); } }
 export class Float64 extends Float<Type.Float64> { constructor() { super(Precision.DOUBLE); } }
 
+Object.defineProperty(Float16.prototype, 'ArrayType', { value: Uint16Array });
+Object.defineProperty(Float32.prototype, 'ArrayType', { value: Float32Array });
+Object.defineProperty(Float64.prototype, 'ArrayType', { value: Float64Array });
+
 export interface Binary extends DataType<Type.Binary> { TArray: Uint8Array; TValue: Uint8Array; }
 export class Binary extends DataType<Type.Binary> {
     constructor() {
@@ -226,15 +237,15 @@ export class Bool extends DataType<Type.Bool> {
 
 export interface Decimal extends DataType<Type.Decimal> { TArray: Uint32Array; TValue: Uint32Array; ArrayType: typeof Uint32Array; }
 export class Decimal extends DataType<Type.Decimal> {
-    constructor(protected _scale: number,
-                protected _precision: number) {
+    constructor(public readonly scale: number,
+                public readonly precision: number) {
         super();
     }
     public get typeId() { return Type.Decimal as Type.Decimal; }
-    public get scale() { return this._scale; }
-    public get precision() { return this._precision; }
-    public toString() { return `Decimal[${this._precision}e${this._scale > 0 ? `+` : ``}${this._scale}]`; }
+    public toString() { return `Decimal[${this.precision}e${this.scale > 0 ? `+` : ``}${this.scale}]`; }
     protected static [Symbol.toStringTag] = ((proto: Decimal) => {
+        (<any> proto).scale = null;
+        (<any> proto).precision = null;
         (<any> proto).ArrayType = Uint32Array;
         return proto[Symbol.toStringTag] = 'Decimal';
     })(Decimal.prototype);
@@ -244,13 +255,13 @@ export class Decimal extends DataType<Type.Decimal> {
 export type Dates = Type.Date | Type.DateDay | Type.DateMillisecond;
 export interface Date_<T extends Dates = Dates> extends DataType<T> { TArray: Int32Array; TValue: Date; ArrayType: typeof Int32Array; }
 export class Date_<T extends Dates = Dates> extends DataType<T> {
-    constructor(protected _unit: DateUnit) {
+    constructor(public readonly unit: DateUnit) {
         super();
     }
     public get typeId() { return Type.Date as T; }
-    public get unit() { return this._unit; }
-    public toString() { return `Date${(this._unit + 1) * 32}<${DateUnit[this._unit]}>`; }
+    public toString() { return `Date${(this.unit + 1) * 32}<${DateUnit[this.unit]}>`; }
     protected static [Symbol.toStringTag] = ((proto: Date_) => {
+        (<any> proto).unit = null;
         (<any> proto).ArrayType = Int32Array;
         return proto[Symbol.toStringTag] = 'Date';
     })(Date_.prototype);
@@ -272,15 +283,15 @@ type TimesType = {
 
 interface Time_<T extends Times = Times> extends DataType<T> { TArray: Int32Array; TValue: TimesType[T]['TValue']; ArrayType: typeof Int32Array; }
 class Time_<T extends Times = Times> extends DataType<T> {
-    constructor(protected _unit: TimesType[T]['unit'],
-                protected _bitWidth: TimeBitWidth) {
+    constructor(public readonly unit: TimesType[T]['unit'],
+                public readonly bitWidth: TimeBitWidth) {
         super();
     }
     public get typeId() { return Type.Time as T; }
-    public get unit() { return this._unit; }
-    public get bitWidth() { return this._bitWidth; }
-    public toString() { return `Time${this._bitWidth}<${TimeUnit[this._unit]}>`; }
+    public toString() { return `Time${this.bitWidth}<${TimeUnit[this.unit]}>`; }
     protected static [Symbol.toStringTag] = ((proto: Time_) => {
+        (<any> proto).unit = null;
+        (<any> proto).bitWidth = null;
         (<any> proto).ArrayType = Int32Array;
         return proto[Symbol.toStringTag] = 'Time';
     })(Time_.prototype);
@@ -297,14 +308,15 @@ export class TimeNanosecond extends Time_<Type.TimeNanosecond> { constructor() {
 type Timestamps = Type.Timestamp | Type.TimestampSecond | Type.TimestampMillisecond | Type.TimestampMicrosecond | Type.TimestampNanosecond;
 interface Timestamp_<T extends Timestamps = Timestamps> extends DataType<T> { TArray: Int32Array; TValue: number; ArrayType: typeof Int32Array; }
 class Timestamp_<T extends Timestamps = Timestamps> extends DataType<T> {
-    constructor(protected _unit: TimeUnit, protected _timezone?: string | null) {
+    constructor(public readonly unit: TimeUnit,
+                public readonly timezone?: string | null) {
         super();
     }
     public get typeId() { return Type.Timestamp as T; }
-    public get unit() { return this._unit; }
-    public get timezone() { return this._timezone; }
-    public toString() { return `Timestamp<${TimeUnit[this._unit]}${this._timezone ? `, ${this._timezone}` : ``}>`; }
+    public toString() { return `Timestamp<${TimeUnit[this.unit]}${this.timezone ? `, ${this.timezone}` : ``}>`; }
     protected static [Symbol.toStringTag] = ((proto: Timestamp_) => {
+        (<any> proto).unit = null;
+        (<any> proto).timezone = null;
         (<any> proto).ArrayType = Int32Array;
         return proto[Symbol.toStringTag] = 'Timestamp';
     })(Timestamp_.prototype);
@@ -321,13 +333,13 @@ export class TimestampNanosecond extends Timestamp_<Type.TimestampNanosecond> {
 type Intervals = Type.Interval | Type.IntervalDayTime | Type.IntervalYearMonth;
 interface Interval_<T extends Intervals = Intervals> extends DataType<T> { TArray: Int32Array; TValue: Int32Array; ArrayType: typeof Int32Array; }
 class Interval_<T extends Intervals = Intervals> extends DataType<T> {
-    constructor(protected _unit: IntervalUnit) {
+    constructor(public readonly unit: IntervalUnit) {
         super();
     }
     public get typeId() { return Type.Interval as T; }
-    public get unit() { return this._unit; }
-    public toString() { return `Interval<${IntervalUnit[this._unit]}>`; }
+    public toString() { return `Interval<${IntervalUnit[this.unit]}>`; }
     protected static [Symbol.toStringTag] = ((proto: Interval_) => {
+        (<any> proto).unit = null;
         (<any> proto).ArrayType = Int32Array;
         return proto[Symbol.toStringTag] = 'Interval';
     })(Interval_.prototype);
@@ -341,30 +353,31 @@ export class IntervalYearMonth extends Interval_<Type.IntervalYearMonth> { const
 export interface List<T extends DataType = any> extends DataType<Type.List, { [0]: T }>  { TArray: IterableArrayLike<T>; TValue: VType<T>; }
 export class List<T extends DataType = any> extends DataType<Type.List, { [0]: T }> {
     constructor(child: Field<T>) {
-        super([child]);
+        super();
+        this.children = [child];
     }
+    public readonly children: Field<T>[];
     public get typeId() { return Type.List as Type.List; }
-    // @ts-ignore
-    protected _children: Field<T>[];
     public toString() { return `List<${this.valueType}>`; }
-    public get children() { return this._children; }
-    public get valueType(): T { return this._children[0].type as T; }
-    public get valueField(): Field<T> { return this._children[0] as Field<T>; }
+    public get valueType(): T { return this.children[0].type as T; }
+    public get valueField(): Field<T> { return this.children[0] as Field<T>; }
     public get ArrayType(): T['ArrayType'] { return this.valueType.ArrayType; }
     protected static [Symbol.toStringTag] = ((proto: List) => {
+        (<any> proto).children = null;
         return proto[Symbol.toStringTag] = 'List';
     })(List.prototype);
 }
 
 export interface Struct<T extends { [key: string]: DataType } = any> extends DataType<Type.Struct> { TArray: IterableArrayLike<RowLike<T>>; TValue: RowLike<T>; dataTypes: T; }
 export class Struct<T extends { [key: string]: DataType } = any> extends DataType<Type.Struct, T> {
-    constructor(protected _children: Field<T[keyof T]>[]) {
-        super(_children);
+    constructor(public readonly children: Field<T[keyof T]>[]) {
+        super();
+        this.children = children;
     }
     public get typeId() { return Type.Struct as Type.Struct; }
-    public get children() { return this._children; }
-    public toString() { return `Struct<[${this._children.map((f) => f.type).join(`, `)}]>`; }
+    public toString() { return `Struct<[${this.children.map((f) => f.type).join(`, `)}]>`; }
     protected static [Symbol.toStringTag] = ((proto: Struct) => {
+        (<any> proto).children = null;
         return proto[Symbol.toStringTag] = 'Struct';
     })(Struct.prototype);
 }
@@ -373,26 +386,30 @@ export class Struct<T extends { [key: string]: DataType } = any> extends DataTyp
 type Unions = Type.Union | Type.DenseUnion | Type.SparseUnion;
 interface Union_<T extends Unions = Unions> extends DataType<T> { TArray: Int32Array; TValue: any[]; }
 class Union_<T extends Unions = Unions> extends DataType<T> {
-    protected _typeIds: Int32Array;
-    protected _children: Field<any>[];
-    protected _typeIdToChildIndex: { [key: number]: number };
-    constructor(protected _mode: UnionMode, _typeIds: number[] | Int32Array, _children: Field<any>[]) {
-        super(_children);
-        this._children = _children;
-        this._typeIds = _typeIds = Int32Array.from(_typeIds);
-        this._typeIdToChildIndex = _typeIds.reduce((typeIdToChildIndex, typeId, idx) => {
+    public readonly mode: UnionMode;
+    public readonly typeIds: Int32Array;
+    public readonly children: Field<any>[];
+    public readonly typeIdToChildIndex: { [key: number]: number };
+    constructor(mode: UnionMode,
+                typeIds: number[] | Int32Array,
+                children: Field<any>[]) {
+        super();
+        this.mode = mode;
+        this.children = children;
+        this.typeIds = typeIds = Int32Array.from(typeIds);
+        this.typeIdToChildIndex = typeIds.reduce((typeIdToChildIndex, typeId, idx) => {
             return (typeIdToChildIndex[typeId] = idx) && typeIdToChildIndex || typeIdToChildIndex;
         }, Object.create(null) as { [key: number]: number });
     }
     public get typeId() { return Type.Union as T; }
-    public get mode() { return this._mode; }
-    public get typeIds() { return this._typeIds; }
-    public get children() { return this._children; }
-    public get typeIdToChildIndex() { return this._typeIdToChildIndex; }
     public toString() { return `${this[Symbol.toStringTag]}<${
-        this._children.map((x) => `${x.type}`).join(` | `)
+        this.children.map((x) => `${x.type}`).join(` | `)
     }>`; }
     protected static [Symbol.toStringTag] = ((proto: Union_) => {
+        (<any> proto).mode = null;
+        (<any> proto).typeIds = null;
+        (<any> proto).children = null;
+        (<any> proto).typeIdToChildIndex = null;
         (<any> proto).ArrayType = Int32Array;
         return proto[Symbol.toStringTag] = 'Union';
     })(Union_.prototype);
@@ -414,13 +431,13 @@ export class SparseUnion extends Union_<Type.SparseUnion> {
 
 export interface FixedSizeBinary extends DataType<Type.FixedSizeBinary> { TArray: Uint8Array; TValue: Uint8Array; ArrayType: typeof Uint8Array; }
 export class FixedSizeBinary extends DataType<Type.FixedSizeBinary> {
-    constructor(protected _byteWidth: number) {
+    constructor(public readonly byteWidth: number) {
         super();
     }
     public get typeId() { return Type.FixedSizeBinary as Type.FixedSizeBinary; }
-    public get byteWidth() { return this._byteWidth; }
     public toString() { return `FixedSizeBinary[${this.byteWidth}]`; }
     protected static [Symbol.toStringTag] = ((proto: FixedSizeBinary) => {
+        (<any> proto).byteWidth = null;
         (<any> proto).ArrayType = Uint8Array;
         return proto[Symbol.toStringTag] = 'FixedSizeBinary';
     })(FixedSizeBinary.prototype);
@@ -428,34 +445,34 @@ export class FixedSizeBinary extends DataType<Type.FixedSizeBinary> {
 
 export interface FixedSizeList<T extends DataType = any> extends DataType<Type.FixedSizeList> { TArray: IterableArrayLike<T['TArray']>; TValue: VType<T>; }
 export class FixedSizeList<T extends DataType = any> extends DataType<Type.FixedSizeList, { [0]: T }> {
-    constructor(protected _listSize: number, child: Field<T>) {
-        super([child]);
+    public readonly children: Field<T>[];
+    constructor(public readonly listSize: number, child: Field<T>) {
+        super();
+        this.children = [child];
     }
     public get typeId() { return Type.FixedSizeList as Type.FixedSizeList; }
-    // @ts-ignore
-    protected _children: Field<T>[];
-    public get listSize() { return this._listSize; }
-    public get children() { return this._children; }
     public get valueType(): T { return this.children[0].type as T; }
     public get valueField(): Field<T> { return this.children[0] as Field<T>; }
     public get ArrayType(): T['ArrayType'] { return this.valueType.ArrayType; }
-    public toString() { return `FixedSizeList[${this._listSize}]<${this.valueType}>`; }
+    public toString() { return `FixedSizeList[${this.listSize}]<${this.valueType}>`; }
     protected static [Symbol.toStringTag] = ((proto: FixedSizeList) => {
+        (<any> proto).children = null;
+        (<any> proto).listSize = null;
         return proto[Symbol.toStringTag] = 'FixedSizeList';
     })(FixedSizeList.prototype);
 }
 
 export interface Map_<T extends { [key: string]: DataType } = any> extends DataType<Type.Map> { TArray: Uint8Array; TValue: RowLike<T>; dataTypes: T; }
 export class Map_<T extends { [key: string]: DataType } = any> extends DataType<Type.Map, T> {
-    constructor(protected _children: Field<T[keyof T]>[],
-                protected _keysSorted: boolean = false) {
-        super(_children);
+    constructor(public readonly children: Field<T[keyof T]>[],
+                public readonly keysSorted: boolean = false) {
+        super();
     }
     public get typeId() { return Type.Map as Type.Map; }
-    public get children() { return this._children; }
-    public get keysSorted() { return this._keysSorted; }
-    public toString() { return `Map<{${this._children.map((f) => `${f.name}:${f.type}`).join(`, `)}}>`; }
+    public toString() { return `Map<{${this.children.map((f) => `${f.name}:${f.type}`).join(`, `)}}>`; }
     protected static [Symbol.toStringTag] = ((proto: Map_) => {
+        (<any> proto).children = null;
+        (<any> proto).keysSorted = null;
         return proto[Symbol.toStringTag] = 'Map_';
     })(Map_.prototype);
 }
@@ -468,32 +485,30 @@ export type TKeys = Int8 | Int16 | Int32 | Uint8 | Uint16 | Uint32;
 
 export interface Dictionary<T extends DataType = any, TKey extends TKeys = TKeys> extends DataType<Type.Dictionary> { TArray: TKey['TArray']; TValue: T['TValue']; }
 export class Dictionary<T extends DataType = any, TKey extends TKeys = TKeys> extends DataType<Type.Dictionary> {
-    protected _id: number;
-    protected _indices: TKey;
-    protected _dictionary: T;
-    protected _isOrdered: boolean;
-    protected _dictionaryVector: Vector<T>;
-    public set dictionaryVector(v) { this._dictionaryVector = v; }
-    public get dictionaryVector() { return this._dictionaryVector; }
+    public readonly id: number;
+    public readonly indices: TKey;
+    public readonly dictionary: T;
+    public readonly isOrdered: boolean;
+    public dictionaryVector: Vector<T>;
     constructor(dictionary: T, indices: TKey, id?: Long | number | null, isOrdered?: boolean | null, dictionaryVector?: Vector<T>) {
         super();
-        this._indices = indices;
-        this._dictionary = dictionary;
-        this._isOrdered = isOrdered || false;
-        this._dictionaryVector = dictionaryVector!;
-        this._id = id == null ? getId() : typeof id === 'number' ? id : id.low;
+        this.indices = indices;
+        this.dictionary = dictionary;
+        this.isOrdered = isOrdered || false;
+        this.dictionaryVector = dictionaryVector!;
+        this.id = id == null ? getId() : typeof id === 'number' ? id : id.low;
     }
     public get typeId() { return Type.Dictionary as Type.Dictionary; }
-    public get id() { return this._id; }
-    public get indices() { return this._indices; }
-    public get dictionary() { return this._dictionary; }
-    public get isOrdered() { return this._isOrdered; }
-    public set children(_: T['children']) {}
     public get children() { return this.dictionary.children; }
     public get valueType(): T { return this.dictionary as T; }
     public get ArrayType(): T['ArrayType'] { return this.dictionary.ArrayType; }
     public toString() { return `Dictionary<${this.indices}, ${this.dictionary}>`; }
     protected static [Symbol.toStringTag] = ((proto: Dictionary) => {
+        (<any> proto).id = null;
+        (<any> proto).indices = null;
+        (<any> proto).isOrdered = null;
+        (<any> proto).dictionary = null;
+        (<any> proto).dictionaryVector = null;
         return proto[Symbol.toStringTag] = 'Dictionary';
     })(Dictionary.prototype);
 }
diff --git a/js/src/util/bit.ts b/js/src/util/bit.ts
index a87078f..00d860f 100644
--- a/js/src/util/bit.ts
+++ b/js/src/util/bit.ts
@@ -37,11 +37,10 @@ export function truncateBitmap(offset: number, length: number, bitmap: Uint8Arra
     const alignedSize = (bitmap.byteLength + 7) & ~7;
     if (offset > 0 || bitmap.byteLength < alignedSize) {
         const bytes = new Uint8Array(alignedSize);
-        bytes.set((offset % 8 === 0)
-            // If the offset is a multiple of 8 bits, it's safe to slice the bitmap
-            ? bitmap.subarray(offset >> 3)
+        // If the offset is a multiple of 8 bits, it's safe to slice the bitmap
+        bytes.set(offset % 8 === 0 ? bitmap.subarray(offset >> 3) :
             // Otherwise iterate each bit from the offset and return a new one
-            : packBools(iterateBits(bitmap, offset, length, null, getBool)));
+            packBools(iterateBits(bitmap, offset, length, null, getBool)).subarray(0, alignedSize));
         return bytes;
     }
     return bitmap;
@@ -49,9 +48,8 @@ export function truncateBitmap(offset: number, length: number, bitmap: Uint8Arra
 
 /** @ignore */
 export function packBools(values: Iterable<any>) {
-    let n = 0, i = 0;
     let xs: number[] = [];
-    let bit = 0, byte = 0;
+    let i = 0, bit = 0, byte = 0;
     for (const value of values) {
         value && (byte |= 1 << bit);
         if (++bit === 8) {
@@ -60,10 +58,9 @@ export function packBools(values: Iterable<any>) {
         }
     }
     if (i === 0 || bit > 0) { xs[i++] = byte; }
-    if (i % 8 && (n = i + 8 - i % 8)) {
-        do { xs[i] = 0; } while (++i < n);
-    }
-    return new Uint8Array(xs);
+    let b = new Uint8Array((xs.length + 7) & ~7);
+    b.set(xs);
+    return b;
 }
 
 /** @ignore */
diff --git a/js/src/util/bn.ts b/js/src/util/bn.ts
index c49c342..01b7c27 100644
--- a/js/src/util/bn.ts
+++ b/js/src/util/bn.ts
@@ -15,7 +15,9 @@
 // specific language governing permissions and limitations
 // under the License.
 
+import { BigIntArray, BigIntArrayConstructor } from '../interfaces';
 import { toArrayBufferView, ArrayBufferViewInput } from './buffer';
+import { BigIntAvailable, BigInt64Array, BigUint64Array } from './compat';
 
 /** @ignore */
 type BigNumArray = IntArray | UintArray;
@@ -26,21 +28,23 @@ type UintArray = Uint8Array | Uint16Array | Uint32Array | Uint8ClampedArray;
 
 /** @ignore */
 const BigNumNMixin = {
-    toJSON(this: BN<BigNumArray>, ) { return `"${bignumToString(this)}"`; },
-    valueOf(this: BN<BigNumArray>, ) { return bignumToNumber(this); },
-    toString(this: BN<BigNumArray>, ) { return bignumToString(this); },
+    toJSON(this: BN<BigNumArray>) { return `"${bignumToString(this)}"`; },
+    valueOf(this: BN<BigNumArray>) { return bignumToNumber(this); },
+    toString(this: BN<BigNumArray>) { return bignumToString(this); },
     [Symbol.toPrimitive]<T extends BN<BigNumArray>>(this: T, hint: 'string' | 'number' | 'default') {
-        if (hint === 'number') { return bignumToNumber(this); }
-        /** @suppress {missingRequire} */
-        return hint === 'string' || typeof BigInt !== 'function' ?
-            bignumToString(this) : BigInt(bignumToString(this));
+        switch (hint) {
+            case 'number': return bignumToNumber(this);
+            case 'string': return bignumToString(this);
+            case 'default': return bignumToBigInt(this);
+        }
+        return bignumToString(this);
     }
 };
 
 /** @ignore */
-const SignedBigNumNMixin: any = Object.assign({}, BigNumNMixin, { signed: true });
+const SignedBigNumNMixin: any = Object.assign({}, BigNumNMixin, { signed: true, BigIntArray: BigInt64Array });
 /** @ignore */
-const UnsignedBigNumNMixin: any = Object.assign({}, BigNumNMixin, { signed: false });
+const UnsignedBigNumNMixin: any = Object.assign({}, BigNumNMixin, { signed: false, BigIntArray: BigUint64Array });
 
 /** @ignore */
 export class BN<T extends BigNumArray> {
@@ -74,6 +78,7 @@ export interface BN<T extends BigNumArray> extends TypedArrayLike<T> {
     new<T extends ArrayBufferViewInput>(buffer: T, signed?: boolean): T;
 
     readonly signed: boolean;
+    readonly BigIntArray: BigIntArrayConstructor<BigIntArray>;
 
     [Symbol.toStringTag]:
         'Int8Array'         |
@@ -108,20 +113,29 @@ function bignumToNumber<T extends BN<BigNumArray>>({ buffer, byteOffset, length
     let words = new Uint32Array(buffer, byteOffset, length);
     for (let i = 0, n = words.length; i < n;) {
         int64 += words[i++] + (words[i++] * (i ** 32));
-        // int64 += (words[i++] >>> 0) + (words[i++] * (i ** 32));
     }
     return int64;
 }
 
 /** @ignore */
-function bignumToString<T extends BN<BigNumArray>>({ buffer, byteOffset, length }: T) {
+let bignumToString: { <T extends BN<BigNumArray>>(a: T): string; };
+/** @ignore */
+let bignumToBigInt: { <T extends BN<BigNumArray>>(a: T): bigint; };
+
+if (!BigIntAvailable) {
+    bignumToString = decimalToString;
+    bignumToBigInt = <any> bignumToString;
+} else {
+    bignumToBigInt = (<T extends BN<BigNumArray>>(a: T) => a.length === 2 ? new a.BigIntArray(a.buffer, a.byteOffset, 1)[0] : <any>decimalToString(a));
+    bignumToString = (<T extends BN<BigNumArray>>(a: T) => a.length === 2 ? `${new a.BigIntArray(a.buffer, a.byteOffset, 1)[0]}` : decimalToString(a));
+}
 
-    let string = '', i = -1;
+function decimalToString<T extends BN<BigNumArray>>(a: T) {
+    let digits = '';
     let base64 = new Uint32Array(2);
-    let base32 = new Uint16Array(buffer, byteOffset, length * 2);
+    let base32 = new Uint16Array(a.buffer, a.byteOffset, a.length * 2);
     let checks = new Uint32Array((base32 = new Uint16Array(base32).reverse()).buffer);
-    let n = base32.length - 1;
-
+    let i = -1, n = base32.length - 1;
     do {
         for (base64[0] = base32[i = 0]; i < n;) {
             base32[i++] = base64[1] = base64[0] / 10;
@@ -129,10 +143,9 @@ function bignumToString<T extends BN<BigNumArray>>({ buffer, byteOffset, length
         }
         base32[i] = base64[1] = base64[0] / 10;
         base64[0] = base64[0] - base64[1] * 10;
-        string = `${base64[0]}${string}`;
+        digits = `${base64[0]}${digits}`;
     } while (checks[0] || checks[1] || checks[2] || checks[3]);
-
-    return string ? string : `0`;
+    return digits ? digits : `0`;
 }
 
 /** @ignore */
diff --git a/js/src/util/buffer.ts b/js/src/util/buffer.ts
index 13b3f90..e0fb0fd 100644
--- a/js/src/util/buffer.ts
+++ b/js/src/util/buffer.ts
@@ -18,8 +18,9 @@
 import { flatbuffers } from 'flatbuffers';
 import { encodeUtf8 } from '../util/utf8';
 import ByteBuffer = flatbuffers.ByteBuffer;
-import { ArrayBufferViewConstructor } from '../interfaces';
-import { isPromise, isIterable, isAsyncIterable, isIteratorResult } from './compat';
+import { TypedArray, TypedArrayConstructor } from '../interfaces';
+import { BigIntArray, BigIntArrayConstructor } from '../interfaces';
+import { isPromise, isIterable, isAsyncIterable, isIteratorResult, BigInt64Array, BigUint64Array } from './compat';
 
 /** @ignore */
 const SharedArrayBuf = (typeof SharedArrayBuffer !== 'undefined' ? SharedArrayBuffer : ArrayBuffer);
@@ -92,7 +93,9 @@ export type ArrayBufferViewInput = ArrayBufferView | ArrayBufferLike | ArrayBuff
           ReadableStreamReadResult<ArrayBufferView | ArrayBufferLike | ArrayBufferView | Iterable<number> | ArrayLike<number> | ByteBuffer | string | null | undefined> ;
 
 /** @ignore */
-export function toArrayBufferView<T extends ArrayBufferView>(ArrayBufferViewCtor: ArrayBufferViewConstructor<T>, input: ArrayBufferViewInput): T {
+export function toArrayBufferView<T extends TypedArray>(ArrayBufferViewCtor: TypedArrayConstructor<T>, input: ArrayBufferViewInput): T;
+export function toArrayBufferView<T extends BigIntArray>(ArrayBufferViewCtor: BigIntArrayConstructor<T>, input: ArrayBufferViewInput): T;
+export function toArrayBufferView(ArrayBufferViewCtor: any, input: ArrayBufferViewInput) {
 
     let value: any = isIteratorResult(input) ? input.value : input;
 
@@ -114,21 +117,44 @@ export function toArrayBufferView<T extends ArrayBufferView>(ArrayBufferViewCtor
 /** @ignore */ export const toInt8Array = (input: ArrayBufferViewInput) => toArrayBufferView(Int8Array, input);
 /** @ignore */ export const toInt16Array = (input: ArrayBufferViewInput) => toArrayBufferView(Int16Array, input);
 /** @ignore */ export const toInt32Array = (input: ArrayBufferViewInput) => toArrayBufferView(Int32Array, input);
+/** @ignore */ export const toBigInt64Array = (input: ArrayBufferViewInput) => toArrayBufferView(BigInt64Array, input);
 /** @ignore */ export const toUint8Array = (input: ArrayBufferViewInput) => toArrayBufferView(Uint8Array, input);
 /** @ignore */ export const toUint16Array = (input: ArrayBufferViewInput) => toArrayBufferView(Uint16Array, input);
 /** @ignore */ export const toUint32Array = (input: ArrayBufferViewInput) => toArrayBufferView(Uint32Array, input);
+/** @ignore */ export const toBigUint64Array = (input: ArrayBufferViewInput) => toArrayBufferView(BigUint64Array, input);
 /** @ignore */ export const toFloat32Array = (input: ArrayBufferViewInput) => toArrayBufferView(Float32Array, input);
 /** @ignore */ export const toFloat64Array = (input: ArrayBufferViewInput) => toArrayBufferView(Float64Array, input);
 /** @ignore */ export const toUint8ClampedArray = (input: ArrayBufferViewInput) => toArrayBufferView(Uint8ClampedArray, input);
 
 /** @ignore */
+export const toFloat16Array = (input: ArrayBufferViewInput) => {
+    let floats: Float32Array | Float64Array | null = null;
+    if (ArrayBuffer.isView(input)) {
+        switch (input.constructor) {
+            case Float32Array: floats = input as Float32Array; break;
+            case Float64Array: floats = input as Float64Array; break;
+        }
+    } else if (isIterable(input)) {
+        floats = toFloat64Array(input);
+    }
+    if (floats) {
+        const u16s = new Uint16Array(floats.length);
+        for (let i = -1, n = u16s.length; ++i < n;) {
+            u16s[i] = (floats[i] * 32767) + 32767;
+        }
+        return u16s;
+    }
+    return toUint16Array(input);
+};
+
+/** @ignore */
 type ArrayBufferViewIteratorInput = Iterable<ArrayBufferViewInput> | ArrayBufferViewInput;
 
 /** @ignore */
 const pump = <T extends Iterator<any> | AsyncIterator<any>>(iterator: T) => { iterator.next(); return iterator; };
 
 /** @ignore */
-export function* toArrayBufferViewIterator<T extends ArrayBufferView>(ArrayCtor: ArrayBufferViewConstructor<T>, source: ArrayBufferViewIteratorInput) {
+export function* toArrayBufferViewIterator<T extends TypedArray>(ArrayCtor: TypedArrayConstructor<T>, source: ArrayBufferViewIteratorInput) {
 
     const wrap = function*<T>(x: T) { yield x; };
     const buffers: Iterable<ArrayBufferViewInput> =
@@ -160,7 +186,7 @@ export function* toArrayBufferViewIterator<T extends ArrayBufferView>(ArrayCtor:
 type ArrayBufferViewAsyncIteratorInput = AsyncIterable<ArrayBufferViewInput> | Iterable<ArrayBufferViewInput> | PromiseLike<ArrayBufferViewInput> | ArrayBufferViewInput;
 
 /** @ignore */
-export async function* toArrayBufferViewAsyncIterator<T extends ArrayBufferView>(ArrayCtor: ArrayBufferViewConstructor<T>, source: ArrayBufferViewAsyncIteratorInput): AsyncIterableIterator<T> {
+export async function* toArrayBufferViewAsyncIterator<T extends TypedArray>(ArrayCtor: TypedArrayConstructor<T>, source: ArrayBufferViewAsyncIteratorInput): AsyncIterableIterator<T> {
 
     // if a Promise, unwrap the Promise and iterate the resolved value
     if (isPromise<ArrayBufferViewInput>(source)) {
diff --git a/js/src/util/compat.ts b/js/src/util/compat.ts
index 1e5a9c0..a43a4e7 100644
--- a/js/src/util/compat.ts
+++ b/js/src/util/compat.ts
@@ -40,6 +40,43 @@ export interface Observable<T> {
     subscribe: (observer: Observer<T>) => Subscription;
 }
 
+/** @ignore */
+const [BigIntCtor, BigIntAvailable] = (() => {
+    const BigIntUnavailableError = () => { throw new Error('BigInt is not available in this environment'); };
+    function BigIntUnavailable() { throw BigIntUnavailableError(); }
+    BigIntUnavailable.asIntN = () => { throw BigIntUnavailableError(); };
+    BigIntUnavailable.asUintN = () => { throw BigIntUnavailableError(); };
+    return typeof BigInt !== 'undefined' ? [BigInt, true] : [<any> BigIntUnavailable, false];
+})() as [BigIntConstructor, boolean];
+
+/** @ignore */
+const [BigInt64ArrayCtor, BigInt64ArrayAvailable] = (() => {
+    const BigInt64ArrayUnavailableError = () => { throw new Error('BigInt64Array is not available in this environment'); };
+    class BigInt64ArrayUnavailable {
+        static get BYTES_PER_ELEMENT() { return 8; }
+        static of() { throw BigInt64ArrayUnavailableError(); }
+        static from() { throw BigInt64ArrayUnavailableError(); }
+        constructor() { throw BigInt64ArrayUnavailableError(); }
+    }
+    return typeof BigInt64Array !== 'undefined' ? [BigInt64Array, true] : [<any> BigInt64ArrayUnavailable, false];
+})() as [BigInt64ArrayConstructor, boolean];
+
+/** @ignore */
+const [BigUint64ArrayCtor, BigUint64ArrayAvailable] = (() => {
+    const BigUint64ArrayUnavailableError = () => { throw new Error('BigUint64Array is not available in this environment'); };
+    class BigUint64ArrayUnavailable {
+        static get BYTES_PER_ELEMENT() { return 8; }
+        static of() { throw BigUint64ArrayUnavailableError(); }
+        static from() { throw BigUint64ArrayUnavailableError(); }
+        constructor() { throw BigUint64ArrayUnavailableError(); }
+    }
+    return typeof BigUint64Array !== 'undefined' ? [BigUint64Array, true] : [<any> BigUint64ArrayUnavailable, false];
+})() as [BigUint64ArrayConstructor, boolean];
+
+export { BigIntCtor as BigInt, BigIntAvailable };
+export { BigInt64ArrayCtor as BigInt64Array, BigInt64ArrayAvailable };
+export { BigUint64ArrayCtor as BigUint64Array, BigUint64ArrayAvailable };
+
 /** @ignore */ const isNumber = (x: any) => typeof x === 'number';
 /** @ignore */ const isBoolean = (x: any) => typeof x === 'boolean';
 /** @ignore */ const isFunction = (x: any) => typeof x === 'function';
diff --git a/js/src/util/vector.ts b/js/src/util/vector.ts
index 8acf42e..c1ce3cd 100644
--- a/js/src/util/vector.ts
+++ b/js/src/util/vector.ts
@@ -18,6 +18,7 @@
 import { Vector } from '../vector';
 import { Row, kLength } from '../vector/row';
 import { compareArrayLike } from '../util/buffer';
+import { BigInt, BigIntAvailable } from './compat';
 
 /** @ignore */
 type RangeLike = { length: number; stride?: number };
@@ -59,11 +60,16 @@ export function clampRange<T extends RangeLike, N extends ClampRangeThen<T> = Cl
     return then ? then(source, lhs, rhs) : [lhs, rhs];
 }
 
+const big0 = BigIntAvailable ? BigInt(0) : 0;
+
 /** @ignore */
 export function createElementComparator(search: any) {
+    let typeofSearch = typeof search;
     // Compare primitives
-    if (search == null || typeof search !== 'object') {
-        return (value: any) => value === search;
+    if (typeofSearch !== 'object' || search === null) {
+        return typeofSearch !== 'bigint'
+            ? (value: any) => value === search
+            : (value: any) => (big0 + value) === search;
     }
     // Compare Dates
     if (search instanceof Date) {
diff --git a/js/src/vector/float.ts b/js/src/vector/float.ts
index 4c01736..56a2bfb 100644
--- a/js/src/vector/float.ts
+++ b/js/src/vector/float.ts
@@ -18,23 +18,49 @@
 import { Data } from '../data';
 import { Vector } from '../vector';
 import { BaseVector } from './base';
+import { Vector as V } from '../interfaces';
 import { Float, Float16, Float32, Float64 } from '../type';
+import { toFloat16Array, toFloat32Array, toFloat64Array } from '../util/buffer';
 
 export class FloatVector<T extends Float = Float> extends BaseVector<T> {
-    public static from<T extends Float16>(data: T['TArray']): Float16Vector;
-    public static from<T extends Float32>(data: T['TArray']): Float32Vector;
-    public static from<T extends Float64>(data: T['TArray']): Float64Vector;
+
+    public static from(this: typeof FloatVector, data: Float16['TArray']): Float16Vector;
+    public static from(this: typeof FloatVector, data: Float32['TArray']): Float32Vector;
+    public static from(this: typeof FloatVector, data: Float64['TArray']): Float64Vector;
+    public static from<T extends Float>(this: typeof FloatVector, data: T['TArray']): V<T>;
+
+    public static from(this: typeof Float16Vector, data: Float16['TArray'] | Iterable<number>): Float16Vector;
+    public static from(this: typeof Float32Vector, data: Float32['TArray'] | Iterable<number>): Float32Vector;
+    public static from(this: typeof Float64Vector, data: Float64['TArray'] | Iterable<number>): Float64Vector;
     /** @nocollapse */
     public static from<T extends Float>(data: T['TArray']) {
+        let type: Float | null = null;
+        switch (this) {
+            case Float16Vector: data = toFloat16Array(data); break;
+            case Float32Vector: data = toFloat32Array(data); break;
+            case Float64Vector: data = toFloat64Array(data); break;
+        }
         switch (data.constructor) {
-            case Uint16Array: return Vector.new(Data.Float(new Float16(), 0, data.length, 0, null, data));
-            case Float32Array: return Vector.new(Data.Float(new Float32(), 0, data.length, 0, null, data));
-            case Float64Array: return Vector.new(Data.Float(new Float64(), 0, data.length, 0, null, data));
+            case Uint16Array:  type = new Float16(); break;
+            case Float32Array: type = new Float32(); break;
+            case Float64Array: type = new Float64(); break;
         }
-        throw new TypeError('Unrecognized Float data');
+        return type !== null
+            ? Vector.new(Data.Float(type, 0, data.length, 0, null, data))
+            : (() => { throw new TypeError('Unrecognized FloatVector input'); })();
     }
 }
 
-export class Float16Vector extends FloatVector<Float16> {}
+export class Float16Vector extends FloatVector<Float16> {
+    // Since JS doesn't have half floats, `toArray()` returns a zero-copy slice
+    // of the underlying Uint16Array data. This behavior ensures we don't incur
+    // extra compute or copies if you're calling `toArray()` in order to create
+    // a buffer for something like WebGL. Buf if you're using JS and want typed
+    // arrays of 4-to-8-byte precision, these methods will enumerate the values
+    // and clamp to the desired byte lengths.
+    public toFloat32Array() { return new Float32Array(this as Iterable<number>); }
+    public toFloat64Array() { return new Float64Array(this as Iterable<number>); }
+}
+
 export class Float32Vector extends FloatVector<Float32> {}
 export class Float64Vector extends FloatVector<Float64> {}
diff --git a/js/src/vector/int.ts b/js/src/vector/int.ts
index b14a4d8..57f5cba 100644
--- a/js/src/vector/int.ts
+++ b/js/src/vector/int.ts
@@ -20,35 +20,91 @@ import { Vector } from '../vector';
 import { BaseVector } from './base';
 import { Vector as V } from '../interfaces';
 import { Int, Uint8, Uint16, Uint32, Uint64, Int8, Int16, Int32, Int64 } from '../type';
+import {
+    toInt8Array, toInt16Array, toInt32Array,
+    toUint8Array, toUint16Array, toUint32Array,
+    toBigInt64Array, toBigUint64Array
+} from '../util/buffer';
 
 export class IntVector<T extends Int = Int> extends BaseVector<T> {
-    public static from<T extends Int>(data: T['TArray']): V<T>;
-    public static from<T extends Int64>(data: T['TArray'], is64: true): V<T>;
-    public static from<T extends Uint64>(data: T['TArray'], is64: true): V<T>;
+
+    public static from(this: typeof IntVector, data: Int8Array): Int8Vector;
+    public static from(this: typeof IntVector, data: Int16Array): Int16Vector;
+    public static from(this: typeof IntVector, data: Int32Array): Int32Vector;
+    public static from(this: typeof IntVector, data: Uint8Array): Uint8Vector;
+    public static from(this: typeof IntVector, data: Uint16Array): Uint16Vector;
+    public static from(this: typeof IntVector, data: Uint32Array): Uint32Vector;
+
+    // @ts-ignore
+    public static from(this: typeof IntVector, data: Int32Array, is64: true): Int64Vector;
+    public static from(this: typeof IntVector, data: Uint32Array, is64: true): Uint64Vector;
+    public static from<T extends Int>(this: typeof IntVector, data: T['TArray']): V<T>;
+
+    public static from(this: typeof Int8Vector,   data: Int8['TArray']   | Iterable<number>): Int8Vector;
+    public static from(this: typeof Int16Vector,  data: Int16['TArray']  | Iterable<number>): Int16Vector;
+    public static from(this: typeof Int32Vector,  data: Int32['TArray']  | Iterable<number>): Int32Vector;
+    public static from(this: typeof Int64Vector,  data: Int32['TArray']  | Iterable<number>): Int64Vector;
+    public static from(this: typeof Uint8Vector,  data: Uint8['TArray']  | Iterable<number>): Uint8Vector;
+    public static from(this: typeof Uint16Vector, data: Uint16['TArray'] | Iterable<number>): Uint16Vector;
+    public static from(this: typeof Uint32Vector, data: Uint32['TArray'] | Iterable<number>): Uint32Vector;
+    public static from(this: typeof Uint64Vector, data: Uint32['TArray'] | Iterable<number>): Uint64Vector;
+
     /** @nocollapse */
-    public static from(data: any, is64?: boolean) {
-        if (is64 === true) {
-            return data instanceof Int32Array
-                ? Vector.new(Data.Int(new Int64(), 0, data.length * 0.5, 0, null, data))
-                : Vector.new(Data.Int(new Uint64(), 0, data.length * 0.5, 0, null, data));
+    public static from<T extends Int>(data: T['TArray'], is64?: boolean) {
+        let length: number = 0;
+        let type: Int | null = null;
+        switch (this) {
+            case Int8Vector:   data = toInt8Array(data);   is64 = false; break;
+            case Int16Vector:  data = toInt16Array(data);  is64 = false; break;
+            case Int32Vector:  data = toInt32Array(data);  is64 = false; break;
+            case Int64Vector:  data = toInt32Array(data);  is64 =  true; break;
+            case Uint8Vector:  data = toUint8Array(data);  is64 = false; break;
+            case Uint16Vector: data = toUint16Array(data); is64 = false; break;
+            case Uint32Vector: data = toUint32Array(data); is64 = false; break;
+            case Uint64Vector: data = toUint32Array(data); is64 =  true; break;
         }
-        switch (data.constructor) {
-            case Int8Array: return Vector.new(Data.Int(new Int8(), 0, data.length, 0, null, data));
-            case Int16Array: return Vector.new(Data.Int(new Int16(), 0, data.length, 0, null, data));
-            case Int32Array: return Vector.new(Data.Int(new Int32(), 0, data.length, 0, null, data));
-            case Uint8Array: return Vector.new(Data.Int(new Uint8(), 0, data.length, 0, null, data));
-            case Uint16Array: return Vector.new(Data.Int(new Uint16(), 0, data.length, 0, null, data));
-            case Uint32Array: return Vector.new(Data.Int(new Uint32(), 0, data.length, 0, null, data));
+        if (is64 === true) {
+            length = data.length * 0.5;
+            type = data instanceof Int32Array ? new Int64() : new Uint64();
+        } else {
+            length = data.length;
+            switch (data.constructor) {
+                case Int8Array:   type = new Int8();   break;
+                case Int16Array:  type = new Int16();  break;
+                case Int32Array:  type = new Int32();  break;
+                case Uint8Array:  type = new Uint8();  break;
+                case Uint16Array: type = new Uint16(); break;
+                case Uint32Array: type = new Uint32(); break;
+            }
         }
-        throw new TypeError('Unrecognized Int data');
+        return type !== null
+            ? Vector.new(Data.Int(type, 0, length, 0, null, data))
+            : (() => { throw new TypeError('Unrecognized IntVector input'); })();
     }
 }
 
 export class Int8Vector extends IntVector<Int8> {}
 export class Int16Vector extends IntVector<Int16> {}
 export class Int32Vector extends IntVector<Int32> {}
-export class Int64Vector extends IntVector<Int64> {}
+export class Int64Vector extends IntVector<Int64> {
+    public toBigInt64Array() {
+        return toBigInt64Array(this.values);
+    }
+}
+
 export class Uint8Vector extends IntVector<Uint8> {}
 export class Uint16Vector extends IntVector<Uint16> {}
 export class Uint32Vector extends IntVector<Uint32> {}
-export class Uint64Vector extends IntVector<Uint64> {}
+export class Uint64Vector extends IntVector<Uint64> {
+    public toBigUint64Array() {
+        return toBigUint64Array(this.values);
+    }
+}
+
+export interface Int64Vector extends IntVector<Int64> {
+    indexOf(value: Int64['TValue'] | bigint | null, fromIndex?: number): number;
+}
+
+export interface Uint64Vector extends IntVector<Uint64> {
+    indexOf(value: Uint64['TValue'] | bigint | null, fromIndex?: number): number;
+}
diff --git a/js/src/visitor/jsonvectorassembler.ts b/js/src/visitor/jsonvectorassembler.ts
index ab435d2..291a057 100644
--- a/js/src/visitor/jsonvectorassembler.ts
+++ b/js/src/visitor/jsonvectorassembler.ts
@@ -169,6 +169,6 @@ function* binaryToString(vector: Vector<Binary> | Vector<FixedSizeBinary>) {
 /** @ignore */
 function* bigNumsToStrings(values: Uint32Array | Int32Array, stride: number) {
     for (let i = -1, n = values.length / stride; ++i < n;) {
-        yield `${BN.new(values.subarray((i + 0) * stride, (i + 1) * stride))}`;
+        yield `${BN.new(values.subarray((i + 0) * stride, (i + 1) * stride), false)}`;
     }
 }
diff --git a/js/src/visitor/toarray.ts b/js/src/visitor/toarray.ts
index 4330550..0239329 100644
--- a/js/src/visitor/toarray.ts
+++ b/js/src/visitor/toarray.ts
@@ -16,9 +16,9 @@
 // under the License.
 
 import { Data } from '../data';
+import { Type } from '../enum';
 import { Visitor } from '../visitor';
 import { Vector } from '../interfaces';
-import { Type, Precision } from '../enum';
 import { instance as iteratorVisitor } from './iterator';
 import {
     DataType, Dictionary,
@@ -91,13 +91,10 @@ function arrayOfVector<T extends DataType>(vector: Vector<T>): T['TArray'] {
 
     // Fast case, return subarray if possible
     switch (type.typeId) {
-        case Type.Int: case Type.Decimal:
+        case Type.Int:
+        case Type.Float: case Type.Decimal:
         case Type.Time: case Type.Timestamp:
             return vector.values.subarray(0, length * stride);
-        case Type.Float:
-            return (type as Float).precision === Precision.HALF /* Precision.HALF */
-                ? new Float32Array(vector[Symbol.iterator]())
-                : vector.values.subarray(0, length * stride);
     }
 
     // Otherwise if not primitive, slow copy
diff --git a/js/test/unit/vector/float16-vector-tests.ts b/js/test/unit/vector/float16-vector-tests.ts
deleted file mode 100644
index 72bbf1b..0000000
--- a/js/test/unit/vector/float16-vector-tests.ts
+++ /dev/null
@@ -1,73 +0,0 @@
-// 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 { Data, Float16, Vector, util } from '../../Arrow';
-const { joinUint8Arrays } = util;
-
-const newFloat16Vector = (length: number, data: Uint16Array) => Vector.new(Data.Float(new Float16(), 0, length, 0, null, data));
-const randomBytes = (n: number) => Uint8Array.from({ length: n }, () => Math.random() * 255 | 0);
-const bytes = Array.from({ length: 5 }, () => randomBytes(64));
-
-describe('Float16Vector', () => {
-    const values = new Uint16Array(joinUint8Arrays(bytes)[0].buffer);
-    const vector = bytes
-        .map((b) => new Uint16Array(b.buffer))
-        .map((b) => newFloat16Vector(b.length, b))
-        .reduce((v: any, v2) => v.concat(v2));
-    const n = values.length;
-    const clamp = (x: number) => (x -  32767) / 32767;
-    const float16s = new Float32Array([...values].map((x) => clamp(x)));
-    test(`gets expected values`, () => {
-        let i = -1;
-        while (++i < n) {
-            expect(vector.get(i)).toEqual(clamp(values[i]));
-        }
-    });
-    test(`iterates expected values`, () => {
-        expect.hasAssertions();
-        let i = -1;
-        for (let v of vector) {
-            expect(++i).toBeLessThan(n);
-            expect(v).toEqual(clamp(values[i]));
-        }
-    });
-    test(`indexOf returns expected values`, () => {
-        const randomValues = new Uint16Array(randomBytes(64).buffer);
-        for (let value of [...values, ...randomValues]) {
-            const expected = values.indexOf(value);
-            expect(vector.indexOf(clamp(value))).toEqual(expected);
-        }
-    });
-    test(`slices the entire array`, () => {
-        expect(vector.slice().toArray()).toEqual(float16s);
-    });
-    test(`slice returns a TypedArray`, () => {
-        expect(vector.slice().toArray()).toBeInstanceOf(Float32Array);
-    });
-    test(`slices from -20 to length`, () => {
-        expect(vector.slice(-20).toArray()).toEqual(float16s.slice(-20));
-    });
-    test(`slices from 0 to -20`, () => {
-        expect(vector.slice(0, -20).toArray()).toEqual(float16s.slice(0, -20));
-    });
-    test(`slices the array from 0 to length - 20`, () => {
-        expect(vector.slice(0, n - 20).toArray()).toEqual(float16s.slice(0, n - 20));
-    });
-    test(`slices the array from 0 to length + 20`, () => {
-        expect(vector.slice(0, n + 20).toArray()).toEqual(float16s.slice(0, n + 20));
-    });
-});
diff --git a/js/test/unit/vector/numeric-vector-tests.ts b/js/test/unit/vector/numeric-vector-tests.ts
index 4a339ea..c178aae 100644
--- a/js/test/unit/vector/numeric-vector-tests.ts
+++ b/js/test/unit/vector/numeric-vector-tests.ts
@@ -17,73 +17,308 @@
 
 import {
     util,
-    DataType, Data, Vector,
-    Float, Float32, Float64,
-    Int, Int8, Int16, Int32, Int64,
-    Uint8, Uint16, Uint32, Uint64,
+    Data, Vector,
+    Float, Float16, Float32, Float64,
+    Int, Int8, Int16, Int32, Int64, Uint8, Uint16, Uint32, Uint64,
+    FloatVector, Float16Vector, Float32Vector, Float64Vector,
+    IntVector, Int8Vector, Int16Vector, Int32Vector, Int64Vector,
+    Uint8Vector, Uint16Vector, Uint32Vector, Uint64Vector,
 } from '../../Arrow';
 
+import { Vector as V } from '../../../src/interfaces';
+import { TypedArray, TypedArrayConstructor } from '../../../src/interfaces';
+import { BigIntArray, BigIntArrayConstructor } from '../../../src/interfaces';
+
 const { joinUint8Arrays } = util;
-const randomBytes = (n: number) => Uint8Array.from({ length: n }, () => Math.random() * 255 | 0);
-const bytes = Array.from({ length: 5 }, () => randomBytes(64));
-
-describe(`Int8Vector`, () => { testIntVector(Int8); });
-describe(`Int16Vector`, () => { testIntVector(Int16); });
-describe(`Int32Vector`, () => { testIntVector(Int32); });
-describe(`Int64Vector`, () => { testIntVector(Int64); });
-describe(`Uint64Vector`, () => { testIntVector(Uint64); });
-describe(`Uint8Vector`, () => { testIntVector(Uint8); });
-describe(`Uint16Vector`, () => { testIntVector(Uint16); });
-describe(`Uint32Vector`, () => { testIntVector(Uint32); });
-describe(`Float32Vector`, () => { testFloatVector(Float32); });
-describe(`Float64Vector`, () => { testFloatVector(Float64); });
-
-function testIntVector<T extends Int>(DataType: new () => T) {
+const uint16ToFloat64 = (x: number) => (x -  32767) / 32767;
+const uint16ToFloat64Array = (b: ArrayBuffer) => new Float64Array([...new Uint16Array(b)].map(uint16ToFloat64));
+const randomBytes = (n: number) => Uint16Array.from({ length: n / 2 }, () => (Math.random() * 65536) | 0).buffer;
+
+const testValueBuffers = Array.from({ length: 5 }, () => randomBytes(64));
+const testValuesBuffer = joinUint8Arrays(testValueBuffers.map((b) => new Uint8Array(b)))[0].buffer;
+
+const checkType = <T, R extends T>(Ctor: new (...args: any) => T, inst: R) => expect(inst).toBeInstanceOf(Ctor);
+const valuesArray = <T extends TypedArray>(ArrayType: TypedArrayConstructor<T>) => [...valuesTyped<T>(ArrayType)];
+const valuesTyped = <T extends TypedArray>(ArrayType: TypedArrayConstructor<T>) => new ArrayType(testValuesBuffer);
+const bigIntValuesTyped = <T extends BigIntArray>(ArrayType: BigIntArrayConstructor<T>) => new ArrayType(testValuesBuffer);
+const bigIntValuesArray = <T extends BigIntArray>(ArrayType: BigIntArrayConstructor<T>) => [...bigIntValuesTyped<T>(ArrayType)];
+
+describe(`FloatVector`, () => {
+
+    describe(`FloatVector.from infers the type from the input TypedArray`, () => {
+
+        const u16s = valuesTyped(Uint16Array);
+        const f16s = valuesArray(Uint16Array).map(uint16ToFloat64);
+        const f32s = valuesTyped(Float32Array);
+        const f64s = valuesTyped(Float64Array);
+        const f16Vec = FloatVector.from(u16s);
+        const f32Vec = FloatVector.from(valuesTyped(Float32Array));
+        const f64Vec = FloatVector.from(valuesTyped(Float64Array));
+
+        // test strong typing at compile-time
+        test(`return type is correct`, () => checkType(Float16Vector, f16Vec));
+        test(`return type is correct`, () => checkType(Float32Vector, f32Vec));
+        test(`return type is correct`, () => checkType(Float64Vector, f64Vec));
+        test(`throws on bad input`, () => {
+            expect(() => FloatVector.from(<any> {})).toThrow('Unrecognized FloatVector input');
+        });
+
+        testAndValidateVector(f16Vec, u16s, f16s);
+        testAndValidateVector(f32Vec, f32s);
+        testAndValidateVector(f64Vec, f64s);
+    });
+
+    describe(`Float16Vector`, () => {
+        testFloatVector(Float16, valuesArray(Uint16Array).map(uint16ToFloat64));
+        describe(`Float16Vector.from accepts regular Arrays`, () => {
+            const u16s = valuesTyped(Uint16Array);
+            const f16s = valuesArray(Uint16Array).map(uint16ToFloat64);
+            const vector = Float16Vector.from(f16s);
+            test(`return type is correct`, () => checkType(Float16Vector, vector));
+            testAndValidateVector(vector, u16s, f16s);
+        });
+        describe(`Float16Vector.from accepts Uint16Arrays`, () => {
+            const u16s = valuesTyped(Uint16Array);
+            const f16s = valuesArray(Uint16Array).map(uint16ToFloat64);
+            const vector = Float16Vector.from(u16s);
+            test(`return type is correct`, () => checkType(Float16Vector, vector));
+            testAndValidateVector(vector, u16s, f16s);
+        });
+    });
+    describe(`Float32Vector`, () => {
+        testFloatVector(Float32);
+        describe(`Float32Vector.from accepts regular Arrays`, () => {
+            const values = valuesArray(Float32Array);
+            const vector = Float32Vector.from(values);
+            testAndValidateVector(vector, valuesTyped(Float32Array), values);
+            test(`return type is correct`, () => checkType(Float32Vector, vector));
+        });
+    });
+    describe(`Float64Vector`, () => {
+        testFloatVector(Float64);
+        describe(`Float64Vector.from accepts regular Arrays`, () => {
+            const values = valuesArray(Float64Array);
+            const vector = Float64Vector.from(values);
+            testAndValidateVector(vector, valuesTyped(Float64Array), values);
+            test(`return type is correct`, () => checkType(Float64Vector, vector));
+        });
+    });
+});
+
+describe(`IntVector`, () => {
+
+    describe(`IntVector.from infers the type from the input TypedArray`, () => {
+
+        const i8s = valuesTyped(Int8Array);
+        const i16s = valuesTyped(Int16Array);
+        const i32s = valuesTyped(Int32Array);
+        const i64s = valuesTyped(Int32Array);
+        const u8s = valuesTyped(Uint8Array);
+        const u16s = valuesTyped(Uint16Array);
+        const u32s = valuesTyped(Uint32Array);
+        const u64s = valuesTyped(Uint32Array);
+        const i8Vec = IntVector.from(i8s);
+        const i16Vec = IntVector.from(i16s);
+        const i32Vec = IntVector.from(i32s);
+        const i64Vec = IntVector.from(i64s, true);
+        const u8Vec = IntVector.from(u8s);
+        const u16Vec = IntVector.from(u16s);
+        const u32Vec = IntVector.from(u32s);
+        const u64Vec = IntVector.from(u64s, true);
+
+        // test strong typing at compile-time
+        test(`return type is correct`, () => checkType(Int8Vector, i8Vec));
+        test(`return type is correct`, () => checkType(Int16Vector, i16Vec));
+        test(`return type is correct`, () => checkType(Int32Vector, i32Vec));
+        test(`return type is correct`, () => checkType(Int64Vector, i64Vec));
+        test(`return type is correct`, () => checkType(Uint8Vector, u8Vec));
+        test(`return type is correct`, () => checkType(Uint16Vector, u16Vec));
+        test(`return type is correct`, () => checkType(Uint32Vector, u32Vec));
+        test(`return type is correct`, () => checkType(Uint64Vector, u64Vec));
+        test(`throws on bad input`, () => {
+            expect(() => IntVector.from(<any> {})).toThrow('Unrecognized IntVector input');
+        });
+
+        testAndValidateVector(i8Vec, i8s);
+        testAndValidateVector(i16Vec, i16s);
+        testAndValidateVector(i32Vec, i32s);
+        testAndValidateVector(i64Vec, i64s);
+        testAndValidateVector(u8Vec, u8s);
+        testAndValidateVector(u16Vec, u16s);
+        testAndValidateVector(u32Vec, u32s);
+        testAndValidateVector(u64Vec, u64s);
+    });
+
+    describe(`Int8Vector`, () => {
+        testIntVector(Int8);
+        describe(`Int8Vector.from accepts regular Arrays`, () => {
+            const values = valuesArray(Int8Array);
+            const vector = Int8Vector.from(values);
+            testAndValidateVector(vector, valuesTyped(Int8Array), values);
+            test(`return type is correct`, () => checkType(Int8Vector, vector));
+        });
+    });
+    describe(`Int16Vector`, () => {
+        testIntVector(Int16);
+        describe(`Int16Vector.from accepts regular Arrays`, () => {
+            const values = valuesArray(Int16Array);
+            const vector = Int16Vector.from(values);
+            testAndValidateVector(vector, valuesTyped(Int16Array), values);
+            test(`return type is correct`, () => checkType(Int16Vector, vector));
+        });
+    });
+    describe(`Int32Vector`, () => {
+        testIntVector(Int32);
+        describe(`Int32Vector.from accepts regular Arrays`, () => {
+            const values = valuesArray(Int32Array);
+            const vector = Int32Vector.from(values);
+            testAndValidateVector(vector, valuesTyped(Int32Array), values);
+            test(`return type is correct`, () => checkType(Int32Vector, vector));
+        });
+    });
+    describe(`Int64Vector`, () => {
+        testIntVector(Int64);
+        testIntVector(Int64, bigIntValuesArray(BigInt64Array));
+        describe(`Int64Vector.from accepts regular Arrays`, () => {
+            const values = valuesArray(Int32Array);
+            const vector = Int64Vector.from(values);
+            testAndValidateVector(vector, valuesTyped(Int32Array), values);
+            test(`return type is correct`, () => checkType(Int64Vector, vector));
+        });
+    });
+    describe(`Uint8Vector`, () => {
+        testIntVector(Uint8);
+        describe(`Uint8Vector.from accepts regular Arrays`, () => {
+            const values = valuesArray(Uint8Array);
+            const vector = Uint8Vector.from(values);
+            testAndValidateVector(vector, valuesTyped(Uint8Array), values);
+            test(`return type is correct`, () => checkType(Uint8Vector, vector));
+        });
+    });
+    describe(`Uint16Vector`, () => {
+        testIntVector(Uint16);
+        describe(`Uint16Vector.from accepts regular Arrays`, () => {
+            const values = valuesArray(Uint16Array);
+            const vector = Uint16Vector.from(values);
+            testAndValidateVector(vector, valuesTyped(Uint16Array), values);
+            test(`return type is correct`, () => checkType(Uint16Vector, vector));
+        });
+    });
+    describe(`Uint32Vector`, () => {
+        testIntVector(Uint32);
+        describe(`Uint32Vector.from accepts regular Arrays`, () => {
+            const values = valuesArray(Uint32Array);
+            const vector = Uint32Vector.from(values);
+            testAndValidateVector(vector, valuesTyped(Uint32Array), values);
+            test(`return type is correct`, () => checkType(Uint32Vector, vector));
+        });
+    });
+    describe(`Uint64Vector`, () => {
+        testIntVector(Uint64);
+        testIntVector(Uint64, bigIntValuesArray(BigUint64Array));
+        describe(`Uint64Vector.from accepts regular Arrays`, () => {
+            const values = valuesArray(Uint32Array);
+            const vector = Uint64Vector.from(values);
+            testAndValidateVector(vector, valuesTyped(Uint32Array), values);
+            test(`return type is correct`, () => checkType(Uint64Vector, vector));
+        });
+    });
+});
+
+function testIntVector<T extends Int>(DataType: new () => T, values?: Array<any>) {
 
     const type = new DataType();
+    const ArrayType = type.ArrayType;
     const stride = type.bitWidth < 64 ? 1 : 2;
-    const values = new type.ArrayType(joinUint8Arrays(bytes)[0].buffer);
 
-    const vector = bytes
-        .map((b) => new type.ArrayType(b.buffer))
+    const typed = valuesTyped(ArrayType);
+    const jsArray = values || [...typed];
+    const vector = Vector.new(Data.Int(type, 0, typed.length / stride, 0, null, typed));
+    const chunked = testValueBuffers.map((b) => new ArrayType(b))
         .map((b) => Vector.new(Data.Int(type, 0, b.length / stride, 0, null, b)))
         .reduce((v: any, v2) => v.concat(v2));
 
-    gets_expected_values(vector, values);
-    iterates_expected_values(vector, values);
-    indexof_returns_expected_values(vector, values);
+    const vectorBegin = (vector.length * .25) | 0;
+    const vectorEnd = (vector.length * .75) | 0;
+    const typedBegin = vectorBegin * (typed.length / vector.length);
+    const typedEnd = vectorEnd * (typed.length / vector.length);
+    const jsArrayBegin = vectorBegin * (jsArray.length / vector.length);
+    const jsArrayEnd = vectorEnd * (jsArray.length / vector.length);
 
-    slice_returns_a_typedarray(vector);
-    slices_the_entire_array(vector, values);
-    slices_from_minus_20_to_length(vector, values);
-    slices_from_0_to_minus_20(vector, values);
-    slices_the_array_from_0_to_length_minus_20(vector, values);
-    slices_the_array_from_0_to_length_plus_20(vector, values);
+    const combos = [[`vector`, vector], [`chunked`, chunked]] as [string, V<T>][];
+    combos.forEach(([chunksType, vector]) => {
+        describe(chunksType, () => {
+            // test base case no slicing
+            describe(`base case no slicing`, () => testAndValidateVector(vector, typed, jsArray));
+            // test slicing without args
+            describe(`slicing without args`, () => testAndValidateVector(vector.slice(), typed.slice(), jsArray.slice()));
+            // test slicing the middle half
+            describe(`slice the middle half`, () => testAndValidateVector(
+                vector.slice(vectorBegin, vectorEnd),
+                typed.slice(typedBegin, typedEnd),
+                jsArray.slice(jsArrayBegin, jsArrayEnd)
+            ));
+            // test splicing out the middle half
+            describe(`splicing out the middle half`, () => testAndValidateVector(
+                vector.slice(0, vectorBegin).concat(vector.slice(vectorEnd)),
+                new ArrayType([...typed.slice(0, typedBegin), ...typed.slice(typedEnd)]),
+                [...jsArray.slice(0, jsArrayBegin), ...jsArray.slice(jsArrayEnd)]
+            ));
+        });
+    });
 }
 
-function testFloatVector<T extends Float>(DataType: new () => T) {
+function testFloatVector<T extends Float>(DataType: new () => T, values?: Array<any>) {
 
     const type = new DataType();
-    const values = new type.ArrayType(joinUint8Arrays(bytes)[0].buffer);
+    const ArrayType = type.ArrayType;
 
-    const vector = bytes
-        .map((b) => new type.ArrayType(b.buffer))
+    const typed = valuesTyped(ArrayType);
+    const jsArray = values || [...typed];
+    const vector = Vector.new(Data.Float(type, 0, typed.length, 0, null, typed));
+    const chunked = testValueBuffers.map((b) => new ArrayType(b))
         .map((b) => Vector.new(Data.Float(type, 0, b.length, 0, null, b)))
         .reduce((v: any, v2) => v.concat(v2));
 
-    gets_expected_values(vector, values);
-    iterates_expected_values(vector, values);
-    indexof_returns_expected_values(vector, values);
+    const begin = (vector.length * .25) | 0;
+    const end = (vector.length * .75) | 0;
+    const combos = [[`vector`, vector], [`chunked`, chunked]] as [string, V<T>][];
 
+    combos.forEach(([chunksType, vector]) => {
+        describe(chunksType, () => {
+            // test base case no slicing
+            describe(`base case no slicing`, () => testAndValidateVector(vector, typed, jsArray));
+            // test slicing without args
+            describe(`slicing without args`, () => testAndValidateVector(vector.slice(), typed.slice(), jsArray.slice()));
+            // test slicing the middle half
+            describe(`slice the middle half`, () => testAndValidateVector(
+                vector.slice(begin, end),
+                typed.slice(begin, end),
+                jsArray.slice(begin, end)
+            ));
+            // test splicing out the middle half
+            describe(`splicing out the middle half`, () => testAndValidateVector(
+                vector.slice(0, begin).concat(vector.slice(end)),
+                new ArrayType([...typed.slice(0, begin), ...typed.slice(end)]),
+                [...jsArray.slice(0, begin), ...jsArray.slice(end)]
+            ));
+        });
+    });
+}
+
+function testAndValidateVector<T extends Int | Float>(vector: V<T>, typed: T['TArray'], values = [...typed]) {
+    gets_expected_values(vector, typed, values);
+    iterates_expected_values(vector, typed, values);
+    indexof_returns_expected_values(vector, typed, values);
     slice_returns_a_typedarray(vector);
-    slices_the_entire_array(vector, values);
-    slices_from_minus_20_to_length(vector, values);
-    slices_from_0_to_minus_20(vector, values);
-    slices_the_array_from_0_to_length_minus_20(vector, values);
-    slices_the_array_from_0_to_length_plus_20(vector, values);
+    slices_the_entire_array(vector, typed);
+    slices_from_minus_20_to_length(vector, typed);
+    slices_from_0_to_minus_20(vector, typed);
+    slices_the_array_from_0_to_length_minus_20(vector, typed);
+    slices_the_array_from_0_to_length_plus_20(vector, typed);
 }
 
-function gets_expected_values<T extends DataType>(vector: Vector<T>, values: T['TArray']) {
+function gets_expected_values<T extends Int | Float>(vector: V<T>, typed: T['TArray'], values = [...typed]) {
     test(`gets expected values`, () => {
         expect.hasAssertions();
         let i = -1, n = vector.length;
@@ -93,18 +328,26 @@ function gets_expected_values<T extends DataType>(vector: Vector<T>, values: T['
                 while (++i < n) {
                     expect(vector.get(i)).toEqual(values[i]);
                 }
+            } else if (typeof values[0] === 'bigint') {
+                while (++i < n) {
+                    const x: any = vector.get(i)!;
+                    expect(0n + x).toEqual(values[i]);
+                }
             } else {
+                const vector64 = vector as Vector<Int64 | Uint64>;
+                const ArrayType = (vector as Vector<Int64 | Uint64>).ArrayType;
+                const i64 = () => new ArrayType(values.slice(stride * i, stride * (i + 1)));
                 while (++i < n) {
-                    expect(vector.get(i)!.subarray(0, stride))
-                        .toEqual(values.slice(stride * i, stride * (i + 1)));
+                    expect(vector64.get(i)!.subarray(0, stride)).toEqual(i64());
                 }
             }
         } catch (e) { throw new Error(`${i}: ${e}`); }
     });
 }
 
-function iterates_expected_values<T extends DataType>(vector: Vector<T>, values: T['TArray']) {
+function iterates_expected_values<T extends Int | Float>(vector: V<T>, typed: T['TArray'], values = [...typed]) {
     test(`iterates expected values`, () => {
+        expect.hasAssertions();
         let i = -1, n = vector.length;
         let stride = vector.stride;
         try {
@@ -113,78 +356,129 @@ function iterates_expected_values<T extends DataType>(vector: Vector<T>, values:
                     expect(++i).toBeLessThan(n);
                     expect(v).toEqual(values[i]);
                 }
-            } else {
+            } else if (typeof values[0] === 'bigint') {
+                let x: any;
                 for (let v of vector) {
+                    x = v;
                     expect(++i).toBeLessThan(n);
-                    expect(v!.subarray(0, stride))
-                        .toEqual(values.slice(stride * i, stride * (i + 1)));
+                    expect(0n + x).toEqual(values[i]);
+                }
+            } else {
+                const vector64 = vector as Vector<Int64 | Uint64>;
+                const ArrayType = (vector as Vector<Int64 | Uint64>).ArrayType;
+                const i64 = () => new ArrayType(values.slice(stride * i, stride * (i + 1)));
+                for (let v of vector64) {
+                    expect(++i).toBeLessThan(n);
+                    expect(v!.subarray(0, stride)).toEqual(i64());
                 }
             }
         } catch (e) { throw new Error(`${i}: ${e}`); }
     });
 }
 
-function indexof_returns_expected_values<T extends DataType>(vector: Vector<T>, values: T['TArray']) {
+function indexof_returns_expected_values<T extends Int | Float>(vector: V<T>, typed: T['TArray'], values: any = [...typed]) {
     test(`indexOf returns expected values`, () => {
 
-        // Create a set of test data composed of all of the actual values and a few random values
-        let testValues = new vector.ArrayType(joinUint8Arrays([
-            ...bytes,
-            ...[randomBytes(8 * 2 * vector.ArrayType.BYTES_PER_ELEMENT)]
-        ])[0].buffer);
+        expect.hasAssertions();
+
+        const stride = vector.stride;
+        const BPE = vector.ArrayType.BYTES_PER_ELEMENT;
+        const isBigInt = typeof values[0] === 'bigint';
+        const isInt64 = vector.type.compareTo(new Int64());
+        const isFloat16 = vector.type.compareTo(new Float16());
+
+        // Create a few random values
+        let missing: any = new vector.ArrayType(randomBytes(8 * 2 * BPE));
+
+        // Special cases convert the values and/or missing to the
+        // representations that indexOf() expects to receive
+
+        if (isFloat16) {
+            missing = uint16ToFloat64Array(missing);
+        } else if (isBigInt) {
+            const BigIntArray = isInt64 ? BigInt64Array : BigUint64Array;
+            missing = Array.from({ length: missing.length / stride },
+                (_, i) => new BigIntArray(missing.buffer, BPE * stride * i, 1)[0]);
+        } else if (stride !== 1) {
+            values = Array.from({ length: typed.length / stride },
+                (_, i) => typed.slice(stride * i, stride * (i + 1)));
+            missing = Array.from({ length: missing.length / stride },
+                (_, i) => missing.slice(stride * i, stride * (i + 1)));
+        }
+
+        // Combine with the expected values and shuffle the order
+        const shuffled = shuffle(values.concat([...missing]));
+
+        let i = -1, n = shuffled.length;
 
-        let i = -1, n, stride = vector.stride;
         try {
-            if (vector.stride === 1) {
-                for (const value of testValues) {
-                    ++i;
-                    const expected = values.indexOf(value);
-                    expect(vector.indexOf(value)).toEqual(expected);
-                }
+            if (!isBigInt) {
+                while (++i < n) { expect(vector.indexOf(shuffled[i])) .toEqual(values.indexOf(shuffled[i])); }
             } else {
-                for (i = -1, n = testValues.length / stride | 0; ++i < n;) {
-                    const value = testValues.slice(stride * i, stride * (i + 1));
-                    const expected = values.findIndex((d: number, i: number) =>
-                        i % stride === 0 && d === value[0] && testValues[i + 1] === value[1]);
-                    expect(vector.indexOf(value)).toEqual(expected >= 0 ? expected / stride : -1);
+                // Distinguish the bigint comparisons to ensure the indexOf type signature accepts bigints
+                let shuffled64 = shuffled as bigint[];
+                if (isInt64) {
+                    let vector64 = vector as Int64Vector;
+                    while (++i < n) { expect(vector64.indexOf(shuffled64[i])).toEqual(values.indexOf(shuffled64[i])); }
+                } else {
+                    let vector64 = vector as Uint64Vector;
+                    while (++i < n) { expect(vector64.indexOf(shuffled64[i])).toEqual(values.indexOf(shuffled64[i])); }
                 }
             }
         } catch (e) { throw new Error(`${i}: ${e}`); }
     });
 }
 
-function slice_returns_a_typedarray<T extends DataType>(vector: Vector<T>) {
+function slice_returns_a_typedarray<T extends Int | Float>(vector: V<T>) {
     test(`slice returns a TypedArray`, () => {
+        expect.hasAssertions();
         expect(vector.slice().toArray()).toBeInstanceOf(vector.ArrayType);
     });
 }
 
-function slices_the_entire_array<T extends DataType>(vector: Vector<T>, values: T['TArray']) {
+function slices_the_entire_array<T extends Int | Float>(vector: V<T>, values: T['TArray']) {
     test(`slices the entire array`, () => {
+        expect.hasAssertions();
         expect(vector.slice().toArray()).toEqual(values);
     });
 }
 
-function slices_from_minus_20_to_length<T extends DataType>(vector: Vector<T>, values: T['TArray']) {
+function slices_from_minus_20_to_length<T extends Int | Float>(vector: V<T>, values: T['TArray']) {
     test(`slices from -20 to length`, () => {
+        expect.hasAssertions();
         expect(vector.slice(-20).toArray()).toEqual(values.slice(-(20 * vector.stride)));
     });
 }
 
-function slices_from_0_to_minus_20<T extends DataType>(vector: Vector<T>, values: T['TArray']) {
+function slices_from_0_to_minus_20<T extends Int | Float>(vector: V<T>, values: T['TArray']) {
     test(`slices from 0 to -20`, () => {
+        expect.hasAssertions();
         expect(vector.slice(0, -20).toArray()).toEqual(values.slice(0, -(20 * vector.stride)));
     });
 }
 
-function slices_the_array_from_0_to_length_minus_20 <T extends DataType>(vector: Vector<T>, values: T['TArray']) {
+function slices_the_array_from_0_to_length_minus_20 <T extends Int | Float>(vector: V<T>, values: T['TArray']) {
     test(`slices the array from 0 to length - 20`, () => {
+        expect.hasAssertions();
         expect(vector.slice(0, vector.length - 20).toArray()).toEqual(values.slice(0, values.length - (20 * vector.stride)));
     });
 }
 
-function slices_the_array_from_0_to_length_plus_20<T extends DataType>(vector: Vector<T>, values: T['TArray']) {
+function slices_the_array_from_0_to_length_plus_20<T extends Int | Float>(vector: V<T>, values: T['TArray']) {
     test(`slices the array from 0 to length + 20`, () => {
+        expect.hasAssertions();
         expect(vector.slice(0, vector.length + 20).toArray()).toEqual(values.slice(0, values.length + (20 * vector.stride)));
     });
 }
+
+function shuffle(input: any[]) {
+    const result = input.slice();
+    let j, tmp, i = result.length;
+    while (--i > 0) {
+        j = (Math.random() * (i + 1)) | 0;
+        tmp = result[i];
+        result[i] = result[j];
+        result[j] = tmp;
+    }
+    return result;
+}