You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by we...@apache.org on 2019/09/15 15:48:00 UTC

[arrow] branch master updated: ARROW-5762: [JS] Align Map type impl with the spec

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

wesm 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 abc7860  ARROW-5762: [JS] Align Map type impl with the spec
abc7860 is described below

commit abc786099627ef429109da16b9dc768b4efbd866
Author: ptaylor <pa...@me.com>
AuthorDate: Sun Sep 15 10:47:51 2019 -0500

    ARROW-5762: [JS] Align Map type impl with the spec
    
    This PR closes [ARROW-5762](https://issues.apache.org/jira/browse/ARROW-5762) by aligning the Map implementation with the spec, enabling its inclusion in the integration tests.
    
    Feature-wise, the biggest change is to the `Struct` and `Map` rows. `Row` is now an abstract base class extended by `StructRow` and `MapRow`.
    
    Row implements [JS's native Map](https://developer.mozilla.org/en-US/docs/Web/JavaScript/Reference/Global_Objects/Map) interface (and will pass `row instanceof Map` checks). To align with JS Maps, the `Symbol.iterator` return type changes from `IterableIterator<TValue>` to `IterableIterator<[TKey, TValue]>`.
    
    `Struct` fields are uniform and known in advance, whereas `Map` fields vary by row. The `StructRow` will still take advantage of the `Object.create()` technique to create fast instances using a single `StructRow` instance as its prototype.
    
    However `MapRow` must either be dynamic or have its fields defined on construction, so I've changed `MapRow` to return an ES6 Proxy if available ([supported everywhere except IE11](https://caniuse.com/#search=Proxy)) and fall back to `Object.defineProperties()` if not.
    
    Closes #5371 from trxcllnt/ARROW-5762/map-type and squashes the following commits:
    
    3a0eca91d <ptaylor> ensure generated strings are unique
    da2a1ed77 <ptaylor> Merge branch 'master' into ARROW-5762/map-type
    58f244aa3 <ptaylor> reimplement Map type
    
    Authored-by: ptaylor <pa...@me.com>
    Signed-off-by: Wes McKinney <we...@apache.org>
---
 integration/integration_test.py       |   1 -
 js/DEVELOP.md                         |  58 +++---
 js/src/Arrow.ts                       |   1 -
 js/src/bin/arrow2csv.ts               |   2 +-
 js/src/builder.ts                     |   4 +-
 js/src/builder/map.ts                 |  50 ++++-
 js/src/data.ts                        |   8 +-
 js/src/fb/File.ts                     |  17 --
 js/src/fb/Message.ts                  |  35 ----
 js/src/fb/Schema.ts                   | 367 +++++++++++++++-------------------
 js/src/interfaces.ts                  |  92 ++++-----
 js/src/ipc/metadata/json.ts           |   2 +-
 js/src/ipc/metadata/message.ts        |   2 +-
 js/src/recordbatch.ts                 |  39 ++--
 js/src/table.ts                       |  37 ++--
 js/src/type.ts                        |  31 ++-
 js/src/util/vector.ts                 | 168 ++++++++--------
 js/src/vector/base.ts                 |   2 +-
 js/src/vector/index.ts                |   2 +-
 js/src/vector/map.ts                  |  19 +-
 js/src/vector/row.ts                  | 341 +++++++++++++++++++++++--------
 js/src/vector/struct.ts               |  19 +-
 js/src/visitor/get.ts                 |  16 +-
 js/src/visitor/jsonvectorassembler.ts |   1 +
 js/src/visitor/set.ts                 |  45 +++--
 js/src/visitor/vectorassembler.ts     |   8 +-
 js/src/visitor/vectorloader.ts        |   2 +-
 js/test/generate-test-data.ts         |  83 +++++---
 28 files changed, 806 insertions(+), 646 deletions(-)

diff --git a/integration/integration_test.py b/integration/integration_test.py
index be20b4e..8c04233 100644
--- a/integration/integration_test.py
+++ b/integration/integration_test.py
@@ -1073,7 +1073,6 @@ def generate_map_case():
 
     batch_sizes = [7, 10]
     skip = set()
-    skip.add('JS')  # TODO(ARROW-1279)
     skip.add('Go')  # TODO(ARROW-3679)
     return _generate_file("map", fields, batch_sizes, skip=skip)
 
diff --git a/js/DEVELOP.md b/js/DEVELOP.md
index 8adc962..2cc4434 100644
--- a/js/DEVELOP.md
+++ b/js/DEVELOP.md
@@ -39,8 +39,6 @@ If you’d like to report a bug but don’t have time to fix it, you can still p
 it on JIRA, or email the mailing list
 [dev@arrow.apache.org](http://mail-archives.apache.org/mod_mbox/arrow-dev/)
 
-
-
 # The npm scripts
 
 * `npm run clean` - cleans targets
@@ -68,37 +66,31 @@ Uses [lerna](https://github.com/lerna/lerna) to publish each build target to npm
 
 # Updating the Arrow format flatbuffers generated code
 
-Once generated, the flatbuffers format code needs to be adjusted for our build scripts.
-
-1. Generate the flatbuffers TypeScript source from the Arrow project root directory:
-    ```sh
-    cd $ARROW_HOME
-
-    flatc --ts -o ./js/src/format ./format/*.fbs
-
-    cd ./js/src/format
-
-    # Delete Tensor_generated.js (skip this when we support Tensors)
-    rm ./Tensor_generated.ts
-
-    # Remove "_generated" suffix from TS files
-    mv ./File_generated.ts .File.ts
-    mv ./Schema_generated.ts .Schema.ts
-    mv ./Message_generated.ts .Message.ts
-    ```
-1. Remove Tensor import from `Schema.ts`
-1. Fix all the `flatbuffers` imports
-    ```ts
-    import { flatbuffers } from "./flatbuffers" // <-- change
-    import { flatbuffers } from "flatbuffers" // <-- to this
-    ```
-1. Remove `_generated` from the ES6 imports of the generated files
-    ```ts
-    import * as NS16187549871986683199 from "./Schema_generated"; // <-- change
-    import * as NS16187549871986683199 from "./Schema"; // <------- to this
-    ```
-1. Add `/* tslint:disable:class-name */` to the top of `Schema.ts`
-1. Execute `npm run lint` to fix all the linting errors
+1. Once generated, the flatbuffers format code needs to be adjusted for our build scripts (assumes `gnu-sed`):
+
+```shell
+cd $ARROW_HOME
+
+flatc --ts -o ./js/src/fb ./format/{File,Schema,Message}.fbs
+
+cd ./js/src/fb
+
+# Rename the existing files to <filename>.bak.ts
+mv File{,.bak}.ts && mv Schema{,.bak}.ts && mv Message{,.bak}.ts
+
+# Remove `_generated` from the ES6 imports of the generated files
+sed -i '+s+_generated\";+\";+ig' *_generated.ts
+# Fix all the `flatbuffers` imports
+sed -i '+s+./flatbuffers+flatbuffers+ig' *_generated.ts
+# Fix the Union createTypeIdsVector typings
+sed -i -r '+s+static createTypeIdsVector\(builder: flatbuffers.Builder, data: number\[\] \| Uint8Array+static createTypeIdsVector\(builder: flatbuffers.Builder, data: number\[\] \| Int32Array+ig' Schema_generated.ts
+# Add `/* tslint:disable:class-name */` to the top of `Schema.ts`
+echo -e '/* tslint:disable:class-name */\n' | cat - Schema_generated.ts > Schema1.ts && mv Schema1.ts Schema_generated.ts
+# Remove "_generated" suffix from TS files
+mv File{_generated,}.ts && mv Schema{_generated,}.ts && mv Message{_generated,}.ts
+```
+2. Manually remove `Tensor` and `SparseTensor` imports and exports
+3. Execute `npm run lint` from the `js` directory to fix the linting errors
 
 [1]: mailto:dev-subscribe@arrow.apache.org
 [2]: https://github.com/apache/arrow/tree/master/format
diff --git a/js/src/Arrow.ts b/js/src/Arrow.ts
index 88ebc5a..59fe4fa 100644
--- a/js/src/Arrow.ts
+++ b/js/src/Arrow.ts
@@ -44,7 +44,6 @@ export { Column } from './column';
 export { Visitor } from './visitor';
 export { Schema, Field } from './schema';
 export {
-    Row,
     Vector,
     BaseVector,
     BinaryVector,
diff --git a/js/src/bin/arrow2csv.ts b/js/src/bin/arrow2csv.ts
index c872206..eb27581 100644
--- a/js/src/bin/arrow2csv.ts
+++ b/js/src/bin/arrow2csv.ts
@@ -177,7 +177,7 @@ function batchesToString(state: ToStringState, schema: Schema) {
                     if (rowId++ % 350 === 0) {
                         this.push(`${formatRow(header, maxColWidths, sep)}\n`);
                     }
-                    this.push(`${formatRow([rowId, ...row].map(valueToString), maxColWidths, sep)}\n`);
+                    this.push(`${formatRow([rowId, ...row.toArray()].map(valueToString), maxColWidths, sep)}\n`);
                 }
             }
             cb();
diff --git a/js/src/builder.ts b/js/src/builder.ts
index 4251ce4..d9f032b 100644
--- a/js/src/builder.ts
+++ b/js/src/builder.ts
@@ -25,7 +25,7 @@ import {
     DataType, strideForType,
     Float, Int, Decimal, FixedSizeBinary,
     Date_, Time, Timestamp, Interval,
-    Utf8, Binary, List,
+    Utf8, Binary, List, Map_
 } from './type';
 
 /**
@@ -441,7 +441,7 @@ export abstract class FixedWidthBuilder<T extends Int | Float | FixedSizeBinary
 }
 
 /** @ignore */
-export abstract class VariableWidthBuilder<T extends Binary | Utf8 | List, TNull = any> extends Builder<T, TNull> {
+export abstract class VariableWidthBuilder<T extends Binary | Utf8 | List | Map_, TNull = any> extends Builder<T, TNull> {
     protected _pendingLength: number = 0;
     protected _offsets: OffsetsBufferBuilder;
     protected _pending: Map<number, any> | undefined;
diff --git a/js/src/builder/map.ts b/js/src/builder/map.ts
index 3a66ec7..806fbc0 100644
--- a/js/src/builder/map.ts
+++ b/js/src/builder/map.ts
@@ -16,15 +16,49 @@
 // under the License.
 
 import { Field } from '../schema';
-import { Builder } from '../builder';
-import { DataType, Map_ } from '../type';
+import { DataType, Map_, Struct } from '../type';
+import { Builder, VariableWidthBuilder } from '../builder';
+
+/** @ignore */ type MapValue<K extends DataType = any, V extends DataType = any> = Map_<K, V>['TValue'];
+/** @ignore */ type MapValues<K extends DataType = any, V extends DataType = any> = Map<number, MapValue<K, V> | undefined>;
+/** @ignore */ type MapValueExt<K extends DataType = any, V extends DataType = any> = MapValue<K, V> | { [key: string]: V } | { [key: number]: V } ;
 
 /** @ignore */
-export class MapBuilder<T extends { [key: string]: DataType } = any, TNull = any> extends Builder<Map_<T>, TNull> {
-    public addChild(child: Builder, name = `${this.numChildren}`) {
-        const { children, keysSorted } = this.type;
-        const childIndex = this.children.push(child);
-        this.type = new Map_([...children, new Field(name, child.type, true)], keysSorted);
-        return childIndex;
+export class MapBuilder<K extends DataType = any, V extends DataType = any, TNull = any> extends VariableWidthBuilder<Map_<K, V>, TNull> {
+
+    protected _pending: MapValues<K, V> | undefined;
+    public set(index: number, value: MapValueExt<K, V> | TNull) {
+        return super.set(index, value as MapValue<K, V> | TNull);
+    }
+
+    public setValue(index: number, value: MapValueExt<K, V>) {
+        value = value instanceof Map ? value : new Map(Object.entries(value));
+        const pending = this._pending || (this._pending = new Map() as MapValues<K, V>);
+        const current = pending.get(index);
+        current && (this._pendingLength -= current.size);
+        this._pendingLength += value.size;
+        pending.set(index, value);
+    }
+
+    public addChild(child: Builder<Struct<{ key: K, value: V }>>, name = `${this.numChildren}`) {
+        if (this.numChildren > 0) {
+            throw new Error('ListBuilder can only have one child.');
+        }
+        this.children[this.numChildren] = child;
+        this.type = new Map_<K, V>(new Field(name, child.type, true), this.type.keysSorted);
+        return this.numChildren - 1;
+    }
+
+    protected _flushPending(pending: MapValues<K, V>) {
+        const offsets = this._offsets;
+        const setValue = this._setValue;
+        pending.forEach((value, index) => {
+            if (value === undefined) {
+                offsets.set(index, 0);
+            } else {
+                offsets.set(index, value.size);
+                setValue(this, index, value);
+            }
+        });
     }
 }
diff --git a/js/src/data.ts b/js/src/data.ts
index a17be8c..dcc9c2c 100644
--- a/js/src/data.ts
+++ b/js/src/data.ts
@@ -199,7 +199,7 @@ export class Data<T extends DataType = DataType> {
             case Type.List:            return <unknown> Data.List(            <unknown> type as List,            offset, length, nullCount || 0, buffers[BufferType.VALIDITY], buffers[BufferType.OFFSET] || [], (childData || [])[0]) as Data<T>;
             case Type.FixedSizeList:   return <unknown> Data.FixedSizeList(   <unknown> type as FixedSizeList,   offset, length, nullCount || 0, buffers[BufferType.VALIDITY], (childData || [])[0]) as Data<T>;
             case Type.Struct:          return <unknown> Data.Struct(          <unknown> type as Struct,          offset, length, nullCount || 0, buffers[BufferType.VALIDITY], childData || []) as Data<T>;
-            case Type.Map:             return <unknown> Data.Map(             <unknown> type as Map_,            offset, length, nullCount || 0, buffers[BufferType.VALIDITY], childData || []) as Data<T>;
+            case Type.Map:             return <unknown> Data.Map(             <unknown> type as Map_,            offset, length, nullCount || 0, buffers[BufferType.VALIDITY], buffers[BufferType.OFFSET] || [], (childData || [])[0]) as Data<T>;
             case Type.Union:           return <unknown> Data.Union(           <unknown> type as Union,           offset, length, nullCount || 0, buffers[BufferType.VALIDITY], buffers[BufferType.TYPE] || [], buffers[BufferType.OFFSET] || childData, childData) as Data<T>;
         }
         throw new Error(`Unrecognized typeId ${type.typeId}`);
@@ -262,7 +262,7 @@ export class Data<T extends DataType = DataType> {
         return new Data(type, offset, length, nullCount, [toInt32Array(valueOffsets), undefined, toUint8Array(nullBitmap)], [child]);
     }
     /** @nocollapse */
-    public static FixedSizeList<T extends FixedSizeList>(type: T, offset: number, length: number, nullCount: number, nullBitmap: NullBuffer, child: Data | Vector) {
+    public static FixedSizeList<T extends FixedSizeList>(type: T, offset: number, length: number, nullCount: number, nullBitmap: NullBuffer, child: Data<T['valueType']> | Vector<T['valueType']>) {
         return new Data(type, offset, length, nullCount, [undefined, undefined, toUint8Array(nullBitmap)], [child]);
     }
     /** @nocollapse */
@@ -270,8 +270,8 @@ export class Data<T extends DataType = DataType> {
         return new Data(type, offset, length, nullCount, [undefined, undefined, toUint8Array(nullBitmap)], children);
     }
     /** @nocollapse */
-    public static Map<T extends Map_>(type: T, offset: number, length: number, nullCount: number, nullBitmap: NullBuffer, children: (Data | Vector)[]) {
-        return new Data(type, offset, length, nullCount, [undefined, undefined, toUint8Array(nullBitmap)], children);
+    public static Map<T extends Map_>(type: T, offset: number, length: number, nullCount: number, nullBitmap: NullBuffer, valueOffsets: ValueOffsetsBuffer, child: (Data | Vector)) {
+        return new Data(type, offset, length, nullCount, [toInt32Array(valueOffsets), undefined, toUint8Array(nullBitmap)], [child]);
     }
     public static Union<T extends SparseUnion>(type: T, offset: number, length: number, nullCount: number, nullBitmap: NullBuffer, typeIds: TypeIdsBuffer, children: (Data | Vector)[], _?: any): Data<T>;
     public static Union<T extends DenseUnion>(type: T, offset: number, length: number, nullCount: number, nullBitmap: NullBuffer, typeIds: TypeIdsBuffer, valueOffsets: ValueOffsetsBuffer, children: (Data | Vector)[]): Data<T>;
diff --git a/js/src/fb/File.ts b/js/src/fb/File.ts
index f986926..75e2720 100644
--- a/js/src/fb/File.ts
+++ b/js/src/fb/File.ts
@@ -35,15 +35,6 @@ export namespace org.apache.arrow.flatbuf {
         }
 
         /**
-         * @param flatbuffers.ByteBuffer bb
-         * @param Footer= obj
-         * @returns Footer
-         */
-        static getSizePrefixedRootAsFooter(bb: flatbuffers.ByteBuffer, obj?: Footer): Footer {
-            return (obj || new Footer).__init(bb.readInt32(bb.position()) + bb.position(), bb);
-        }
-
-        /**
          * @returns org.apache.arrow.flatbuf.MetadataVersion
          */
         version(): NS7624605610262437867.org.apache.arrow.flatbuf.MetadataVersion {
@@ -168,14 +159,6 @@ export namespace org.apache.arrow.flatbuf {
             builder.finish(offset);
         }
 
-        /**
-         * @param flatbuffers.Builder builder
-         * @param flatbuffers.Offset offset
-         */
-        static finishSizePrefixedFooterBuffer(builder: flatbuffers.Builder, offset: flatbuffers.Offset) {
-            builder.finish(offset, undefined);
-        }
-
         static createFooter(builder: flatbuffers.Builder, version: NS7624605610262437867.org.apache.arrow.flatbuf.MetadataVersion, schemaOffset: flatbuffers.Offset, dictionariesOffset: flatbuffers.Offset, recordBatchesOffset: flatbuffers.Offset): flatbuffers.Offset {
             Footer.startFooter(builder);
             Footer.addVersion(builder, version);
diff --git a/js/src/fb/Message.ts b/js/src/fb/Message.ts
index d59b2f1..aa14521 100644
--- a/js/src/fb/Message.ts
+++ b/js/src/fb/Message.ts
@@ -126,15 +126,6 @@ export namespace org.apache.arrow.flatbuf {
         }
 
         /**
-         * @param flatbuffers.ByteBuffer bb
-         * @param RecordBatch= obj
-         * @returns RecordBatch
-         */
-        static getSizePrefixedRootAsRecordBatch(bb: flatbuffers.ByteBuffer, obj?: RecordBatch): RecordBatch {
-            return (obj || new RecordBatch).__init(bb.readInt32(bb.position()) + bb.position(), bb);
-        }
-
-        /**
          * number of records / rows. The arrays in the batch should all have this
          * length
          *
@@ -291,15 +282,6 @@ export namespace org.apache.arrow.flatbuf {
         }
 
         /**
-         * @param flatbuffers.ByteBuffer bb
-         * @param DictionaryBatch= obj
-         * @returns DictionaryBatch
-         */
-        static getSizePrefixedRootAsDictionaryBatch(bb: flatbuffers.ByteBuffer, obj?: DictionaryBatch): DictionaryBatch {
-            return (obj || new DictionaryBatch).__init(bb.readInt32(bb.position()) + bb.position(), bb);
-        }
-
-        /**
          * @returns flatbuffers.Long
          */
         id(): flatbuffers.Long {
@@ -405,15 +387,6 @@ export namespace org.apache.arrow.flatbuf {
         }
 
         /**
-         * @param flatbuffers.ByteBuffer bb
-         * @param Message= obj
-         * @returns Message
-         */
-        static getSizePrefixedRootAsMessage(bb: flatbuffers.ByteBuffer, obj?: Message): Message {
-            return (obj || new Message).__init(bb.readInt32(bb.position()) + bb.position(), bb);
-        }
-
-        /**
          * @returns org.apache.arrow.flatbuf.MetadataVersion
          */
         version(): NS7624605610262437867.org.apache.arrow.flatbuf.MetadataVersion {
@@ -549,14 +522,6 @@ export namespace org.apache.arrow.flatbuf {
             builder.finish(offset);
         }
 
-        /**
-         * @param flatbuffers.Builder builder
-         * @param flatbuffers.Offset offset
-         */
-        static finishSizePrefixedMessageBuffer(builder: flatbuffers.Builder, offset: flatbuffers.Offset) {
-            builder.finish(offset, undefined);
-        }
-
         static createMessage(builder: flatbuffers.Builder, version: NS7624605610262437867.org.apache.arrow.flatbuf.MetadataVersion, headerType: org.apache.arrow.flatbuf.MessageHeader, headerOffset: flatbuffers.Offset, bodyLength: flatbuffers.Long, customMetadataOffset: flatbuffers.Offset): flatbuffers.Offset {
             Message.startMessage(builder);
             Message.addVersion(builder, version);
diff --git a/js/src/fb/Schema.ts b/js/src/fb/Schema.ts
index 9910de2..812e806 100644
--- a/js/src/fb/Schema.ts
+++ b/js/src/fb/Schema.ts
@@ -1,6 +1,7 @@
-// automatically generated by the FlatBuffers compiler, do not modify
 /* tslint:disable:class-name */
 
+// automatically generated by the FlatBuffers compiler, do not modify
+
 import { flatbuffers } from 'flatbuffers';
 /**
  * @enum {number}
@@ -109,7 +110,10 @@ export namespace org.apache.arrow.flatbuf {
         FixedSizeBinary = 15,
         FixedSizeList = 16,
         Map = 17,
-        Duration = 18
+        Duration = 18,
+        LargeBinary = 19,
+        LargeUtf8 = 20,
+        LargeList = 21
     }
 }
 
@@ -157,15 +161,6 @@ export namespace org.apache.arrow.flatbuf {
         }
 
         /**
-         * @param flatbuffers.ByteBuffer bb
-         * @param Null= obj
-         * @returns Null
-         */
-        static getSizePrefixedRootAsNull(bb: flatbuffers.ByteBuffer, obj?: Null): Null {
-            return (obj || new Null).__init(bb.readInt32(bb.position()) + bb.position(), bb);
-        }
-
-        /**
          * @param flatbuffers.Builder builder
          */
         static startNull(builder: flatbuffers.Builder) {
@@ -220,15 +215,6 @@ export namespace org.apache.arrow.flatbuf {
         }
 
         /**
-         * @param flatbuffers.ByteBuffer bb
-         * @param Struct_= obj
-         * @returns Struct_
-         */
-        static getSizePrefixedRootAsStruct_(bb: flatbuffers.ByteBuffer, obj?: Struct_): Struct_ {
-            return (obj || new Struct_).__init(bb.readInt32(bb.position()) + bb.position(), bb);
-        }
-
-        /**
          * @param flatbuffers.Builder builder
          */
         static startStruct_(builder: flatbuffers.Builder) {
@@ -279,15 +265,6 @@ export namespace org.apache.arrow.flatbuf {
         }
 
         /**
-         * @param flatbuffers.ByteBuffer bb
-         * @param List= obj
-         * @returns List
-         */
-        static getSizePrefixedRootAsList(bb: flatbuffers.ByteBuffer, obj?: List): List {
-            return (obj || new List).__init(bb.readInt32(bb.position()) + bb.position(), bb);
-        }
-
-        /**
          * @param flatbuffers.Builder builder
          */
         static startList(builder: flatbuffers.Builder) {
@@ -310,19 +287,22 @@ export namespace org.apache.arrow.flatbuf {
     }
 }
 /**
+ * Same as List, but with 64-bit offsets, allowing to represent
+ * extremely large data values.
+ *
  * @constructor
  */
 export namespace org.apache.arrow.flatbuf {
-    export class FixedSizeList {
+    export class LargeList {
         bb: flatbuffers.ByteBuffer | null = null;
 
         bb_pos: number = 0;
         /**
          * @param number i
          * @param flatbuffers.ByteBuffer bb
-         * @returns FixedSizeList
+         * @returns LargeList
          */
-        __init(i: number, bb: flatbuffers.ByteBuffer): FixedSizeList {
+        __init(i: number, bb: flatbuffers.ByteBuffer): LargeList {
             this.bb_pos = i;
             this.bb = bb;
             return this;
@@ -330,11 +310,52 @@ export namespace org.apache.arrow.flatbuf {
 
         /**
          * @param flatbuffers.ByteBuffer bb
-         * @param FixedSizeList= obj
+         * @param LargeList= obj
+         * @returns LargeList
+         */
+        static getRootAsLargeList(bb: flatbuffers.ByteBuffer, obj?: LargeList): LargeList {
+            return (obj || new LargeList).__init(bb.readInt32(bb.position()) + bb.position(), bb);
+        }
+
+        /**
+         * @param flatbuffers.Builder builder
+         */
+        static startLargeList(builder: flatbuffers.Builder) {
+            builder.startObject(0);
+        }
+
+        /**
+         * @param flatbuffers.Builder builder
+         * @returns flatbuffers.Offset
+         */
+        static endLargeList(builder: flatbuffers.Builder): flatbuffers.Offset {
+            let offset = builder.endObject();
+            return offset;
+        }
+
+        static createLargeList(builder: flatbuffers.Builder): flatbuffers.Offset {
+            LargeList.startLargeList(builder);
+            return LargeList.endLargeList(builder);
+        }
+    }
+}
+/**
+ * @constructor
+ */
+export namespace org.apache.arrow.flatbuf {
+    export class FixedSizeList {
+        bb: flatbuffers.ByteBuffer | null = null;
+
+        bb_pos: number = 0;
+        /**
+         * @param number i
+         * @param flatbuffers.ByteBuffer bb
          * @returns FixedSizeList
          */
-        static getRootAsFixedSizeList(bb: flatbuffers.ByteBuffer, obj?: FixedSizeList): FixedSizeList {
-            return (obj || new FixedSizeList).__init(bb.readInt32(bb.position()) + bb.position(), bb);
+        __init(i: number, bb: flatbuffers.ByteBuffer): FixedSizeList {
+            this.bb_pos = i;
+            this.bb = bb;
+            return this;
         }
 
         /**
@@ -342,7 +363,7 @@ export namespace org.apache.arrow.flatbuf {
          * @param FixedSizeList= obj
          * @returns FixedSizeList
          */
-        static getSizePrefixedRootAsFixedSizeList(bb: flatbuffers.ByteBuffer, obj?: FixedSizeList): FixedSizeList {
+        static getRootAsFixedSizeList(bb: flatbuffers.ByteBuffer, obj?: FixedSizeList): FixedSizeList {
             return (obj || new FixedSizeList).__init(bb.readInt32(bb.position()) + bb.position(), bb);
         }
 
@@ -441,15 +462,6 @@ export namespace org.apache.arrow.flatbuf {
         }
 
         /**
-         * @param flatbuffers.ByteBuffer bb
-         * @param Map= obj
-         * @returns Map
-         */
-        static getSizePrefixedRootAsMap(bb: flatbuffers.ByteBuffer, obj?: Map): Map {
-            return (obj || new Map).__init(bb.readInt32(bb.position()) + bb.position(), bb);
-        }
-
-        /**
          * Set to true if the keys within each value are sorted
          *
          * @returns boolean
@@ -524,15 +536,6 @@ export namespace org.apache.arrow.flatbuf {
         }
 
         /**
-         * @param flatbuffers.ByteBuffer bb
-         * @param Union= obj
-         * @returns Union
-         */
-        static getSizePrefixedRootAsUnion(bb: flatbuffers.ByteBuffer, obj?: Union): Union {
-            return (obj || new Union).__init(bb.readInt32(bb.position()) + bb.position(), bb);
-        }
-
-        /**
          * @returns org.apache.arrow.flatbuf.UnionMode
          */
         mode(): org.apache.arrow.flatbuf.UnionMode {
@@ -655,15 +658,6 @@ export namespace org.apache.arrow.flatbuf {
         }
 
         /**
-         * @param flatbuffers.ByteBuffer bb
-         * @param Int= obj
-         * @returns Int
-         */
-        static getSizePrefixedRootAsInt(bb: flatbuffers.ByteBuffer, obj?: Int): Int {
-            return (obj || new Int).__init(bb.readInt32(bb.position()) + bb.position(), bb);
-        }
-
-        /**
          * @returns number
          */
         bitWidth(): number {
@@ -748,15 +742,6 @@ export namespace org.apache.arrow.flatbuf {
         }
 
         /**
-         * @param flatbuffers.ByteBuffer bb
-         * @param FloatingPoint= obj
-         * @returns FloatingPoint
-         */
-        static getSizePrefixedRootAsFloatingPoint(bb: flatbuffers.ByteBuffer, obj?: FloatingPoint): FloatingPoint {
-            return (obj || new FloatingPoint).__init(bb.readInt32(bb.position()) + bb.position(), bb);
-        }
-
-        /**
          * @returns org.apache.arrow.flatbuf.Precision
          */
         precision(): org.apache.arrow.flatbuf.Precision {
@@ -826,15 +811,6 @@ export namespace org.apache.arrow.flatbuf {
         }
 
         /**
-         * @param flatbuffers.ByteBuffer bb
-         * @param Utf8= obj
-         * @returns Utf8
-         */
-        static getSizePrefixedRootAsUtf8(bb: flatbuffers.ByteBuffer, obj?: Utf8): Utf8 {
-            return (obj || new Utf8).__init(bb.readInt32(bb.position()) + bb.position(), bb);
-        }
-
-        /**
          * @param flatbuffers.Builder builder
          */
         static startUtf8(builder: flatbuffers.Builder) {
@@ -857,6 +833,8 @@ export namespace org.apache.arrow.flatbuf {
     }
 }
 /**
+ * Opaque binary data
+ *
  * @constructor
  */
 export namespace org.apache.arrow.flatbuf {
@@ -885,15 +863,6 @@ export namespace org.apache.arrow.flatbuf {
         }
 
         /**
-         * @param flatbuffers.ByteBuffer bb
-         * @param Binary= obj
-         * @returns Binary
-         */
-        static getSizePrefixedRootAsBinary(bb: flatbuffers.ByteBuffer, obj?: Binary): Binary {
-            return (obj || new Binary).__init(bb.readInt32(bb.position()) + bb.position(), bb);
-        }
-
-        /**
          * @param flatbuffers.Builder builder
          */
         static startBinary(builder: flatbuffers.Builder) {
@@ -916,19 +885,22 @@ export namespace org.apache.arrow.flatbuf {
     }
 }
 /**
+ * Same as Utf8, but with 64-bit offsets, allowing to represent
+ * extremely large data values.
+ *
  * @constructor
  */
 export namespace org.apache.arrow.flatbuf {
-    export class FixedSizeBinary {
+    export class LargeUtf8 {
         bb: flatbuffers.ByteBuffer | null = null;
 
         bb_pos: number = 0;
         /**
          * @param number i
          * @param flatbuffers.ByteBuffer bb
-         * @returns FixedSizeBinary
+         * @returns LargeUtf8
          */
-        __init(i: number, bb: flatbuffers.ByteBuffer): FixedSizeBinary {
+        __init(i: number, bb: flatbuffers.ByteBuffer): LargeUtf8 {
             this.bb_pos = i;
             this.bb = bb;
             return this;
@@ -936,11 +908,105 @@ export namespace org.apache.arrow.flatbuf {
 
         /**
          * @param flatbuffers.ByteBuffer bb
-         * @param FixedSizeBinary= obj
+         * @param LargeUtf8= obj
+         * @returns LargeUtf8
+         */
+        static getRootAsLargeUtf8(bb: flatbuffers.ByteBuffer, obj?: LargeUtf8): LargeUtf8 {
+            return (obj || new LargeUtf8).__init(bb.readInt32(bb.position()) + bb.position(), bb);
+        }
+
+        /**
+         * @param flatbuffers.Builder builder
+         */
+        static startLargeUtf8(builder: flatbuffers.Builder) {
+            builder.startObject(0);
+        }
+
+        /**
+         * @param flatbuffers.Builder builder
+         * @returns flatbuffers.Offset
+         */
+        static endLargeUtf8(builder: flatbuffers.Builder): flatbuffers.Offset {
+            let offset = builder.endObject();
+            return offset;
+        }
+
+        static createLargeUtf8(builder: flatbuffers.Builder): flatbuffers.Offset {
+            LargeUtf8.startLargeUtf8(builder);
+            return LargeUtf8.endLargeUtf8(builder);
+        }
+    }
+}
+/**
+ * Same as Binary, but with 64-bit offsets, allowing to represent
+ * extremely large data values.
+ *
+ * @constructor
+ */
+export namespace org.apache.arrow.flatbuf {
+    export class LargeBinary {
+        bb: flatbuffers.ByteBuffer | null = null;
+
+        bb_pos: number = 0;
+        /**
+         * @param number i
+         * @param flatbuffers.ByteBuffer bb
+         * @returns LargeBinary
+         */
+        __init(i: number, bb: flatbuffers.ByteBuffer): LargeBinary {
+            this.bb_pos = i;
+            this.bb = bb;
+            return this;
+        }
+
+        /**
+         * @param flatbuffers.ByteBuffer bb
+         * @param LargeBinary= obj
+         * @returns LargeBinary
+         */
+        static getRootAsLargeBinary(bb: flatbuffers.ByteBuffer, obj?: LargeBinary): LargeBinary {
+            return (obj || new LargeBinary).__init(bb.readInt32(bb.position()) + bb.position(), bb);
+        }
+
+        /**
+         * @param flatbuffers.Builder builder
+         */
+        static startLargeBinary(builder: flatbuffers.Builder) {
+            builder.startObject(0);
+        }
+
+        /**
+         * @param flatbuffers.Builder builder
+         * @returns flatbuffers.Offset
+         */
+        static endLargeBinary(builder: flatbuffers.Builder): flatbuffers.Offset {
+            let offset = builder.endObject();
+            return offset;
+        }
+
+        static createLargeBinary(builder: flatbuffers.Builder): flatbuffers.Offset {
+            LargeBinary.startLargeBinary(builder);
+            return LargeBinary.endLargeBinary(builder);
+        }
+    }
+}
+/**
+ * @constructor
+ */
+export namespace org.apache.arrow.flatbuf {
+    export class FixedSizeBinary {
+        bb: flatbuffers.ByteBuffer | null = null;
+
+        bb_pos: number = 0;
+        /**
+         * @param number i
+         * @param flatbuffers.ByteBuffer bb
          * @returns FixedSizeBinary
          */
-        static getRootAsFixedSizeBinary(bb: flatbuffers.ByteBuffer, obj?: FixedSizeBinary): FixedSizeBinary {
-            return (obj || new FixedSizeBinary).__init(bb.readInt32(bb.position()) + bb.position(), bb);
+        __init(i: number, bb: flatbuffers.ByteBuffer): FixedSizeBinary {
+            this.bb_pos = i;
+            this.bb = bb;
+            return this;
         }
 
         /**
@@ -948,7 +1014,7 @@ export namespace org.apache.arrow.flatbuf {
          * @param FixedSizeBinary= obj
          * @returns FixedSizeBinary
          */
-        static getSizePrefixedRootAsFixedSizeBinary(bb: flatbuffers.ByteBuffer, obj?: FixedSizeBinary): FixedSizeBinary {
+        static getRootAsFixedSizeBinary(bb: flatbuffers.ByteBuffer, obj?: FixedSizeBinary): FixedSizeBinary {
             return (obj || new FixedSizeBinary).__init(bb.readInt32(bb.position()) + bb.position(), bb);
         }
 
@@ -1022,15 +1088,6 @@ export namespace org.apache.arrow.flatbuf {
         }
 
         /**
-         * @param flatbuffers.ByteBuffer bb
-         * @param Bool= obj
-         * @returns Bool
-         */
-        static getSizePrefixedRootAsBool(bb: flatbuffers.ByteBuffer, obj?: Bool): Bool {
-            return (obj || new Bool).__init(bb.readInt32(bb.position()) + bb.position(), bb);
-        }
-
-        /**
          * @param flatbuffers.Builder builder
          */
         static startBool(builder: flatbuffers.Builder) {
@@ -1081,15 +1138,6 @@ export namespace org.apache.arrow.flatbuf {
         }
 
         /**
-         * @param flatbuffers.ByteBuffer bb
-         * @param Decimal= obj
-         * @returns Decimal
-         */
-        static getSizePrefixedRootAsDecimal(bb: flatbuffers.ByteBuffer, obj?: Decimal): Decimal {
-            return (obj || new Decimal).__init(bb.readInt32(bb.position()) + bb.position(), bb);
-        }
-
-        /**
          * Total number of decimal digits
          *
          * @returns number
@@ -1185,15 +1233,6 @@ export namespace org.apache.arrow.flatbuf {
         }
 
         /**
-         * @param flatbuffers.ByteBuffer bb
-         * @param Date= obj
-         * @returns Date
-         */
-        static getSizePrefixedRootAsDate(bb: flatbuffers.ByteBuffer, obj?: Date): Date {
-            return (obj || new Date).__init(bb.readInt32(bb.position()) + bb.position(), bb);
-        }
-
-        /**
          * @returns org.apache.arrow.flatbuf.DateUnit
          */
         unit(): org.apache.arrow.flatbuf.DateUnit {
@@ -1265,15 +1304,6 @@ export namespace org.apache.arrow.flatbuf {
         }
 
         /**
-         * @param flatbuffers.ByteBuffer bb
-         * @param Time= obj
-         * @returns Time
-         */
-        static getSizePrefixedRootAsTime(bb: flatbuffers.ByteBuffer, obj?: Time): Time {
-            return (obj || new Time).__init(bb.readInt32(bb.position()) + bb.position(), bb);
-        }
-
-        /**
          * @returns org.apache.arrow.flatbuf.TimeUnit
          */
         unit(): org.apache.arrow.flatbuf.TimeUnit {
@@ -1365,15 +1395,6 @@ export namespace org.apache.arrow.flatbuf {
         }
 
         /**
-         * @param flatbuffers.ByteBuffer bb
-         * @param Timestamp= obj
-         * @returns Timestamp
-         */
-        static getSizePrefixedRootAsTimestamp(bb: flatbuffers.ByteBuffer, obj?: Timestamp): Timestamp {
-            return (obj || new Timestamp).__init(bb.readInt32(bb.position()) + bb.position(), bb);
-        }
-
-        /**
          * @returns org.apache.arrow.flatbuf.TimeUnit
          */
         unit(): org.apache.arrow.flatbuf.TimeUnit {
@@ -1482,15 +1503,6 @@ export namespace org.apache.arrow.flatbuf {
         }
 
         /**
-         * @param flatbuffers.ByteBuffer bb
-         * @param Interval= obj
-         * @returns Interval
-         */
-        static getSizePrefixedRootAsInterval(bb: flatbuffers.ByteBuffer, obj?: Interval): Interval {
-            return (obj || new Interval).__init(bb.readInt32(bb.position()) + bb.position(), bb);
-        }
-
-        /**
          * @returns org.apache.arrow.flatbuf.IntervalUnit
          */
         unit(): org.apache.arrow.flatbuf.IntervalUnit {
@@ -1558,15 +1570,6 @@ export namespace org.apache.arrow.flatbuf {
         }
 
         /**
-         * @param flatbuffers.ByteBuffer bb
-         * @param Duration= obj
-         * @returns Duration
-         */
-        static getSizePrefixedRootAsDuration(bb: flatbuffers.ByteBuffer, obj?: Duration): Duration {
-            return (obj || new Duration).__init(bb.readInt32(bb.position()) + bb.position(), bb);
-        }
-
-        /**
          * @returns org.apache.arrow.flatbuf.TimeUnit
          */
         unit(): org.apache.arrow.flatbuf.TimeUnit {
@@ -1638,15 +1641,6 @@ export namespace org.apache.arrow.flatbuf {
         }
 
         /**
-         * @param flatbuffers.ByteBuffer bb
-         * @param KeyValue= obj
-         * @returns KeyValue
-         */
-        static getSizePrefixedRootAsKeyValue(bb: flatbuffers.ByteBuffer, obj?: KeyValue): KeyValue {
-            return (obj || new KeyValue).__init(bb.readInt32(bb.position()) + bb.position(), bb);
-        }
-
-        /**
          * @param flatbuffers.Encoding= optionalEncoding
          * @returns string|Uint8Array|null
          */
@@ -1740,15 +1734,6 @@ export namespace org.apache.arrow.flatbuf {
         }
 
         /**
-         * @param flatbuffers.ByteBuffer bb
-         * @param DictionaryEncoding= obj
-         * @returns DictionaryEncoding
-         */
-        static getSizePrefixedRootAsDictionaryEncoding(bb: flatbuffers.ByteBuffer, obj?: DictionaryEncoding): DictionaryEncoding {
-            return (obj || new DictionaryEncoding).__init(bb.readInt32(bb.position()) + bb.position(), bb);
-        }
-
-        /**
          * The known dictionary id in the application where this data is used. In
          * the file or streaming formats, the dictionary ids are found in the
          * DictionaryBatch messages
@@ -1867,15 +1852,6 @@ export namespace org.apache.arrow.flatbuf {
         }
 
         /**
-         * @param flatbuffers.ByteBuffer bb
-         * @param Field= obj
-         * @returns Field
-         */
-        static getSizePrefixedRootAsField(bb: flatbuffers.ByteBuffer, obj?: Field): Field {
-            return (obj || new Field).__init(bb.readInt32(bb.position()) + bb.position(), bb);
-        }
-
-        /**
          * Name is not required, in i.e. a List
          *
          * @param flatbuffers.Encoding= optionalEncoding
@@ -2185,15 +2161,6 @@ export namespace org.apache.arrow.flatbuf {
         }
 
         /**
-         * @param flatbuffers.ByteBuffer bb
-         * @param Schema= obj
-         * @returns Schema
-         */
-        static getSizePrefixedRootAsSchema(bb: flatbuffers.ByteBuffer, obj?: Schema): Schema {
-            return (obj || new Schema).__init(bb.readInt32(bb.position()) + bb.position(), bb);
-        }
-
-        /**
          * endianness of the buffer
          * it is Little Endian by default
          * if endianness doesn't match the underlying system then the vectors need to be converted
@@ -2331,14 +2298,6 @@ export namespace org.apache.arrow.flatbuf {
             builder.finish(offset);
         }
 
-        /**
-         * @param flatbuffers.Builder builder
-         * @param flatbuffers.Offset offset
-         */
-        static finishSizePrefixedSchemaBuffer(builder: flatbuffers.Builder, offset: flatbuffers.Offset) {
-            builder.finish(offset, undefined);
-        }
-
         static createSchema(builder: flatbuffers.Builder, endianness: org.apache.arrow.flatbuf.Endianness, fieldsOffset: flatbuffers.Offset, customMetadataOffset: flatbuffers.Offset): flatbuffers.Offset {
             Schema.startSchema(builder);
             Schema.addEndianness(builder, endianness);
diff --git a/js/src/interfaces.ts b/js/src/interfaces.ts
index bebcf0c..54fb872 100644
--- a/js/src/interfaces.ts
+++ b/js/src/interfaces.ts
@@ -251,7 +251,7 @@ type DataTypeToVector<T extends DataType = any> = {
     [Type.Interval             ]: T extends type.Interval             ? vecs.IntervalVector                                 : vecs.BaseVector<T> ;
     [Type.IntervalDayTime      ]: T extends type.IntervalDayTime      ? vecs.IntervalDayTimeVector                          : vecs.BaseVector<T> ;
     [Type.IntervalYearMonth    ]: T extends type.IntervalYearMonth    ? vecs.IntervalYearMonthVector                        : vecs.BaseVector<T> ;
-    [Type.Map                  ]: T extends type.Map_                 ? vecs.MapVector<T['dataTypes']>                      : vecs.BaseVector<T> ;
+    [Type.Map                  ]: T extends type.Map_                 ? vecs.MapVector<T['keyType'], T['valueType']>        : vecs.BaseVector<T> ;
     [Type.List                 ]: T extends type.List                 ? vecs.ListVector<T['valueType']>                     : vecs.BaseVector<T> ;
     [Type.Struct               ]: T extends type.Struct               ? vecs.StructVector<T['dataTypes']>                   : vecs.BaseVector<T> ;
     [Type.Dictionary           ]: T extends type.Dictionary           ? vecs.DictionaryVector<T['valueType'], T['indices']> : vecs.BaseVector<T> ;
@@ -347,7 +347,7 @@ type TypeToBuilder<T extends Type = any, TNull = any> = {
     [Type.Interval             ]: builders.IntervalBuilder<any, TNull>        ;
     [Type.IntervalDayTime      ]: builders.IntervalDayTimeBuilder<TNull>      ;
     [Type.IntervalYearMonth    ]: builders.IntervalYearMonthBuilder<TNull>    ;
-    [Type.Map                  ]: builders.MapBuilder<any, TNull>             ;
+    [Type.Map                  ]: builders.MapBuilder<any, any, TNull>        ;
     [Type.List                 ]: builders.ListBuilder<any, TNull>            ;
     [Type.Struct               ]: builders.StructBuilder<any, TNull>          ;
     [Type.Dictionary           ]: builders.DictionaryBuilder<any, TNull>      ;
@@ -356,48 +356,48 @@ type TypeToBuilder<T extends Type = any, TNull = any> = {
 
 /** @ignore */
 type DataTypeToBuilder<T extends DataType = any, TNull = any> = {
-    [key: number               ]:                                                                                              builders.Builder<any, TNull> ;
-    [Type.Null                 ]: T extends type.Null                 ? builders.NullBuilder<TNull>                          : builders.Builder<any, TNull> ;
-    [Type.Bool                 ]: T extends type.Bool                 ? builders.BoolBuilder<TNull>                          : builders.Builder<any, TNull> ;
-    [Type.Int8                 ]: T extends type.Int8                 ? builders.Int8Builder<TNull>                          : builders.Builder<any, TNull> ;
-    [Type.Int16                ]: T extends type.Int16                ? builders.Int16Builder<TNull>                         : builders.Builder<any, TNull> ;
-    [Type.Int32                ]: T extends type.Int32                ? builders.Int32Builder<TNull>                         : builders.Builder<any, TNull> ;
-    [Type.Int64                ]: T extends type.Int64                ? builders.Int64Builder<TNull>                         : builders.Builder<any, TNull> ;
-    [Type.Uint8                ]: T extends type.Uint8                ? builders.Uint8Builder<TNull>                         : builders.Builder<any, TNull> ;
-    [Type.Uint16               ]: T extends type.Uint16               ? builders.Uint16Builder<TNull>                        : builders.Builder<any, TNull> ;
-    [Type.Uint32               ]: T extends type.Uint32               ? builders.Uint32Builder<TNull>                        : builders.Builder<any, TNull> ;
-    [Type.Uint64               ]: T extends type.Uint64               ? builders.Uint64Builder<TNull>                        : builders.Builder<any, TNull> ;
-    [Type.Int                  ]: T extends type.Int                  ? builders.IntBuilder<T, TNull>                        : builders.Builder<any, TNull> ;
-    [Type.Float16              ]: T extends type.Float16              ? builders.Float16Builder<TNull>                       : builders.Builder<any, TNull> ;
-    [Type.Float32              ]: T extends type.Float32              ? builders.Float32Builder<TNull>                       : builders.Builder<any, TNull> ;
-    [Type.Float64              ]: T extends type.Float64              ? builders.Float64Builder<TNull>                       : builders.Builder<any, TNull> ;
-    [Type.Float                ]: T extends type.Float                ? builders.FloatBuilder<T, TNull>                      : builders.Builder<any, TNull> ;
-    [Type.Utf8                 ]: T extends type.Utf8                 ? builders.Utf8Builder<TNull>                          : builders.Builder<any, TNull> ;
-    [Type.Binary               ]: T extends type.Binary               ? builders.BinaryBuilder<TNull>                        : builders.Builder<any, TNull> ;
-    [Type.FixedSizeBinary      ]: T extends type.FixedSizeBinary      ? builders.FixedSizeBinaryBuilder<TNull>               : builders.Builder<any, TNull> ;
-    [Type.Date                 ]: T extends type.Date_                ? builders.DateBuilder<T, TNull>                       : builders.Builder<any, TNull> ;
-    [Type.DateDay              ]: T extends type.DateDay              ? builders.DateDayBuilder<TNull>                       : builders.Builder<any, TNull> ;
-    [Type.DateMillisecond      ]: T extends type.DateMillisecond      ? builders.DateMillisecondBuilder<TNull>               : builders.Builder<any, TNull> ;
-    [Type.Timestamp            ]: T extends type.Timestamp            ? builders.TimestampBuilder<T, TNull>                  : builders.Builder<any, TNull> ;
-    [Type.TimestampSecond      ]: T extends type.TimestampSecond      ? builders.TimestampSecondBuilder<TNull>               : builders.Builder<any, TNull> ;
-    [Type.TimestampMillisecond ]: T extends type.TimestampMillisecond ? builders.TimestampMillisecondBuilder<TNull>          : builders.Builder<any, TNull> ;
-    [Type.TimestampMicrosecond ]: T extends type.TimestampMicrosecond ? builders.TimestampMicrosecondBuilder<TNull>          : builders.Builder<any, TNull> ;
-    [Type.TimestampNanosecond  ]: T extends type.TimestampNanosecond  ? builders.TimestampNanosecondBuilder<TNull>           : builders.Builder<any, TNull> ;
-    [Type.Time                 ]: T extends type.Time                 ? builders.TimeBuilder<T, TNull>                       : builders.Builder<any, TNull> ;
-    [Type.TimeSecond           ]: T extends type.TimeSecond           ? builders.TimeSecondBuilder<TNull>                    : builders.Builder<any, TNull> ;
-    [Type.TimeMillisecond      ]: T extends type.TimeMillisecond      ? builders.TimeMillisecondBuilder<TNull>               : builders.Builder<any, TNull> ;
-    [Type.TimeMicrosecond      ]: T extends type.TimeMicrosecond      ? builders.TimeMicrosecondBuilder<TNull>               : builders.Builder<any, TNull> ;
-    [Type.TimeNanosecond       ]: T extends type.TimeNanosecond       ? builders.TimeNanosecondBuilder<TNull>                : builders.Builder<any, TNull> ;
-    [Type.Decimal              ]: T extends type.Decimal              ? builders.DecimalBuilder<TNull>                       : builders.Builder<any, TNull> ;
-    [Type.Union                ]: T extends type.Union                ? builders.UnionBuilder<T, TNull>                      : builders.Builder<any, TNull> ;
-    [Type.DenseUnion           ]: T extends type.DenseUnion           ? builders.DenseUnionBuilder<T, TNull>                 : builders.Builder<any, TNull> ;
-    [Type.SparseUnion          ]: T extends type.SparseUnion          ? builders.SparseUnionBuilder<T, TNull>                : builders.Builder<any, TNull> ;
-    [Type.Interval             ]: T extends type.Interval             ? builders.IntervalBuilder<T, TNull>                   : builders.Builder<any, TNull> ;
-    [Type.IntervalDayTime      ]: T extends type.IntervalDayTime      ? builders.IntervalDayTimeBuilder<TNull>               : builders.Builder<any, TNull> ;
-    [Type.IntervalYearMonth    ]: T extends type.IntervalYearMonth    ? builders.IntervalYearMonthBuilder<TNull>             : builders.Builder<any, TNull> ;
-    [Type.Map                  ]: T extends type.Map_                 ? builders.MapBuilder<T['dataTypes'], TNull>           : builders.Builder<any, TNull> ;
-    [Type.List                 ]: T extends type.List                 ? builders.ListBuilder<T['valueType'], TNull>          : builders.Builder<any, TNull> ;
-    [Type.Struct               ]: T extends type.Struct               ? builders.StructBuilder<T['dataTypes'], TNull>        : builders.Builder<any, TNull> ;
-    [Type.Dictionary           ]: T extends type.Dictionary           ? builders.DictionaryBuilder<T, TNull>                 : builders.Builder<any, TNull> ;
-    [Type.FixedSizeList        ]: T extends type.FixedSizeList        ? builders.FixedSizeListBuilder<T['valueType'], TNull> : builders.Builder<any, TNull> ;
+    [key: number               ]:                                                                                                  builders.Builder<any, TNull> ;
+    [Type.Null                 ]: T extends type.Null                 ? builders.NullBuilder<TNull>                              : builders.Builder<any, TNull> ;
+    [Type.Bool                 ]: T extends type.Bool                 ? builders.BoolBuilder<TNull>                              : builders.Builder<any, TNull> ;
+    [Type.Int8                 ]: T extends type.Int8                 ? builders.Int8Builder<TNull>                              : builders.Builder<any, TNull> ;
+    [Type.Int16                ]: T extends type.Int16                ? builders.Int16Builder<TNull>                             : builders.Builder<any, TNull> ;
+    [Type.Int32                ]: T extends type.Int32                ? builders.Int32Builder<TNull>                             : builders.Builder<any, TNull> ;
+    [Type.Int64                ]: T extends type.Int64                ? builders.Int64Builder<TNull>                             : builders.Builder<any, TNull> ;
+    [Type.Uint8                ]: T extends type.Uint8                ? builders.Uint8Builder<TNull>                             : builders.Builder<any, TNull> ;
+    [Type.Uint16               ]: T extends type.Uint16               ? builders.Uint16Builder<TNull>                            : builders.Builder<any, TNull> ;
+    [Type.Uint32               ]: T extends type.Uint32               ? builders.Uint32Builder<TNull>                            : builders.Builder<any, TNull> ;
+    [Type.Uint64               ]: T extends type.Uint64               ? builders.Uint64Builder<TNull>                            : builders.Builder<any, TNull> ;
+    [Type.Int                  ]: T extends type.Int                  ? builders.IntBuilder<T, TNull>                            : builders.Builder<any, TNull> ;
+    [Type.Float16              ]: T extends type.Float16              ? builders.Float16Builder<TNull>                           : builders.Builder<any, TNull> ;
+    [Type.Float32              ]: T extends type.Float32              ? builders.Float32Builder<TNull>                           : builders.Builder<any, TNull> ;
+    [Type.Float64              ]: T extends type.Float64              ? builders.Float64Builder<TNull>                           : builders.Builder<any, TNull> ;
+    [Type.Float                ]: T extends type.Float                ? builders.FloatBuilder<T, TNull>                          : builders.Builder<any, TNull> ;
+    [Type.Utf8                 ]: T extends type.Utf8                 ? builders.Utf8Builder<TNull>                              : builders.Builder<any, TNull> ;
+    [Type.Binary               ]: T extends type.Binary               ? builders.BinaryBuilder<TNull>                            : builders.Builder<any, TNull> ;
+    [Type.FixedSizeBinary      ]: T extends type.FixedSizeBinary      ? builders.FixedSizeBinaryBuilder<TNull>                   : builders.Builder<any, TNull> ;
+    [Type.Date                 ]: T extends type.Date_                ? builders.DateBuilder<T, TNull>                           : builders.Builder<any, TNull> ;
+    [Type.DateDay              ]: T extends type.DateDay              ? builders.DateDayBuilder<TNull>                           : builders.Builder<any, TNull> ;
+    [Type.DateMillisecond      ]: T extends type.DateMillisecond      ? builders.DateMillisecondBuilder<TNull>                   : builders.Builder<any, TNull> ;
+    [Type.Timestamp            ]: T extends type.Timestamp            ? builders.TimestampBuilder<T, TNull>                      : builders.Builder<any, TNull> ;
+    [Type.TimestampSecond      ]: T extends type.TimestampSecond      ? builders.TimestampSecondBuilder<TNull>                   : builders.Builder<any, TNull> ;
+    [Type.TimestampMillisecond ]: T extends type.TimestampMillisecond ? builders.TimestampMillisecondBuilder<TNull>              : builders.Builder<any, TNull> ;
+    [Type.TimestampMicrosecond ]: T extends type.TimestampMicrosecond ? builders.TimestampMicrosecondBuilder<TNull>              : builders.Builder<any, TNull> ;
+    [Type.TimestampNanosecond  ]: T extends type.TimestampNanosecond  ? builders.TimestampNanosecondBuilder<TNull>               : builders.Builder<any, TNull> ;
+    [Type.Time                 ]: T extends type.Time                 ? builders.TimeBuilder<T, TNull>                           : builders.Builder<any, TNull> ;
+    [Type.TimeSecond           ]: T extends type.TimeSecond           ? builders.TimeSecondBuilder<TNull>                        : builders.Builder<any, TNull> ;
+    [Type.TimeMillisecond      ]: T extends type.TimeMillisecond      ? builders.TimeMillisecondBuilder<TNull>                   : builders.Builder<any, TNull> ;
+    [Type.TimeMicrosecond      ]: T extends type.TimeMicrosecond      ? builders.TimeMicrosecondBuilder<TNull>                   : builders.Builder<any, TNull> ;
+    [Type.TimeNanosecond       ]: T extends type.TimeNanosecond       ? builders.TimeNanosecondBuilder<TNull>                    : builders.Builder<any, TNull> ;
+    [Type.Decimal              ]: T extends type.Decimal              ? builders.DecimalBuilder<TNull>                           : builders.Builder<any, TNull> ;
+    [Type.Union                ]: T extends type.Union                ? builders.UnionBuilder<T, TNull>                          : builders.Builder<any, TNull> ;
+    [Type.DenseUnion           ]: T extends type.DenseUnion           ? builders.DenseUnionBuilder<T, TNull>                     : builders.Builder<any, TNull> ;
+    [Type.SparseUnion          ]: T extends type.SparseUnion          ? builders.SparseUnionBuilder<T, TNull>                    : builders.Builder<any, TNull> ;
+    [Type.Interval             ]: T extends type.Interval             ? builders.IntervalBuilder<T, TNull>                       : builders.Builder<any, TNull> ;
+    [Type.IntervalDayTime      ]: T extends type.IntervalDayTime      ? builders.IntervalDayTimeBuilder<TNull>                   : builders.Builder<any, TNull> ;
+    [Type.IntervalYearMonth    ]: T extends type.IntervalYearMonth    ? builders.IntervalYearMonthBuilder<TNull>                 : builders.Builder<any, TNull> ;
+    [Type.Map                  ]: T extends type.Map_                 ? builders.MapBuilder<T['keyType'], T['valueType'], TNull> : builders.Builder<any, TNull> ;
+    [Type.List                 ]: T extends type.List                 ? builders.ListBuilder<T['valueType'], TNull>              : builders.Builder<any, TNull> ;
+    [Type.Struct               ]: T extends type.Struct               ? builders.StructBuilder<T['dataTypes'], TNull>            : builders.Builder<any, TNull> ;
+    [Type.Dictionary           ]: T extends type.Dictionary           ? builders.DictionaryBuilder<T, TNull>                     : builders.Builder<any, TNull> ;
+    [Type.FixedSizeList        ]: T extends type.FixedSizeList        ? builders.FixedSizeListBuilder<T['valueType'], TNull>     : builders.Builder<any, TNull> ;
 }[T['TType']];
diff --git a/js/src/ipc/metadata/json.ts b/js/src/ipc/metadata/json.ts
index 640a8dd..983fa7e 100644
--- a/js/src/ipc/metadata/json.ts
+++ b/js/src/ipc/metadata/json.ts
@@ -198,7 +198,7 @@ function typeFromJSON(f: any, children?: Field[]): DataType<any> {
         }
         case 'map': {
             const t = f['type'];
-            return new Map_(children || [], t['keysSorted']);
+            return new Map_((children || [])[0], t['keysSorted']);
         }
     }
     throw new Error(`Unrecognized type: "${typeId}"`);
diff --git a/js/src/ipc/metadata/message.ts b/js/src/ipc/metadata/message.ts
index 7f40e8d..c7526f2 100644
--- a/js/src/ipc/metadata/message.ts
+++ b/js/src/ipc/metadata/message.ts
@@ -468,7 +468,7 @@ function decodeFieldType(f: _Field, children?: Field[]): DataType<any> {
         }
         case Type.Map: {
             const t = f.type(new Schema_.org.apache.arrow.flatbuf.Map())!;
-            return new Map_(children || [], t.keysSorted());
+            return new Map_((children || [])[0], t.keysSorted());
         }
     }
     throw new Error(`Unrecognized type: "${Type[typeId]}" (${typeId})`);
diff --git a/js/src/recordbatch.ts b/js/src/recordbatch.ts
index af4991e..bde561d 100644
--- a/js/src/recordbatch.ts
+++ b/js/src/recordbatch.ts
@@ -22,37 +22,36 @@ import { Visitor } from './visitor';
 import { Schema, Field } from './schema';
 import { isIterable } from './util/compat';
 import { Chunked } from './vector/chunked';
-import { MapVector } from './vector/index';
 import { selectFieldArgs } from './util/args';
-import { DataType, Struct, Map_, Dictionary } from './type';
+import { DataType, Struct, Dictionary } from './type';
 import { ensureSameLengthData } from './util/recordbatch';
 import { Clonable, Sliceable, Applicative } from './vector';
-import { VectorBuilderOptions, VectorBuilderOptionsAsync } from './vector/index';
+import { StructVector, VectorBuilderOptions, VectorBuilderOptionsAsync } from './vector/index';
 
 type VectorMap = { [key: string]: Vector };
 type Fields<T extends { [key: string]: DataType }> = (keyof T)[] | Field<T[keyof T]>[];
 type ChildData<T extends { [key: string]: DataType }> = (Data<T[keyof T]> | Vector<T[keyof T]>)[];
 
 export interface RecordBatch<T extends { [key: string]: DataType } = any> {
-    concat(...others: Vector<Map_<T>>[]): Table<T>;
+    concat(...others: Vector<Struct<T>>[]): Table<T>;
     slice(begin?: number, end?: number): RecordBatch<T>;
-    clone(data: Data<Map_<T>>, children?: Vector[]): RecordBatch<T>;
+    clone(data: Data<Struct<T>>, children?: Vector[]): RecordBatch<T>;
 }
 
 export class RecordBatch<T extends { [key: string]: DataType } = any>
-    extends MapVector<T>
+    extends StructVector<T>
     implements Clonable<RecordBatch<T>>,
                Sliceable<RecordBatch<T>>,
-               Applicative<Map_<T>, Table<T>> {
+               Applicative<Struct<T>, Table<T>> {
 
-    public static from<T extends { [key: string]: DataType } = any, TNull = any>(options: VectorBuilderOptions<Struct<T> | Map_<T>, TNull>): Table<T>;
-    public static from<T extends { [key: string]: DataType } = any, TNull = any>(options: VectorBuilderOptionsAsync<Struct<T> | Map_<T>, TNull>): Promise<Table<T>>;
+    public static from<T extends { [key: string]: DataType } = any, TNull = any>(options: VectorBuilderOptions<Struct<T>, TNull>): Table<T>;
+    public static from<T extends { [key: string]: DataType } = any, TNull = any>(options: VectorBuilderOptionsAsync<Struct<T>, TNull>): Promise<Table<T>>;
     /** @nocollapse */
-    public static from<T extends { [key: string]: DataType } = any, TNull = any>(options: VectorBuilderOptions<Struct<T> | Map_<T>, TNull> | VectorBuilderOptionsAsync<Struct<T> | Map_<T>, TNull>) {
-        if (isIterable<(Struct<T> | Map_<T>)['TValue'] | TNull>(options['values'])) {
-            return Table.from(options as VectorBuilderOptions<Struct<T> | Map_<T>, TNull>);
+    public static from<T extends { [key: string]: DataType } = any, TNull = any>(options: VectorBuilderOptions<Struct<T>, TNull> | VectorBuilderOptionsAsync<Struct<T>, TNull>) {
+        if (isIterable<(Struct<T>)['TValue'] | TNull>(options['values'])) {
+            return Table.from(options as VectorBuilderOptions<Struct<T>, TNull>);
         }
-        return Table.from(options as VectorBuilderOptionsAsync<Struct<T> | Map_<T>, TNull>);
+        return Table.from(options as VectorBuilderOptionsAsync<Struct<T>, TNull>);
     }
 
     public static new<T extends VectorMap = any>(children: T): RecordBatch<{ [P in keyof T]: T[P]['type'] }>;
@@ -68,27 +67,27 @@ export class RecordBatch<T extends { [key: string]: DataType } = any>
     protected _dictionaries?: Map<number, Vector>;
 
     constructor(schema: Schema<T>, length: number, children: (Data | Vector)[]);
-    constructor(schema: Schema<T>, data: Data<Map_<T>>, children?: Vector[]);
+    constructor(schema: Schema<T>, data: Data<Struct<T>>, children?: Vector[]);
     constructor(...args: any[]) {
-        let data: Data<Map_<T>>;
+        let data: Data<Struct<T>>;
         let schema = args[0] as Schema<T>;
         let children: Vector[] | undefined;
         if (args[1] instanceof Data) {
-            [, data, children] = (args as [any, Data<Map_<T>>, Vector<T[keyof T]>[]?]);
+            [, data, children] = (args as [any, Data<Struct<T>>, Vector<T[keyof T]>[]?]);
         } else {
             const fields = schema.fields as Field<T[keyof T]>[];
             const [, length, childData] = args as [any, number, Data<T[keyof T]>[]];
-            data = Data.Map(new Map_<T>(fields), 0, length, 0, null, childData);
+            data = Data.Struct(new Struct<T>(fields), 0, length, 0, null, childData);
         }
         super(data, children);
         this._schema = schema;
     }
 
-    public clone(data: Data<Map_<T>>, children = this._children) {
+    public clone(data: Data<Struct<T>>, children = this._children) {
         return new RecordBatch<T>(this._schema, data, children);
     }
 
-    public concat(...others: Vector<Map_<T>>[]): Table<T> {
+    public concat(...others: Vector<Struct<T>>[]): Table<T> {
         const schema = this._schema, chunks = Chunked.flatten(this, ...others);
         return new Table(schema, chunks.map(({ data }) => new RecordBatch(schema, data)));
     }
@@ -130,7 +129,7 @@ class DictionaryCollector extends Visitor {
     public dictionaries = new Map<number, Vector>();
     public static collect<T extends RecordBatch>(batch: T) {
         return new DictionaryCollector().visit(
-            batch.data, new Map_(batch.schema.fields)
+            batch.data, new Struct(batch.schema.fields)
         ).dictionaries;
     }
     public visit(data: Data, type: DataType) {
diff --git a/js/src/table.ts b/js/src/table.ts
index 25fcf23..b7cdbe2 100644
--- a/js/src/table.ts
+++ b/js/src/table.ts
@@ -21,15 +21,13 @@ import { Schema, Field } from './schema';
 import { RecordBatch, _InternalEmptyPlaceholderRecordBatch } from './recordbatch';
 import { DataFrame } from './compute/dataframe';
 import { RecordBatchReader } from './ipc/reader';
-import { DataType, RowLike, Struct, Map_ } from './type';
+import { DataType, RowLike, Struct } from './type';
 import { selectColumnArgs, selectArgs } from './util/args';
 import { Clonable, Sliceable, Applicative } from './vector';
 import { isPromise, isIterable, isAsyncIterable } from './util/compat';
-import { distributeColumnsIntoRecordBatches } from './util/recordbatch';
-import { distributeVectorsIntoRecordBatches } from './util/recordbatch';
-import { Vector, Chunked, MapVector, StructVector } from './vector/index';
 import { RecordBatchFileWriter, RecordBatchStreamWriter } from './ipc/writer';
-import { VectorBuilderOptions, VectorBuilderOptionsAsync } from './vector/index';
+import { distributeColumnsIntoRecordBatches, distributeVectorsIntoRecordBatches } from './util/recordbatch';
+import { Vector, Chunked, StructVector, VectorBuilderOptions, VectorBuilderOptionsAsync } from './vector/index';
 
 type VectorMap = { [key: string]: Vector };
 type Fields<T extends { [key: string]: DataType }> = (keyof T)[] | Field<T[keyof T]>[];
@@ -42,7 +40,7 @@ export interface Table<T extends { [key: string]: DataType } = any> {
     [Symbol.iterator](): IterableIterator<RowLike<T>>;
 
     slice(begin?: number, end?: number): Table<T>;
-    concat(...others: Vector<Map_<T>>[]): Table<T>;
+    concat(...others: Vector<Struct<T>>[]): Table<T>;
     clone(chunks?: RecordBatch<T>[], offsets?: Uint32Array): Table<T>;
 
     scan(next: import('./compute/dataframe').NextFunc, bind?: import('./compute/dataframe').BindFunc): void;
@@ -51,11 +49,11 @@ export interface Table<T extends { [key: string]: DataType } = any> {
 }
 
 export class Table<T extends { [key: string]: DataType } = any>
-    extends Chunked<Map_<T>>
+    extends Chunked<Struct<T>>
     implements DataFrame<T>,
                Clonable<Table<T>>,
                Sliceable<Table<T>>,
-               Applicative<Map_<T>, Table<T>> {
+               Applicative<Struct<T>, Table<T>> {
 
     /** @nocollapse */
     public static empty<T extends { [key: string]: DataType } = {}>(schema = new Schema<T>([])) { return new Table<T>(schema, []); }
@@ -69,8 +67,8 @@ export class Table<T extends { [key: string]: DataType } = any>
     public static from<T extends { [key: string]: DataType } = any>(source: import('./ipc/reader').FromArg4): Promise<Table<T>>;
     public static from<T extends { [key: string]: DataType } = any>(source: import('./ipc/reader').FromArg5): Promise<Table<T>>;
     public static from<T extends { [key: string]: DataType } = any>(source: PromiseLike<RecordBatchReader<T>>): Promise<Table<T>>;
-    public static from<T extends { [key: string]: DataType } = any, TNull = any>(options: VectorBuilderOptions<Struct<T> | Map_<T>, TNull>): Table<T>;
-    public static from<T extends { [key: string]: DataType } = any, TNull = any>(options: VectorBuilderOptionsAsync<Struct<T> | Map_<T>, TNull>): Promise<Table<T>>;
+    public static from<T extends { [key: string]: DataType } = any, TNull = any>(options: VectorBuilderOptions<Struct<T>, TNull>): Table<T>;
+    public static from<T extends { [key: string]: DataType } = any, TNull = any>(options: VectorBuilderOptionsAsync<Struct<T>, TNull>): Promise<Table<T>>;
     /** @nocollapse */
     public static from<T extends { [key: string]: DataType } = any, TNull = any>(input?: any) {
 
@@ -111,11 +109,6 @@ export class Table<T extends { [key: string]: DataType } = any>
     }
 
     /** @nocollapse */
-    public static fromMap<T extends { [key: string]: DataType } = any>(vector: Vector<Map_<T>>) {
-        return Table.new<T>(vector.data.childData as Data<T[keyof T]>[], vector.type.children);
-    }
-
-    /** @nocollapse */
     public static fromStruct<T extends { [key: string]: DataType } = any>(vector: Vector<Struct<T>>) {
         return Table.new<T>(vector.data.childData as Data<T[keyof T]>[], vector.type.children);
     }
@@ -196,7 +189,7 @@ export class Table<T extends { [key: string]: DataType } = any>
 
         chunks[0] || (chunks[0] = new _InternalEmptyPlaceholderRecordBatch(schema));
 
-        super(new Map_(schema.fields), chunks);
+        super(new Struct(schema.fields), chunks);
 
         this._schema = schema;
         this._chunks = chunks;
@@ -283,21 +276,17 @@ export class Table<T extends { [key: string]: DataType } = any>
     }
 }
 
-function tableFromIterable<T extends { [key: string]: DataType } = any, TNull = any>(input: VectorBuilderOptions<Struct<T> | Map_<T>, TNull>) {
+function tableFromIterable<T extends { [key: string]: DataType } = any, TNull = any>(input: VectorBuilderOptions<Struct<T>, TNull>) {
     const { type } = input;
-    if (type instanceof Map_) {
-        return Table.fromMap(MapVector.from(input as VectorBuilderOptions<Map_<T>, TNull>));
-    } else if (type instanceof Struct) {
+    if (type instanceof Struct) {
         return Table.fromStruct(StructVector.from(input as VectorBuilderOptions<Struct<T>, TNull>));
     }
     return null;
 }
 
-function tableFromAsyncIterable<T extends { [key: string]: DataType } = any, TNull = any>(input: VectorBuilderOptionsAsync<Struct<T> | Map_<T>, TNull>) {
+function tableFromAsyncIterable<T extends { [key: string]: DataType } = any, TNull = any>(input: VectorBuilderOptionsAsync<Struct<T>, TNull>) {
     const { type } = input;
-    if (type instanceof Map_) {
-        return MapVector.from(input as VectorBuilderOptionsAsync<Map_<T>, TNull>).then((vector) => Table.fromMap(vector));
-    } else if (type instanceof Struct) {
+    if (type instanceof Struct) {
         return StructVector.from(input as VectorBuilderOptionsAsync<Struct<T>, TNull>).then((vector) => Table.fromStruct(vector));
     }
     return null;
diff --git a/js/src/type.ts b/js/src/type.ts
index d1aee98..602a838 100644
--- a/js/src/type.ts
+++ b/js/src/type.ts
@@ -38,9 +38,16 @@ export type IntBitWidth = 8 | 16 | 32 | 64;
 export type IsSigned = { 'true': true; 'false': false };
 /** @ignore */
 export type RowLike<T extends { [key: string]: DataType }> =
-      ( Iterable<T[keyof T]['TValue'] | null> )
+      ( Iterable<[string, T[keyof T]['TValue'] | null]> )
     & { [P in keyof T]: T[P]['TValue'] | null }
     & { get<K extends keyof T>(key: K): T[K]['TValue'] | null; }
+    & { set<K extends keyof T>(key: K, val: T[K]['TValue'] | null): void; }
+    ;
+
+/** @ignore */
+export type MapLike<K extends DataType = any, V extends DataType = any> =
+      { [P in K['TValue']]: V['TValue'] | null }
+    & ( Map<K['TValue'], V['TValue'] | null> )
     ;
 
 export interface DataType<TType extends Type = Type, TChildren extends { [key: string]: DataType } = any> {
@@ -426,7 +433,7 @@ export class Struct<T extends { [key: string]: DataType } = any> extends DataTyp
         this.children = children;
     }
     public get typeId() { return Type.Struct as Type.Struct; }
-    public toString() { return `Struct<[${this.children.map((f) => f.type).join(`, `)}]>`; }
+    public toString() { return `Struct<{${this.children.map((f) => `${f.name}:${f.type}`).join(`, `)}}>`; }
     protected static [Symbol.toStringTag] = ((proto: Struct) => {
         (<any> proto).children = null;
         return proto[Symbol.toStringTag] = 'Struct';
@@ -522,15 +529,25 @@ export class FixedSizeList<T extends DataType = any> extends DataType<Type.Fixed
 }
 
 /** @ignore */
-export interface Map_<T extends { [key: string]: DataType } = any> extends DataType<Type.Map> { TArray: IterableArrayLike<RowLike<T>>; TValue: RowLike<T>; dataTypes: T; }
+export interface Map_<TKey extends DataType = any, TValue extends DataType = any> extends DataType<Type.Map> {
+    TArray: IterableArrayLike<Map<TKey['TValue'], TValue['TValue'] | null>>;
+    TChild: Struct<{ key: TKey, value: TValue }>;
+    TValue: MapLike<TKey, TValue>;
+}
+
 /** @ignore */
-export class Map_<T extends { [key: string]: DataType } = any> extends DataType<Type.Map, T> {
-    constructor(public readonly children: Field<T[keyof T]>[],
-                public readonly keysSorted: boolean = false) {
+export class Map_<TKey extends DataType = any, TValue extends DataType = any> extends DataType<Type.Map> {
+    constructor(child: Field<Struct<{ key: TKey, value: TValue }>>, keysSorted = false) {
         super();
+        this.children = [child];
+        this.keysSorted = keysSorted;
     }
+    public readonly keysSorted: boolean;
+    public readonly children: Field<Struct<{ key: TKey, value: TValue }>>[];
     public get typeId() { return Type.Map as Type.Map; }
-    public toString() { return `Map<{${this.children.map((f) => `${f.name}:${f.type}`).join(`, `)}}>`; }
+    public get keyType(): TKey { return this.children[0].type.children[0].type as TKey; }
+    public get valueType(): TValue { return this.children[0].type.children[1].type as TValue; }
+    public toString() { return `Map<{${this.children[0].type.children.map((f) => `${f.name}:${f.type}`).join(`, `)}}>`; }
     protected static [Symbol.toStringTag] = ((proto: Map_) => {
         (<any> proto).children = null;
         (<any> proto).keysSorted = null;
diff --git a/js/src/util/vector.ts b/js/src/util/vector.ts
index 9fd9d99..56cd2e3 100644
--- a/js/src/util/vector.ts
+++ b/js/src/util/vector.ts
@@ -16,7 +16,7 @@
 // under the License.
 
 import { Vector } from '../vector';
-import { Row, kLength } from '../vector/row';
+import { MapRow, StructRow } from '../vector/row';
 import { compareArrayLike } from '../util/buffer';
 import { BigInt, BigIntAvailable } from './compat';
 
@@ -81,110 +81,118 @@ export function createElementComparator(search: any) {
         const valueOfSearch = search.valueOf();
         return (value: any) => value instanceof Date ? (value.valueOf() === valueOfSearch) : false;
     }
+    // Compare TypedArrays
     if (ArrayBuffer.isView(search)) {
         return (value: any) => value ? compareArrayLike(search, value) : false;
     }
+    // Compare Maps and Rows
+    if (search instanceof Map) { return creatMapComparator(search); }
     // Compare Array-likes
-    if (Array.isArray(search)) {
-        return createArrayLikeComparator(search);
-    }
-    // Compare Rows
-    if (search instanceof Row) {
-        return createRowComparator(search);
-    }
+    if (Array.isArray(search)) { return createArrayLikeComparator(search); }
     // Compare Vectors
-    if (search instanceof Vector) {
-        return createVectorComparator(search);
-    }
+    if (search instanceof Vector) { return createVectorComparator(search); }
     // Compare non-empty Objects
-    const keys = Object.keys(search);
-    if (keys.length > 0) {
-        return createObjectKeysComparator(search, keys);
+    return createObjectComparator(search);
+}
+
+/** @ignore */
+function createArrayLikeComparator(lhs: ArrayLike<any>) {
+    const comparitors = [] as ((x: any) => boolean)[];
+    for (let i = -1, n = lhs.length; ++i < n;) {
+        comparitors[i] = createElementComparator(lhs[i]);
     }
-    // No valid comparator
-    return () => false;
+    return createSubElementsComparator(comparitors);
 }
 
 /** @ignore */
-function createArrayLikeComparator(search: ArrayLike<any>) {
-    const n = search.length;
-    const fns = [] as ((x: any) => boolean)[];
-    for (let i = -1; ++i < n;) {
-        fns[i] = createElementComparator((search as any)[i]);
+function creatMapComparator(lhs: Map<any, any>) {
+    let i = -1;
+    const comparitors = [] as ((x: any) => boolean)[];
+    lhs.forEach((v) => comparitors[++i] = createElementComparator(v));
+    return createSubElementsComparator(comparitors);
+}
+
+/** @ignore */
+function createVectorComparator(lhs: Vector<any>) {
+    const comparitors = [] as ((x: any) => boolean)[];
+    for (let i = -1, n = lhs.length; ++i < n;) {
+        comparitors[i] = createElementComparator(lhs.get(i));
     }
-    return (value: any) => {
-        if (!value) { return false; }
-        // Handle the case where the search element is an Array, but the
-        // values are Rows or Vectors, e.g. list.indexOf(['foo', 'bar'])
-        if (value instanceof Row) {
-            if (value[kLength] !== n) { return false; }
-            for (let i = -1; ++i < n;) {
-                if (!(fns[i](value.get(i)))) { return false; }
-            }
-            return true;
-        }
-        if (value.length !== n) { return false; }
-        if (value instanceof Vector) {
-            for (let i = -1; ++i < n;) {
-                if (!(fns[i](value.get(i)))) { return false; }
-            }
-            return true;
-        }
-        for (let i = -1; ++i < n;) {
-            if (!(fns[i](value[i]))) { return false; }
-        }
-        return true;
-    };
+    return createSubElementsComparator(comparitors);
 }
 
 /** @ignore */
-function createRowComparator(search: Row<any>) {
-    const n = search[kLength];
-    const C = search.constructor as any;
-    const fns = [] as ((x: any) => boolean)[];
-    for (let i = -1; ++i < n;) {
-        fns[i] = createElementComparator(search.get(i));
+function createObjectComparator(lhs: any) {
+    const keys = Object.keys(lhs);
+    // Only compare non-empty Objects
+    if (keys.length === 0) { return () => false; }
+    const comparitors = [] as ((x: any) => boolean)[];
+    for (let i = -1, n = keys.length; ++i < n;) {
+        comparitors[i] = createElementComparator(lhs[keys[i]]);
     }
-    return (value: any) => {
-        if (!(value instanceof C)) { return false; }
-        if (!(value[kLength] === n)) { return false; }
-        for (let i = -1; ++i < n;) {
-            if (!(fns[i](value.get(i)))) { return false; }
+    return createSubElementsComparator(comparitors, keys);
+}
+
+function createSubElementsComparator(comparitors: ((x: any) => boolean)[], keys?: Iterable<string>) {
+    return (rhs: any) => {
+        if (!rhs || typeof rhs !== 'object') {
+            return false;
         }
-        return true;
+        switch (rhs.constructor) {
+            case Array: return compareArray(comparitors, rhs);
+            case Map:
+            case MapRow:
+            case StructRow:
+                return compareObject(comparitors, rhs, rhs.keys());
+            case Object:
+            case undefined: // support `Object.create(null)` objects
+                return compareObject(comparitors, rhs, keys || Object.keys(rhs));
+        }
+        return rhs instanceof Vector ? compareVector(comparitors, rhs) : false;
     };
 }
 
-/** @ignore */
-function createVectorComparator(search: Vector<any>) {
-    const n = search.length;
-    const C = search.constructor as any;
-    const fns = [] as ((x: any) => boolean)[];
+function compareArray(comparitors: ((x: any) => boolean)[], arr: any[]) {
+    const n = comparitors.length;
+    if (arr.length !== n) { return false; }
     for (let i = -1; ++i < n;) {
-        fns[i] = createElementComparator((search as any).get(i));
+        if (!(comparitors[i](arr[i]))) { return false; }
     }
-    return (value: any) => {
-        if (!(value instanceof C)) { return false; }
-        if (!(value.length === n)) { return false; }
-        for (let i = -1; ++i < n;) {
-            if (!(fns[i](value.get(i)))) { return false; }
-        }
-        return true;
-    };
+    return true;
 }
 
-/** @ignore */
-function createObjectKeysComparator(search: any, keys: string[]) {
-    const n = keys.length;
-    const fns = [] as ((x: any) => boolean)[];
+function compareVector(comparitors: ((x: any) => boolean)[], vec: Vector) {
+    const n = comparitors.length;
+    if (vec.length !== n) { return false; }
     for (let i = -1; ++i < n;) {
-        fns[i] = createElementComparator(search[keys[i]]);
+        if (!(comparitors[i](vec.get(i)))) { return false; }
     }
-    return (value: any) => {
-        if (!value || typeof value !== 'object') { return false; }
-        for (let i = -1; ++i < n;) {
-            if (!(fns[i](value[keys[i]]))) { return false; }
+    return true;
+}
+
+function compareObject(comparitors: ((x: any) => boolean)[], obj: Map<any, any>, keys: Iterable<string>) {
+
+    const lKeyItr = keys[Symbol.iterator]();
+    const rKeyItr = obj instanceof Map ? obj.keys() : Object.keys(obj)[Symbol.iterator]();
+    const rValItr = obj instanceof Map ? obj.values() : Object.values(obj)[Symbol.iterator]();
+
+    let i = 0;
+    let n = comparitors.length;
+    let rVal = rValItr.next();
+    let lKey = lKeyItr.next();
+    let rKey = rKeyItr.next();
+
+    for (; i < n && !lKey.done && !rKey.done && !rVal.done;
+         ++i, lKey = lKeyItr.next(), rKey = rKeyItr.next(), rVal = rValItr.next()) {
+        if (lKey.value !== rKey.value || !comparitors[i](rVal.value)) {
+            break;
         }
+    }
+    if (i === n && lKey.done && rKey.done && rVal.done) {
         return true;
-    };
+    }
+    lKeyItr.return && lKeyItr.return();
+    rKeyItr.return && rKeyItr.return();
+    rValItr.return && rValItr.return();
+    return false;
 }
diff --git a/js/src/vector/base.ts b/js/src/vector/base.ts
index 72aa673..5fdf1c2 100644
--- a/js/src/vector/base.ts
+++ b/js/src/vector/base.ts
@@ -99,7 +99,7 @@ export abstract class BaseVector<T extends DataType = any> extends AbstractVecto
     public toJSON(): any { return [...this]; }
 
     protected _sliceInternal(self: this, begin: number, end: number) {
-        return self.clone(self.data.slice(begin, end - begin));
+        return self.clone(self.data.slice(begin, end - begin), null!);
     }
 
     // @ts-ignore
diff --git a/js/src/vector/index.ts b/js/src/vector/index.ts
index 546e21b..c110dba 100644
--- a/js/src/vector/index.ts
+++ b/js/src/vector/index.ts
@@ -15,7 +15,6 @@
 // specific language governing permissions and limitations
 // under the License.
 
-export { Row } from './row';
 export { Vector } from '../vector';
 export { BaseVector } from './base';
 export { BinaryVector } from './binary';
@@ -37,6 +36,7 @@ export { TimestampVector, TimestampSecondVector, TimestampMillisecondVector, Tim
 export { TimeVector, TimeSecondVector, TimeMillisecondVector, TimeMicrosecondVector, TimeNanosecondVector } from './time';
 export { UnionVector, DenseUnionVector, SparseUnionVector } from './union';
 export { Utf8Vector } from './utf8';
+export { MapRow, StructRow } from './row';
 
 import * as fn from '../util/fn';
 import { Data } from '../data';
diff --git a/js/src/vector/map.ts b/js/src/vector/map.ts
index f038471..db7726a 100644
--- a/js/src/vector/map.ts
+++ b/js/src/vector/map.ts
@@ -15,20 +15,21 @@
 // specific language governing permissions and limitations
 // under the License.
 
+import { MapRow } from './row';
 import { Field } from '../schema';
 import { Vector } from '../vector';
 import { BaseVector } from './base';
-import { RowProxyGenerator } from './row';
-import { DataType, Map_, Struct } from '../type';
+import { DataType, Map_, Struct, List } from '../type';
 
 /** @ignore */
-export class MapVector<T extends { [key: string]: DataType } = any> extends BaseVector<Map_<T>> {
-    public asStruct() {
-        return Vector.new(this.data.clone(new Struct<T>(this.type.children as Field<T[keyof T]>[])));
+export class MapVector<K extends DataType = any, V extends DataType = any> extends BaseVector<Map_<K, V>> {
+    public asList() {
+        const child = this.type.children[0] as Field<Struct<{ key: K, value: V }>>;
+        return Vector.new(this.data.clone(new List<Struct<{ key: K, value: V }>>(child)));
     }
-    // @ts-ignore
-    private _rowProxy: RowProxyGenerator<T>;
-    public get rowProxy(): RowProxyGenerator<T> {
-        return this._rowProxy || (this._rowProxy = RowProxyGenerator.new<T>(this, this.type.children || [], true));
+    public bind(index: number): Map_<K, V>['TValue'] {
+        const child = this.getChildAt<Struct<{ key: K, value: V }>>(0);
+        const { [index]: begin, [index + 1]: end } = this.valueOffsets;
+        return new MapRow(child!.slice(begin, end));
     }
 }
diff --git a/js/src/vector/row.ts b/js/src/vector/row.ts
index 3b3c9cb..7305627 100644
--- a/js/src/vector/row.ts
+++ b/js/src/vector/row.ts
@@ -15,106 +15,281 @@
 // specific language governing permissions and limitations
 // under the License.
 
-import { Field } from '../schema';
-import { MapVector } from '../vector/map';
-import { DataType } from '../type';
+import { Vector } from '../vector';
+import { StructVector } from './struct';
 import { valueToString } from '../util/pretty';
-import { StructVector } from '../vector/struct';
+import { DataType, Struct, RowLike } from '../type';
 
-/** @ignore */ export const kLength = Symbol.for('length');
-/** @ignore */ export const kParent = Symbol.for('parent');
-/** @ignore */ export const kRowIndex = Symbol.for('rowIndex');
-/** @ignore */ const columnDescriptor = { enumerable: true, configurable: false, get: null as any };
-/** @ignore */ const rowLengthDescriptor = { writable: false, enumerable: false, configurable: false, value: -1 };
-/** @ignore */ const rowParentDescriptor = { writable: false, enumerable: false, configurable: false, value: null as any };
+/** @ignore */ const kParent = Symbol.for('parent');
+/** @ignore */ const kRowIndex = Symbol.for('rowIndex');
+/** @ignore */ const kKeyToIdx = Symbol.for('keyToIdx');
+/** @ignore */ const kIdxToVal = Symbol.for('idxToVal');
+/** @ignore */ const kCustomInspect = Symbol.for('nodejs.util.inspect.custom');
 
-/** @ignore */
-export class Row<T extends { [key: string]: DataType }> implements Iterable<T[keyof T]['TValue']> {
-    [key: string]: T[keyof T]['TValue'];
-    // @ts-ignore
-    public [kParent]: MapVector<T> | StructVector<T>;
-    // @ts-ignore
-    public [kRowIndex]: number;
-    // @ts-ignore
-    public readonly [kLength]: number;
-    *[Symbol.iterator]() {
-        for (let i = -1, n = this[kLength]; ++i < n;) {
-            yield this[i];
-        }
+abstract class Row<K extends PropertyKey = any, V = any> implements Map<K, V> {
+
+    public readonly size: number;
+    public readonly [Symbol.toStringTag]: string;
+
+    protected [kRowIndex]: number;
+    protected [kParent]: Vector<Struct>;
+    protected [kKeyToIdx]: Map<K, number>;
+    protected [kIdxToVal]: V[];
+
+    constructor(parent: Vector<Struct>, numKeys: number) {
+        this[kParent] = parent;
+        this.size = numKeys;
     }
-    public get<K extends keyof T>(key: K) {
-        return (this as any)[key] as T[K]['TValue'];
+
+    public abstract keys(): IterableIterator<K>;
+    public abstract values(): IterableIterator<V>;
+    public abstract getKey(idx: number): K;
+    public abstract getIndex(key: K): number;
+    public abstract getValue(idx: number): V;
+    public abstract setValue(idx: number, val: V): void;
+
+    public entries() { return this[Symbol.iterator](); }
+
+    public has(key: K) { return this.get(key) !== undefined; }
+
+    public get(key: K) {
+        let val = undefined;
+        if (key !== null && key !== undefined) {
+            const ktoi = this[kKeyToIdx] || (this[kKeyToIdx] = new Map());
+            let idx = ktoi.get(key);
+            if (idx !== undefined) {
+                const itov = this[kIdxToVal] || (this[kIdxToVal] = new Array(this.size));
+                ((val = itov[idx]) !== undefined) || (itov[idx] = val = this.getValue(idx));
+            } else if ((idx = this.getIndex(key)) > -1) {
+                ktoi.set(key, idx);
+                const itov = this[kIdxToVal] || (this[kIdxToVal] = new Array(this.size));
+                ((val = itov[idx]) !== undefined) || (itov[idx] = val = this.getValue(idx));
+            }
+        }
+        return val;
     }
-    public toJSON(): any {
-        return DataType.isStruct(this[kParent].type) ? [...this] :
-            this[kParent].type.children.reduce((props: any, { name }: Field<T[keyof T]>) => {
-                return (props[name] = (this as any)[name]) && props || props;
-            }, {});
+
+    public set(key: K, val: V) {
+        if (key !== null && key !== undefined) {
+            const ktoi = this[kKeyToIdx] || (this[kKeyToIdx] = new Map());
+            let idx = ktoi.get(key);
+            if (idx === undefined) {
+                ktoi.set(key, idx = this.getIndex(key));
+            }
+            if (idx > -1) {
+                const itov = this[kIdxToVal] || (this[kIdxToVal] = new Array(this.size));
+                itov[idx] = <any> this.setValue(idx, val);
+            }
+        }
+        return this;
     }
-    public inspect() { return this.toString(); }
-    public [Symbol.for('nodejs.util.inspect.custom')]() { return this.toString(); }
-    public toString() {
-        return DataType.isStruct(this[kParent].type) ?
-            `[ ${[...this].map((x) => valueToString(x)).join(', ')} ]` :
-            `{ ${
-                this[kParent].type.children.reduce((xs: string[], { name }: Field<T[keyof T]>) => {
-                    return [...xs, `"${name}": ${valueToString((this as any)[name])}`];
-                }, []).join(', ')
-            } }`
-                ;
+
+    public clear(): void { throw new Error(`Clearing ${this[Symbol.toStringTag]} not supported.`); }
+
+    public delete(_: K): boolean { throw new Error(`Deleting ${this[Symbol.toStringTag]} values not supported.`); }
+
+    public *[Symbol.iterator](): IterableIterator<[K, V]> {
+
+        const ki = this.keys();
+        const vi = this.values();
+        const ktoi = this[kKeyToIdx] || (this[kKeyToIdx] = new Map());
+        const itov = this[kIdxToVal] || (this[kIdxToVal] = new Array(this.size));
+
+        for (let k: K, v: V, i = 0, kr: IteratorResult<K>, vr: IteratorResult<V>;
+            !((kr = ki.next()).done || (vr = vi.next()).done);
+            ++i
+        ) {
+            k = kr.value;
+            v = vr.value;
+            itov[i] = v;
+            ktoi.has(k) || ktoi.set(k, i);
+            yield [k, v];
+        }
     }
-}
 
-/** @ignore */
-export class RowProxyGenerator<T extends { [key: string]: DataType }> {
-    /** @nocollapse */
-    public static new<T extends { [key: string]: DataType }>(parent: MapVector<T> | StructVector<T>, schemaOrFields: T | Field[], fieldsAreEnumerable = false): RowProxyGenerator<T> {
-        let schema: T, fields: Field[];
-        if (Array.isArray(schemaOrFields)) {
-            fields = schemaOrFields;
-        } else {
-            schema = schemaOrFields;
-            fieldsAreEnumerable = true;
-            fields = Object.keys(schema).map((x) => new Field(x, schema[x]));
+    public forEach(callbackfn: (value: V, key: K, map: Map<K, V>) => void, thisArg?: any): void {
+
+        const ki = this.keys();
+        const vi = this.values();
+        const callback = thisArg === undefined ? callbackfn :
+            (v: V, k: K, m: Map<K, V>) => callbackfn.call(thisArg, v, k, m);
+        const ktoi = this[kKeyToIdx] || (this[kKeyToIdx] = new Map());
+        const itov = this[kIdxToVal] || (this[kIdxToVal] = new Array(this.size));
+
+        for (let k: K, v: V, i = 0, kr: IteratorResult<K>, vr: IteratorResult<V>;
+            !((kr = ki.next()).done || (vr = vi.next()).done);
+            ++i
+        ) {
+            k = kr.value;
+            v = vr.value;
+            itov[i] = v;
+            ktoi.has(k) || ktoi.set(k, i);
+            callback(v, k, this);
         }
-        return new RowProxyGenerator<T>(parent, fields, fieldsAreEnumerable);
     }
 
-    private rowPrototype: Row<T>;
+    public toArray() { return [...this.values()]; }
+    public toJSON() {
+        const obj = {} as any;
+        this.forEach((val, key) => obj[key] = val);
+        return obj;
+    }
 
-    private constructor(parent: MapVector<T> | StructVector<T>, fields: Field[], fieldsAreEnumerable: boolean) {
-        const proto = Object.create(Row.prototype);
+    public inspect() { return this.toString(); }
+    public [kCustomInspect]() { return this.toString(); }
+    public toString() {
+        const str: string[] = [];
+        this.forEach((val, key) => {
+            key = valueToString(key);
+            val = valueToString(val);
+            str.push(`${key}: ${val}`);
+        });
+        return `{ ${str.join(', ')} }`;
+    }
 
-        rowParentDescriptor.value = parent;
-        rowLengthDescriptor.value = fields.length;
-        Object.defineProperty(proto, kParent, rowParentDescriptor);
-        Object.defineProperty(proto, kLength, rowLengthDescriptor);
-        fields.forEach((field, columnIndex) => {
-            if (!proto.hasOwnProperty(field.name)) {
-                columnDescriptor.enumerable = fieldsAreEnumerable;
-                columnDescriptor.get || (columnDescriptor.get = this._bindGetter(columnIndex));
-                Object.defineProperty(proto, field.name, columnDescriptor);
-            }
-            if (!proto.hasOwnProperty(columnIndex)) {
-                columnDescriptor.enumerable = !fieldsAreEnumerable;
-                columnDescriptor.get || (columnDescriptor.get = this._bindGetter(columnIndex));
-                Object.defineProperty(proto, columnIndex, columnDescriptor);
-            }
-            columnDescriptor.get = null as any;
+    protected static [Symbol.toStringTag] = ((proto: Row) => {
+        Object.defineProperties(proto, {
+            'size': { writable: true, enumerable: false, configurable: false, value: 0 },
+            [kParent]: { writable: true, enumerable: false, configurable: false, value: null },
+            [kRowIndex]: { writable: true, enumerable: false, configurable: false, value: -1 },
         });
+        return (proto as any)[Symbol.toStringTag] = 'Row';
+    })(Row.prototype);
+}
 
-        this.rowPrototype = proto;
+export class MapRow<K extends DataType = any, V extends DataType = any> extends Row<K['TValue'], V['TValue'] | null> {
+    constructor(slice: Vector<Struct<{ key: K, value: V }>>) {
+        super(slice, slice.length);
+        return createRowProxy(this);
+    }
+    public keys() {
+        return this[kParent].getChildAt(0)![Symbol.iterator]();
+    }
+    public values() {
+        return this[kParent].getChildAt(1)![Symbol.iterator]();
+    }
+    public getKey(idx: number): K['TValue'] {
+        return this[kParent].getChildAt(0)!.get(idx);
+    }
+    public getIndex(key: K['TValue']): number {
+        return this[kParent].getChildAt(0)!.indexOf(key);
     }
+    public getValue(index: number): V['TValue'] | null {
+        return this[kParent].getChildAt(1)!.get(index);
+    }
+    public setValue(index: number, value: V['TValue'] | null): void {
+        this[kParent].getChildAt(1)!.set(index, value);
+    }
+}
 
-    private _bindGetter(columnIndex: number) {
-        return function(this: Row<T>) {
-            const child = this[kParent].getChildAt(columnIndex);
-            return child ? child.get(this[kRowIndex]) : null;
-        };
+export class StructRow<T extends { [key: string]: DataType } = any> extends Row<keyof T, T[keyof T]['TValue'] | null> {
+    constructor(parent: StructVector<T>) {
+        super(parent, parent.type.children.length);
+        return defineRowProxyProperties(this);
+    }
+    public *keys() {
+        for (const field of this[kParent].type.children) {
+            yield field.name as keyof T;
+        }
+    }
+    public *values() {
+        for (const field of this[kParent].type.children) {
+            yield (this as RowLike<T>)[field.name];
+        }
+    }
+    public getKey(idx: number): keyof T {
+        return this[kParent].type.children[idx].name as keyof T;
     }
-    public bind(rowIndex: number) {
-        const bound: Row<T> = Object.create(this.rowPrototype);
-        bound[kRowIndex] = rowIndex;
-        return bound;
+    public getIndex(key: keyof T): number {
+        return this[kParent].type.children.findIndex((f) => f.name === key);
+    }
+    public getValue(index: number): T[keyof T]['TValue'] | null {
+        return this[kParent].getChildAt(index)!.get(this[kRowIndex]);
+    }
+    public setValue(index: number, value: T[keyof T]['TValue'] | null): void {
+        return this[kParent].getChildAt(index)!.set(this[kRowIndex], value);
     }
 }
+
+Object.setPrototypeOf(Row.prototype, Map.prototype);
+
+/** @ignore */
+const defineRowProxyProperties = (() => {
+    const desc = { enumerable: true, configurable: false, get: null as any, set: null as any };
+    return <T extends Row>(row: T) => {
+        let idx = -1, ktoi = row[kKeyToIdx] || (row[kKeyToIdx] = new Map());
+        const getter = (key: any) => function(this: T) { return this.get(key); };
+        const setter = (key: any) => function(this: T, val: any) { return this.set(key, val); };
+        for (const key of row.keys()) {
+            ktoi.set(key, ++idx);
+            desc.get = getter(key);
+            desc.set = setter(key);
+            row.hasOwnProperty(key) || (desc.enumerable = true, Object.defineProperty(row, key, desc));
+            row.hasOwnProperty(idx) || (desc.enumerable = false, Object.defineProperty(row, idx, desc));
+        }
+        desc.get = desc.set = null;
+        return row;
+    };
+})();
+
+/** @ignore */
+const createRowProxy = (() => {
+    if (typeof Proxy === 'undefined') {
+        return defineRowProxyProperties;
+    }
+    const has = Row.prototype.has;
+    const get = Row.prototype.get;
+    const set = Row.prototype.set;
+    const getKey = Row.prototype.getKey;
+    const RowProxyHandler: ProxyHandler<Row> = {
+        isExtensible() { return false; },
+        deleteProperty() { return false; },
+        preventExtensions() { return true; },
+        ownKeys(row: Row) { return [...row.keys()].map((x) => `${x}`); },
+        has(row: Row, key: PropertyKey) {
+            switch (key) {
+                case 'getKey': case 'getIndex': case 'getValue': case 'setValue': case 'toArray': case 'toJSON': case 'inspect':
+                case 'constructor': case 'isPrototypeOf': case 'propertyIsEnumerable': case 'toString': case 'toLocaleString': case 'valueOf':
+                case 'size': case 'has': case 'get': case 'set': case 'clear': case 'delete': case 'keys': case 'values': case 'entries': case 'forEach':
+                case '__proto__': case '__defineGetter__': case '__defineSetter__': case 'hasOwnProperty': case '__lookupGetter__': case '__lookupSetter__':
+                case Symbol.iterator: case Symbol.toStringTag: case kParent: case kRowIndex: case kIdxToVal: case kKeyToIdx: case kCustomInspect:
+                    return true;
+            }
+            if (typeof key === 'number' && !row.has(key)) {
+                key = row.getKey(key);
+            }
+            return row.has(key);
+        },
+        get(row: Row, key: PropertyKey, receiver: any) {
+            switch (key) {
+                case 'getKey': case 'getIndex': case 'getValue': case 'setValue': case 'toArray': case 'toJSON': case 'inspect':
+                case 'constructor': case 'isPrototypeOf': case 'propertyIsEnumerable': case 'toString': case 'toLocaleString': case 'valueOf':
+                case 'size': case 'has': case 'get': case 'set': case 'clear': case 'delete': case 'keys': case 'values': case 'entries': case 'forEach':
+                case '__proto__': case '__defineGetter__': case '__defineSetter__': case 'hasOwnProperty': case '__lookupGetter__': case '__lookupSetter__':
+                case Symbol.iterator: case Symbol.toStringTag: case kParent: case kRowIndex: case kIdxToVal: case kKeyToIdx: case kCustomInspect:
+                    return Reflect.get(row, key, receiver);
+            }
+            if (typeof key === 'number' && !has.call(receiver, key)) {
+                key = getKey.call(receiver, key);
+            }
+            return get.call(receiver, key);
+        },
+        set(row: Row, key: PropertyKey, val: any, receiver: any) {
+            switch (key) {
+                case kParent: case kRowIndex: case kIdxToVal: case kKeyToIdx:
+                    return Reflect.set(row, key, val, receiver);
+                case 'getKey': case 'getIndex': case 'getValue': case 'setValue': case 'toArray': case 'toJSON': case 'inspect':
+                case 'constructor': case 'isPrototypeOf': case 'propertyIsEnumerable': case 'toString': case 'toLocaleString': case 'valueOf':
+                case 'size': case 'has': case 'get': case 'set': case 'clear': case 'delete': case 'keys': case 'values': case 'entries': case 'forEach':
+                case '__proto__': case '__defineGetter__': case '__defineSetter__': case 'hasOwnProperty': case '__lookupGetter__': case '__lookupSetter__':
+                case Symbol.iterator: case Symbol.toStringTag:
+                    return false;
+            }
+            if (typeof key === 'number' && !has.call(receiver, key)) {
+                key = getKey.call(receiver, key);
+            }
+            return has.call(receiver, key) ? !!set.call(receiver, key, val) : false;
+        },
+    };
+    return <T extends Row>(row: T) => new Proxy(row, RowProxyHandler) as T;
+})();
diff --git a/js/src/vector/struct.ts b/js/src/vector/struct.ts
index e5d4907..00af4ae 100644
--- a/js/src/vector/struct.ts
+++ b/js/src/vector/struct.ts
@@ -15,20 +15,19 @@
 // specific language governing permissions and limitations
 // under the License.
 
-import { Field } from '../schema';
-import { Vector } from '../vector';
+import { StructRow } from './row';
 import { BaseVector } from './base';
-import { RowProxyGenerator } from './row';
-import { DataType, Map_, Struct } from '../type';
+import { DataType, Struct } from '../type';
 
+/** @ignore */ const kRowIndex = Symbol.for('rowIndex');
 /** @ignore */
 export class StructVector<T extends { [key: string]: DataType } = any> extends BaseVector<Struct<T>> {
-    public asMap(keysSorted: boolean = false) {
-        return Vector.new(this.data.clone(new Map_<T>(this.type.children as Field<T[keyof T]>[], keysSorted)));
-    }
     // @ts-ignore
-    private _rowProxy: RowProxyGenerator<T>;
-    public get rowProxy(): RowProxyGenerator<T> {
-        return this._rowProxy || (this._rowProxy = RowProxyGenerator.new<T>(this, this.type.children || [], false));
+    private _row: StructRow<T>;
+    public bind(index: number): Struct<T>['TValue'] {
+        const proto = this._row || (this._row = new StructRow<T>(this));
+        const bound = Object.create(proto);
+        bound[kRowIndex] = index;
+        return bound;
     }
 }
diff --git a/js/src/visitor/get.ts b/js/src/visitor/get.ts
index eea1409..733418c 100644
--- a/js/src/visitor/get.ts
+++ b/js/src/visitor/get.ts
@@ -210,11 +210,13 @@ const getList = <T extends List>(vector: VectorType<T>, index: number): T['TValu
 };
 
 /** @ignore */
-const getNested = <
-    S extends { [key: string]: DataType },
-    V extends VectorType<Map_<S>> | VectorType<Struct<S>>
->(vector: V, index: number): V['TValue'] => {
-    return vector.rowProxy.bind(index) as V['TValue'];
+const getMap = <T extends Map_>(vector: VectorType<T>, index: number): T['TValue'] => {
+    return vector.bind(index) as T['TValue'];
+};
+
+/** @ignore */
+const getStruct = <T extends Struct>(vector: VectorType<T>, index: number): T['TValue'] => {
+    return vector.bind(index) as T['TValue'];
 };
 
 /* istanbul ignore next */
@@ -304,7 +306,7 @@ GetVisitor.prototype.visitTimeMicrosecond      =      getTimeMicrosecond;
 GetVisitor.prototype.visitTimeNanosecond       =       getTimeNanosecond;
 GetVisitor.prototype.visitDecimal              =              getDecimal;
 GetVisitor.prototype.visitList                 =                 getList;
-GetVisitor.prototype.visitStruct               =               getNested;
+GetVisitor.prototype.visitStruct               =               getStruct;
 GetVisitor.prototype.visitUnion                =                getUnion;
 GetVisitor.prototype.visitDenseUnion           =           getDenseUnion;
 GetVisitor.prototype.visitSparseUnion          =          getSparseUnion;
@@ -313,7 +315,7 @@ GetVisitor.prototype.visitInterval             =             getInterval;
 GetVisitor.prototype.visitIntervalDayTime      =      getIntervalDayTime;
 GetVisitor.prototype.visitIntervalYearMonth    =    getIntervalYearMonth;
 GetVisitor.prototype.visitFixedSizeList        =        getFixedSizeList;
-GetVisitor.prototype.visitMap                  =               getNested;
+GetVisitor.prototype.visitMap                  =                  getMap;
 
 /** @ignore */
 export const instance = new GetVisitor();
diff --git a/js/src/visitor/jsonvectorassembler.ts b/js/src/visitor/jsonvectorassembler.ts
index a575367..90155c3 100644
--- a/js/src/visitor/jsonvectorassembler.ts
+++ b/js/src/visitor/jsonvectorassembler.ts
@@ -153,6 +153,7 @@ export class JSONVectorAssembler extends Visitor {
     }
     public visitMap<T extends Map_>(vector: V<T>) {
         return {
+            'OFFSET': [...vector.valueOffsets],
             'children': vector.type.children.map((f, i) =>
                 this.visit(new Column(f, [vector.getChildAt(i)!])))
         };
diff --git a/js/src/visitor/set.ts b/js/src/visitor/set.ts
index c7adc65..307c839 100644
--- a/js/src/visitor/set.ts
+++ b/js/src/visitor/set.ts
@@ -16,6 +16,8 @@
 // under the License.
 
 import { Data } from '../data';
+import { Field } from '../schema';
+import { Vector } from '../vector';
 import { Visitor } from '../visitor';
 import { encodeUtf8 } from '../util/utf8';
 import { VectorType } from '../interfaces';
@@ -221,35 +223,34 @@ const setDecimal = <T extends Decimal>({ values }: VectorType<T>, index: number,
 
 /** @ignore */
 const setList = <T extends List>(vector: VectorType<T>, index: number, value: T['TValue']): void => {
-    const values = vector.getChildAt(0)!;
-    const { valueOffsets } = vector;
-    let idx = -1, offset = valueOffsets[index];
-    let end = Math.min(offset + value.length, valueOffsets[index + 1]);
-    while (offset < end) {
-        values.set(offset++, value.get(++idx));
+    const values = vector.getChildAt(0)!, valueOffsets = vector.valueOffsets;
+    for (let idx = -1, itr = valueOffsets[index], end = valueOffsets[index + 1]; itr < end;) {
+        values.set(itr++, value.get(++idx));
     }
 };
 
 /** @ignore */
-const setStruct = <
-    S extends { [key: string]: DataType },
-    V extends VectorType<Map_<S>> | VectorType<Struct<S>>
->(vector: V, index: number, value: V['TValue']) => {
-    vector.type.children.forEach((_field, idx) => {
-        const child = vector.getChildAt(idx);
-        child && child.set(index, value[idx]);
-    });
+const setMap = <T extends Map_>(vector: VectorType<T>, index: number, value: T['TValue']) => {
+    const values = vector.getChildAt(0)!, valueOffsets = vector.valueOffsets;
+    const entries = value instanceof Map ? [...value] : Object.entries(value);
+    for (let idx = -1, itr = valueOffsets[index], end = valueOffsets[index + 1]; itr < end;) {
+        values.set(itr++, entries[++idx]);
+    }
 };
 
+/** @ignore */ const _setStructArrayValue = (o: number, v: any[]) => (c: Vector | null, _: Field, i: number) => c && c.set(o, v[i]);
+/** @ignore */ const _setStructVectorValue = (o: number, v: Vector) => (c: Vector | null, _: Field, i: number) => c && c.set(o, v.get(i));
+/** @ignore */ const _setStructMapValue = (o: number, v: Map<string, any>) => (c: Vector | null, f: Field, _: number) => c && c.set(o, v.get(f.name));
+/** @ignore */ const _setStructObjectValue = (o: number, v: { [key: string]: any }) => (c: Vector | null, f: Field, _: number) => c && c.set(o, v[f.name]);
 /** @ignore */
-const setMap = <
-    S extends { [key: string]: DataType },
-    V extends VectorType<Map_<S>> | VectorType<Struct<S>>
->(vector: V, index: number, value: V['TValue']) => {
-    vector.type.children.forEach(({ name }, idx) => {
-        const child = vector.getChildAt(idx);
-        child && child.set(index, value[name]);
-    });
+const setStruct = <T extends Struct>(vector: VectorType<T>, index: number, value: T['TValue']) => {
+
+    const setValue = value instanceof Map    ? _setStructMapValue(index, value)    :
+                     value instanceof Vector ? _setStructVectorValue(index, value) :
+                     Array.isArray(value)    ? _setStructArrayValue(index, value)  :
+                                               _setStructObjectValue(index, value) ;
+
+    vector.type.children.forEach((f: Field, i: number) => setValue(vector.getChildAt(i), f, i));
 };
 
 /* istanbul ignore next */
diff --git a/js/src/visitor/vectorassembler.ts b/js/src/visitor/vectorassembler.ts
index 45af164..e6de584 100644
--- a/js/src/visitor/vectorassembler.ts
+++ b/js/src/visitor/vectorassembler.ts
@@ -198,9 +198,9 @@ function assembleFlatListVector<T extends Utf8 | Binary>(this: VectorAssembler,
 }
 
 /** @ignore */
-function assembleListVector<T extends List | FixedSizeList>(this: VectorAssembler, vector: V<T>) {
+function assembleListVector<T extends Map_ | List | FixedSizeList>(this: VectorAssembler, vector: V<T>) {
     const { length, valueOffsets } = vector;
-    // If we have valueOffsets (ListVector), push that buffer first
+    // If we have valueOffsets (MapVector, ListVector), push that buffer first
     if (valueOffsets) {
         addBuffer.call(this, rebaseValueOffsets(valueOffsets[0], length, valueOffsets));
     }
@@ -209,7 +209,7 @@ function assembleListVector<T extends List | FixedSizeList>(this: VectorAssemble
 }
 
 /** @ignore */
-function assembleNestedVector<T extends Struct | Map_ | Union>(this: VectorAssembler, vector: V<T>) {
+function assembleNestedVector<T extends Struct | Union>(this: VectorAssembler, vector: V<T>) {
     return this.visitMany(vector.type.children.map((_, i) => vector.getChildAt(i)!).filter(Boolean))[0];
 }
 
@@ -228,4 +228,4 @@ VectorAssembler.prototype.visitStruct          =   assembleNestedVector;
 VectorAssembler.prototype.visitUnion           =          assembleUnion;
 VectorAssembler.prototype.visitInterval        =     assembleFlatVector;
 VectorAssembler.prototype.visitFixedSizeList   =     assembleListVector;
-VectorAssembler.prototype.visitMap             =   assembleNestedVector;
+VectorAssembler.prototype.visitMap             =     assembleListVector;
diff --git a/js/src/visitor/vectorloader.ts b/js/src/visitor/vectorloader.ts
index 3d0ccd6..a508688 100644
--- a/js/src/visitor/vectorloader.ts
+++ b/js/src/visitor/vectorloader.ts
@@ -73,7 +73,7 @@ export class VectorLoader extends Visitor {
     public visitDictionary      <T extends type.Dictionary>      (type: T, { length, nullCount } = this.nextFieldNode()) { return      Data.Dictionary(type, 0, length, nullCount, this.readNullBitmap(type, nullCount), this.readData(type.indices), this.readDictionary(type));                        }
     public visitInterval        <T extends type.Interval>        (type: T, { length, nullCount } = this.nextFieldNode()) { return        Data.Interval(type, 0, length, nullCount, this.readNullBitmap(type, nullCount), this.readData(type));                                                           }
     public visitFixedSizeList   <T extends type.FixedSizeList>   (type: T, { length, nullCount } = this.nextFieldNode()) { return   Data.FixedSizeList(type, 0, length, nullCount, this.readNullBitmap(type, nullCount), this.visit(type.children[0]));                                                  }
-    public visitMap             <T extends type.Map_>            (type: T, { length, nullCount } = this.nextFieldNode()) { return             Data.Map(type, 0, length, nullCount, this.readNullBitmap(type, nullCount), this.visitMany(type.children));                                                 }
+    public visitMap             <T extends type.Map_>            (type: T, { length, nullCount } = this.nextFieldNode()) { return             Data.Map(type, 0, length, nullCount, this.readNullBitmap(type, nullCount), this.readOffsets(type), this.visit(type.children[0]));                          }
 
     protected nextFieldNode() { return this.nodes[++this.nodesIndex]; }
     protected nextBufferRange() { return this.buffers[++this.buffersIndex]; }
diff --git a/js/test/generate-test-data.ts b/js/test/generate-test-data.ts
index 28edb20..3822b53 100644
--- a/js/test/generate-test-data.ts
+++ b/js/test/generate-test-data.ts
@@ -66,7 +66,7 @@ interface TestDataVectorGenerator extends Visitor {
     visit<T extends Dictionary>      (type: T, length?: number, nullCount?: number, dictionary?: Vector): GeneratedVector<V<T>>;
     visit<T extends Union>           (type: T, length?: number, nullCount?: number, children?: Vector[]): GeneratedVector<V<T>>;
     visit<T extends Struct>          (type: T, length?: number, nullCount?: number, children?: Vector[]): GeneratedVector<V<T>>;
-    visit<T extends Map_>            (type: T, length?: number, nullCount?: number, children?: Vector[]): GeneratedVector<V<T>>;
+    visit<T extends Map_>            (type: T, length?: number, nullCount?: number, child?: Vector): GeneratedVector<V<T>>;
     visit<T extends DataType>        (type: T, length?: number, ...args: any[]): GeneratedVector<V<T>>;
 
     visitNull:            typeof generateNull;
@@ -126,10 +126,17 @@ const defaultStructChildren = () => [
     new Field('struct[2]', new List(new Field('list[DateDay]', new DateDay())))
 ];
 
+const defaultMapChild = () => [
+    new Field('', new Struct<{ key: Utf8, value: Float32 }>([
+        new Field('key', new Utf8()),
+        new Field('value', new Float32())
+    ]))
+][0];
+
 const defaultUnionChildren = () => [
     new Field('union[0]', new Float64()),
     new Field('union[1]', new Dictionary(new Uint32(), new Int32())),
-    new Field('union[2]', new Map_(defaultStructChildren()))
+    new Field('union[2]', new Map_(defaultMapChild()))
 ];
 
 export interface GeneratedTable {
@@ -221,7 +228,7 @@ export const dictionary = <T extends DataType = Utf8, TKey extends TKeys = Int32
 export const intervalDayTime = (length = 100, nullCount = length * 0.2 | 0) => vectorGenerator.visit(new IntervalDayTime(), length, nullCount);
 export const intervalYearMonth = (length = 100, nullCount = length * 0.2 | 0) => vectorGenerator.visit(new IntervalYearMonth(), length, nullCount);
 export const fixedSizeList = (length = 100, nullCount = length * 0.2 | 0, listSize = 2, child = defaultListChild) => vectorGenerator.visit(new FixedSizeList(listSize, child), length, nullCount);
-export const map = <T extends { [key: string]: DataType } = any>(length = 100, nullCount = length * 0.2 | 0, children: Field<T[keyof T]>[] = <any> defaultStructChildren()) => vectorGenerator.visit(new Map_<T>(children), length, nullCount);
+export const map = <TKey extends DataType = any, TValue extends DataType = any>(length = 100, nullCount = length * 0.2 | 0, 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, binary, fixedSizeBinary, dateDay, dateMillisecond, timestampSecond, timestampMillisecond, timestampMicrosecond, timestampNanosecond, timeSecond, timeMillisecond, timeMicrosecond, timeNanosecond, decimal, list, struct, denseUnion, sparseUnion, dictionary, intervalDayTime, intervalYearMonth, fixedSizeList, map
@@ -270,7 +277,6 @@ function generateFloat<T extends Float>(this: TestDataVectorGenerator, type: T,
     const values = memoize(() => {
         const values = [] as (number | null)[];
         iterateBitmap(length, nullBitmap, (i, valid) => {
-            // values[i] = !valid ? null : precision > 0 ? data[i] : (data[i] - 32767) / 32767;
             values[i] = !valid ? null : precision > 0 ? data[i] : util.uint16ToFloat64(data[i]);
         });
         return values;
@@ -281,17 +287,29 @@ function generateFloat<T extends Float>(this: TestDataVectorGenerator, type: T,
 
 function generateUtf8<T extends Utf8>(this: TestDataVectorGenerator, type: T, length = 100, nullCount = length * 0.2 | 0): GeneratedVector<V<T>> {
     const nullBitmap = createBitmap(length, nullCount);
-    const offsets = createVariableWidthOffsets(length, nullBitmap);
-    const values = [...offsets.slice(1)]
+    const offsets = createVariableWidthOffsets(length, nullBitmap, undefined, undefined, nullCount != 0);
+    const values: string[] = new Array(offsets.length - 1).fill(null);
+    [...offsets.slice(1)]
         .map((o, i) => isValid(nullBitmap, i) ? o - offsets[i] : null)
-        .map((length) => length == null ? null : randomString(length));
+        .reduce((map, length, i) => {
+            if (length !== null) {
+                if (length > 0) {
+                    do {
+                        values[i] = randomString(length);
+                    } while (map.has(values[i]));
+                    return map.set(values[i], i);
+                }
+                values[i] = '';
+            }
+            return map;
+        }, new Map<string, number>());
     const data = createVariableWidthBytes(length, nullBitmap, offsets, (i) => encodeUtf8(values[i]));
     return { values: () => values, vector: Vector.new(Data.Utf8(type, 0, length, nullCount, nullBitmap, offsets, data)) };
 }
 
 function generateBinary<T extends Binary>(this: TestDataVectorGenerator, type: T, length = 100, nullCount = length * 0.2 | 0): GeneratedVector<V<T>> {
     const nullBitmap = createBitmap(length, nullCount);
-    const offsets = createVariableWidthOffsets(length, nullBitmap);
+    const offsets = createVariableWidthOffsets(length, nullBitmap, undefined, undefined, nullCount != 0);
     const values = [...offsets.slice(1)]
         .map((o, i) => isValid(nullBitmap, i) ? o - offsets[i] : null)
         .map((length) => length == null ? null : randomBytes(length));
@@ -512,30 +530,43 @@ function generateStruct<T extends Struct>(this: TestDataVectorGenerator, type: T
     const values = memoize(() => {
         const values = [] as any[];
         const childValues = cols.map((x) => x());
+        const names = type.children.map((f) => f.name);
         iterateBitmap(length, nullBitmap, (i, valid) => {
-            values[i] = !valid ? null : childValues.map((col) => col[i]);
+            values[i] = !valid ? null : childValues.reduce((row, col, j) => ({
+                ...row, [names[j]]: col[i]
+            }), {});
         });
         return values;
     });
     return { values, vector: Vector.new(Data.Struct(type, 0, length, nullCount, nullBitmap, vecs)) };
 }
 
-function generateMap<T extends Map_>(this: TestDataVectorGenerator, type: T, length = 100, nullCount = length * 0.2 | 0, children = type.children.map((f) => this.visit(f.type, length, nullCount))): GeneratedVector<V<T>> {
-    const vecs = children.map(({ vector }) => vector);
-    const cols = children.map(({ values }) => values);
+function generateMap<T extends Map_>(this: TestDataVectorGenerator,
+                                     type: T, length = 100, nullCount = length * 0.2 | 0,
+                                     child = this.visit(type.children[0].type, length * 3, 0, [
+                                         this.visit(type.children[0].type.children[0].type, length * 3, 0),
+                                         this.visit(type.children[0].type.children[1].type, length * 3, nullCount * 3)
+                                     ])): GeneratedVector<V<T>> {
+
+    type K = T['keyType']['TValue'];
+    type V = T['valueType']['TValue'];
+
+    const childVec = child.vector;
     const nullBitmap = createBitmap(length, nullCount);
+    const stride = childVec.length / (length - nullCount);
+    const offsets = createVariableWidthOffsets(length, nullBitmap, childVec.length, stride);
     const values = memoize(() => {
-        const values = [] as any[];
-        const childValues = cols.map((x) => x());
-        const names = type.children.map((f) => f.name);
-        iterateBitmap(length, nullBitmap, (i, valid) => {
-            values[i] = !valid ? null : childValues.reduce((row, col, j) => ({
-                ...row, [names[j]]: col[i]
-            }), {});
-        });
+        const childValues = child.values() as { key: K; value: V; }[];
+        const values: (T['TValue'] | null)[] = [...offsets.slice(1)]
+            .map((offset, i) => isValid(nullBitmap, i) ? offset : null)
+            .map((o, i) => o == null ? null : (() => {
+                const slice = childValues.slice(offsets[i], o);
+                const pairs = slice.map(({ key, value }) => [key, value]);
+                return new Map<K, V>(pairs as any as (readonly [K, V])[]);
+            })());
         return values;
     });
-    return { values, vector: Vector.new(Data.Map(type, 0, length, nullCount, nullBitmap, vecs)) };
+    return { values, vector: Vector.new(Data.Map(type, 0, length, nullCount, nullBitmap, offsets, childVec)) };
 }
 
 type TypedArrayConstructor =
@@ -594,10 +625,16 @@ function createBitmap(length: number, nullCount: number) {
     return bytes;
 }
 
-function createVariableWidthOffsets(length: number, nullBitmap: Uint8Array, max = Infinity, stride = 20) {
+function createVariableWidthOffsets(length: number, nullBitmap: Uint8Array, max = Infinity, stride = 20, allowEmpty = true) {
     const offsets = new Int32Array(length + 1);
     iterateBitmap(length, nullBitmap, (i, valid) => {
-        offsets[i + 1] = valid ? Math.min(max, offsets[i] + (rand() * stride | 0)) : offsets[i];
+        if (!valid) {
+            offsets[i + 1] = offsets[i];
+        } else {
+            do {
+                offsets[i + 1] = Math.min(max, offsets[i] + (rand() * stride | 0));
+            } while (!allowEmpty && offsets[i + 1] === offsets[i]);
+        }
     });
     return offsets;
 }