You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by do...@apache.org on 2023/09/28 21:41:01 UTC

[arrow] branch main updated: GH-21815: [JS] Add support for Duration type (#37341)

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

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


The following commit(s) were added to refs/heads/main by this push:
     new befcc90def GH-21815: [JS] Add support for Duration type (#37341)
befcc90def is described below

commit befcc90defcc6b2fc35e8b3226b1ee38851e0cdc
Author: František Nečas <fr...@protonmail.com>
AuthorDate: Thu Sep 28 23:40:54 2023 +0200

    GH-21815: [JS] Add support for Duration type (#37341)
    
    ### Rationale for this change
    
    The `Duration` type is currently not supported and trying to deserialize a Table containing the type (e.g. using `tableFromIPC`) fails with `Unrecognized type` error. This PR aims to fix that.
    
    ### What changes are included in this PR?
    
    - definition of the `Duration` data type
    - updates to the visitor classes so that things like parsing work correctly
    - test coverage for the type
    - documentation update
    
    ### Are these changes tested?
    
    Yes, I extended the data generator with the new type so that the type is tested by the existing tests.
    
    ### Are there any user-facing changes?
    
    Yes, I've updated the documentation status page. I also noticed that it was outdated for JavaScript, i.e. there is already support for `Decimal` type so I updated this as well.
    
    Closes: https://github.com/apache/arrow/issues/21815
    Closes: https://github.com/apache/arrow/issues/35439
    * Closes: #21815
    
    Lead-authored-by: František Necas <fr...@protonmail.com>
    Co-authored-by: ptaylor <pa...@me.com>
    Signed-off-by: Dominik Moritz <do...@gmail.com>
---
 dev/archery/archery/integration/datagen.py |  3 +-
 docs/source/status.rst                     |  4 +--
 js/src/Arrow.dom.ts                        |  2 ++
 js/src/Arrow.ts                            |  2 ++
 js/src/builder.ts                          |  6 ++--
 js/src/builder/duration.ts                 | 46 ++++++++++++++++++++++++++++++
 js/src/data.ts                             | 11 +++++++
 js/src/enum.ts                             |  7 ++++-
 js/src/interfaces.ts                       | 16 +++++++++++
 js/src/ipc/metadata/json.ts                |  6 +++-
 js/src/ipc/metadata/message.ts             |  7 ++++-
 js/src/type.ts                             | 34 ++++++++++++++++++++++
 js/src/visitor.ts                          | 27 +++++++++++++++++-
 js/src/visitor/builderctor.ts              |  6 ++++
 js/src/visitor/bytelength.ts               |  5 +++-
 js/src/visitor/get.ts                      | 30 +++++++++++++++++++
 js/src/visitor/indexof.ts                  | 11 +++++++
 js/src/visitor/iterator.ts                 | 11 +++++++
 js/src/visitor/jsontypeassembler.ts        |  3 ++
 js/src/visitor/jsonvectorassembler.ts      |  6 +++-
 js/src/visitor/set.ts                      | 31 ++++++++++++++++++++
 js/src/visitor/typeassembler.ts            |  6 ++++
 js/src/visitor/typecomparator.ts           | 18 ++++++++++++
 js/src/visitor/typector.ts                 |  5 ++++
 js/src/visitor/vectorassembler.ts          |  6 ++--
 js/src/visitor/vectorloader.ts             |  5 +++-
 js/test/data/tables.ts                     |  3 +-
 js/test/generate-test-data.ts              | 20 ++++++++++++-
 js/test/unit/builders/builder-tests.ts     |  4 +++
 js/test/unit/generated-data-tests.ts       |  4 +++
 js/test/unit/visitor-tests.ts              | 11 +++++++
 31 files changed, 338 insertions(+), 18 deletions(-)

diff --git a/dev/archery/archery/integration/datagen.py b/dev/archery/archery/integration/datagen.py
index 299881c4b6..8d0cc6b0b0 100644
--- a/dev/archery/archery/integration/datagen.py
+++ b/dev/archery/archery/integration/datagen.py
@@ -1805,8 +1805,7 @@ def get_generated_json_files(tempdir=None):
         generate_datetime_case(),
 
         generate_duration_case()
-        .skip_tester('C#')
-        .skip_tester('JS'),  # TODO(ARROW-5239): Intervals + JS
+        .skip_tester('C#'),
 
         generate_interval_case()
         .skip_tester('C#')
diff --git a/docs/source/status.rst b/docs/source/status.rst
index 6314fd4c8d..e2b3852e22 100644
--- a/docs/source/status.rst
+++ b/docs/source/status.rst
@@ -46,7 +46,7 @@ Data Types
 +-------------------+-------+-------+-------+------------+-------+-------+-------+-------+
 | Decimal128        | ✓     | ✓     | ✓     | ✓          |  ✓    |  ✓    | ✓     |       |
 +-------------------+-------+-------+-------+------------+-------+-------+-------+-------+
-| Decimal256        | ✓     | ✓     | ✓     |            |  ✓    |  ✓    | ✓     |       |
+| Decimal256        | ✓     | ✓     | ✓     | ✓          |  ✓    |  ✓    | ✓     |       |
 +-------------------+-------+-------+-------+------------+-------+-------+-------+-------+
 | Date32/64         | ✓     | ✓     | ✓     | ✓          |  ✓    |  ✓    | ✓     | ✓     |
 +-------------------+-------+-------+-------+------------+-------+-------+-------+-------+
@@ -54,7 +54,7 @@ Data Types
 +-------------------+-------+-------+-------+------------+-------+-------+-------+-------+
 | Timestamp         | ✓     | ✓     | ✓     | ✓          |  ✓    |  ✓    | ✓     |       |
 +-------------------+-------+-------+-------+------------+-------+-------+-------+-------+
-| Duration          | ✓     | ✓     | ✓     |            |       |  ✓    | ✓     |       |
+| Duration          | ✓     | ✓     | ✓     | ✓          |       |  ✓    | ✓     |       |
 +-------------------+-------+-------+-------+------------+-------+-------+-------+-------+
 | Interval          | ✓     | ✓     | ✓     |            |       |  ✓    | ✓     |       |
 +-------------------+-------+-------+-------+------------+-------+-------+-------+-------+
diff --git a/js/src/Arrow.dom.ts b/js/src/Arrow.dom.ts
index 2fdef60c1f..451bf6acb6 100644
--- a/js/src/Arrow.dom.ts
+++ b/js/src/Arrow.dom.ts
@@ -59,6 +59,7 @@ export {
     Union, DenseUnion, SparseUnion,
     Dictionary,
     Interval, IntervalDayTime, IntervalYearMonth,
+    Duration, DurationSecond, DurationMillisecond, DurationMicrosecond, DurationNanosecond,
     FixedSizeList,
     Map_, MapRow,
     Table, makeTable, tableFromArrays,
@@ -86,6 +87,7 @@ export {
     FixedSizeListBuilder,
     FloatBuilder, Float16Builder, Float32Builder, Float64Builder,
     IntervalBuilder, IntervalDayTimeBuilder, IntervalYearMonthBuilder,
+    DurationBuilder, DurationSecondBuilder, DurationMillisecondBuilder, DurationMicrosecondBuilder, DurationNanosecondBuilder,
     IntBuilder, Int8Builder, Int16Builder, Int32Builder, Int64Builder, Uint8Builder, Uint16Builder, Uint32Builder, Uint64Builder,
     ListBuilder,
     MapBuilder,
diff --git a/js/src/Arrow.ts b/js/src/Arrow.ts
index 4a6394c266..714861e764 100644
--- a/js/src/Arrow.ts
+++ b/js/src/Arrow.ts
@@ -48,6 +48,7 @@ export {
     Union, DenseUnion, SparseUnion,
     Dictionary,
     Interval, IntervalDayTime, IntervalYearMonth,
+    Duration, DurationSecond, DurationMillisecond, DurationMicrosecond, DurationNanosecond,
     FixedSizeList,
     Map_
 } from './type.js';
@@ -75,6 +76,7 @@ export { IntBuilder, Int8Builder, Int16Builder, Int32Builder, Int64Builder, Uint
 export { TimeBuilder, TimeSecondBuilder, TimeMillisecondBuilder, TimeMicrosecondBuilder, TimeNanosecondBuilder } from './builder/time.js';
 export { TimestampBuilder, TimestampSecondBuilder, TimestampMillisecondBuilder, TimestampMicrosecondBuilder, TimestampNanosecondBuilder } from './builder/timestamp.js';
 export { IntervalBuilder, IntervalDayTimeBuilder, IntervalYearMonthBuilder } from './builder/interval.js';
+export { DurationBuilder, DurationSecondBuilder, DurationMillisecondBuilder, DurationMicrosecondBuilder, DurationNanosecondBuilder } from './builder/duration.js';
 export { Utf8Builder } from './builder/utf8.js';
 export { BinaryBuilder } from './builder/binary.js';
 export { ListBuilder } from './builder/list.js';
diff --git a/js/src/builder.ts b/js/src/builder.ts
index 90fe3ddcc9..93510eedf8 100644
--- a/js/src/builder.ts
+++ b/js/src/builder.ts
@@ -21,7 +21,7 @@ import { MapRow, kKeys } from './row/map.js';
 import {
     DataType, strideForType,
     Float, Int, Decimal, FixedSizeBinary,
-    Date_, Time, Timestamp, Interval,
+    Date_, Time, Timestamp, Interval, Duration,
     Utf8, Binary, List, Map_,
 } from './type.js';
 import { createIsValidFunction } from './builder/valid.js';
@@ -290,7 +290,7 @@ export abstract class Builder<T extends DataType = any, TNull = any> {
         } else if (valueOffsets = _offsets?.flush(length)) { // Variable-width primitives (Binary, Utf8), and Lists
             // Binary, Utf8
             data = _values?.flush(_offsets.last());
-        } else { // Fixed-width primitives (Int, Float, Decimal, Time, Timestamp, and Interval)
+        } else { // Fixed-width primitives (Int, Float, Decimal, Time, Timestamp, Duration and Interval)
             data = _values?.flush(length);
         }
 
@@ -342,7 +342,7 @@ export abstract class Builder<T extends DataType = any, TNull = any> {
 (Builder.prototype as any)._isValid = () => true;
 
 /** @ignore */
-export abstract class FixedWidthBuilder<T extends Int | Float | FixedSizeBinary | Date_ | Timestamp | Time | Decimal | Interval = any, TNull = any> extends Builder<T, TNull> {
+export abstract class FixedWidthBuilder<T extends Int | Float | FixedSizeBinary | Date_ | Timestamp | Time | Decimal | Interval | Duration = any, TNull = any> extends Builder<T, TNull> {
     constructor(opts: BuilderOptions<T, TNull>) {
         super(opts);
         this._values = new DataBufferBuilder(new this.ArrayType(0), this.stride);
diff --git a/js/src/builder/duration.ts b/js/src/builder/duration.ts
new file mode 100644
index 0000000000..968899ea55
--- /dev/null
+++ b/js/src/builder/duration.ts
@@ -0,0 +1,46 @@
+
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+import { FixedWidthBuilder } from '../builder.js';
+import { Duration, DurationSecond, DurationMillisecond, DurationMicrosecond, DurationNanosecond } from '../type.js';
+import { setDuration, setDurationSecond, setDurationMillisecond, setDurationMicrosecond, setDurationNanosecond } from '../visitor/set.js';
+
+/** @ignore */
+export class DurationBuilder<T extends Duration = Duration, TNull = any> extends FixedWidthBuilder<T, TNull> { }
+
+(DurationBuilder.prototype as any)._setValue = setDuration;
+
+/** @ignore */
+export class DurationSecondBuilder<TNull = any> extends DurationBuilder<DurationSecond, TNull> { }
+
+(DurationSecondBuilder.prototype as any)._setValue = setDurationSecond;
+
+/** @ignore */
+export class DurationMillisecondBuilder<TNull = any> extends DurationBuilder<DurationMillisecond, TNull> { }
+
+(DurationMillisecondBuilder.prototype as any)._setValue = setDurationMillisecond;
+
+/** @ignore */
+export class DurationMicrosecondBuilder<TNull = any> extends DurationBuilder<DurationMicrosecond, TNull> { }
+
+(DurationMicrosecondBuilder.prototype as any)._setValue = setDurationMicrosecond;
+
+/** @ignore */
+export class DurationNanosecondBuilder<TNull = any> extends DurationBuilder<DurationNanosecond, TNull> { }
+
+(DurationNanosecondBuilder.prototype as any)._setValue = setDurationNanosecond;
diff --git a/js/src/data.ts b/js/src/data.ts
index dc423cdb01..1e9df71cff 100644
--- a/js/src/data.ts
+++ b/js/src/data.ts
@@ -257,6 +257,7 @@ import {
     Int,
     Date_,
     Interval,
+    Duration,
     Time,
     Timestamp,
     Union, DenseUnion, SparseUnion,
@@ -390,6 +391,13 @@ class MakeDataVisitor extends Visitor {
         const { ['length']: length = data.length / strideForType(type), ['nullCount']: nullCount = props['nullBitmap'] ? -1 : 0, } = props;
         return new Data(type, offset, length, nullCount, [undefined, data, nullBitmap]);
     }
+    public visitDuration<T extends Duration>(props: DurationDataProps<T>) {
+        const { ['type']: type, ['offset']: offset = 0 } = props;
+        const nullBitmap = toUint8Array(props['nullBitmap']);
+        const data = toArrayBufferView(type.ArrayType, props['data']);
+        const { ['length']: length = data.length, ['nullCount']: nullCount = props['nullBitmap'] ? -1 : 0, } = props;
+        return new Data(type, offset, length, nullCount, [undefined, data, nullBitmap]);
+    }
     public visitFixedSizeList<T extends FixedSizeList>(props: FixedSizeListDataProps<T>) {
         const { ['type']: type, ['offset']: offset = 0, ['child']: child = new MakeDataVisitor().visit({ type: type.valueType }) } = props;
         const nullBitmap = toUint8Array(props['nullBitmap']);
@@ -424,6 +432,7 @@ interface Date_DataProps<T extends Date_> extends DataProps_<T> { data?: DataBuf
 interface TimeDataProps<T extends Time> extends DataProps_<T> { data?: DataBuffer<T> }
 interface TimestampDataProps<T extends Timestamp> extends DataProps_<T> { data?: DataBuffer<T> }
 interface IntervalDataProps<T extends Interval> extends DataProps_<T> { data?: DataBuffer<T> }
+interface DurationDataProps<T extends Duration> extends DataProps_<T> { data?: DataBuffer<T> }
 interface FixedSizeBinaryDataProps<T extends FixedSizeBinary> extends DataProps_<T> { data?: DataBuffer<T> }
 interface BinaryDataProps<T extends Binary> extends DataProps_<T> { valueOffsets: ValueOffsetsBuffer; data?: DataBuffer<T> }
 interface Utf8DataProps<T extends Utf8> extends DataProps_<T> { valueOffsets: ValueOffsetsBuffer; data?: DataBuffer<T> }
@@ -446,6 +455,7 @@ export type DataProps<T extends DataType> = (
     T extends Time /*            */ ? TimeDataProps<T> :
     T extends Timestamp /*       */ ? TimestampDataProps<T> :
     T extends Interval /*        */ ? IntervalDataProps<T> :
+    T extends Duration /*        */ ? DurationDataProps<T> :
     T extends FixedSizeBinary /* */ ? FixedSizeBinaryDataProps<T> :
     T extends Binary /*          */ ? BinaryDataProps<T> :
     T extends Utf8 /*            */ ? Utf8DataProps<T> :
@@ -471,6 +481,7 @@ export function makeData<T extends Date_>(props: Date_DataProps<T>): Data<T>;
 export function makeData<T extends Time>(props: TimeDataProps<T>): Data<T>;
 export function makeData<T extends Timestamp>(props: TimestampDataProps<T>): Data<T>;
 export function makeData<T extends Interval>(props: IntervalDataProps<T>): Data<T>;
+export function makeData<T extends Duration>(props: DurationDataProps<T>): Data<T>;
 export function makeData<T extends FixedSizeBinary>(props: FixedSizeBinaryDataProps<T>): Data<T>;
 export function makeData<T extends Binary>(props: BinaryDataProps<T>): Data<T>;
 export function makeData<T extends Utf8>(props: Utf8DataProps<T>): Data<T>;
diff --git a/js/src/enum.ts b/js/src/enum.ts
index f5856bc06a..4e207dd37c 100644
--- a/js/src/enum.ts
+++ b/js/src/enum.ts
@@ -137,7 +137,7 @@ export enum MessageHeader {
  * nested type consisting of other data types, or another data type (e.g. a
  * timestamp encoded as an int64).
  *
- * **Note**: Only enum values 0-17 (NONE through Map) are written to an Arrow
+ * **Note**: Only enum values 0-18 (NONE through Duration) are written to an Arrow
  * IPC payload.
  *
  * The rest of the values are specified here so TypeScript can narrow the type
@@ -174,6 +174,7 @@ export enum Type {
     FixedSizeBinary = 15, /** Fixed-size binary. Each value occupies the same number of bytes */
     FixedSizeList = 16, /** Fixed-size list. Each value occupies the same number of bytes */
     Map = 17, /** Map of named logical types */
+    Duration = 18, /** Measure of elapsed time in either seconds, miliseconds, microseconds or nanoseconds. */
 
     Dictionary = -1, /** Dictionary aka Category type */
     Int8 = -2,
@@ -201,6 +202,10 @@ export enum Type {
     SparseUnion = -24,
     IntervalDayTime = -25,
     IntervalYearMonth = -26,
+    DurationSecond = -27,
+    DurationMillisecond = -28,
+    DurationMicrosecond = -29,
+    DurationNanosecond = -30
 }
 
 export enum BufferType {
diff --git a/js/src/interfaces.ts b/js/src/interfaces.ts
index 8d61295919..95c5adbb2a 100644
--- a/js/src/interfaces.ts
+++ b/js/src/interfaces.ts
@@ -31,6 +31,7 @@ import type { IntBuilder, Int8Builder, Int16Builder, Int32Builder, Int64Builder,
 import type { TimeBuilder, TimeSecondBuilder, TimeMillisecondBuilder, TimeMicrosecondBuilder, TimeNanosecondBuilder } from './builder/time.js';
 import type { TimestampBuilder, TimestampSecondBuilder, TimestampMillisecondBuilder, TimestampMicrosecondBuilder, TimestampNanosecondBuilder } from './builder/timestamp.js';
 import type { IntervalBuilder, IntervalDayTimeBuilder, IntervalYearMonthBuilder } from './builder/interval.js';
+import type { DurationBuilder, DurationSecondBuilder, DurationMillisecondBuilder, DurationMicrosecondBuilder, DurationNanosecondBuilder } from './builder/duration.js';
 import type { Utf8Builder } from './builder/utf8.js';
 import type { BinaryBuilder } from './builder/binary.js';
 import type { ListBuilder } from './builder/list.js';
@@ -222,6 +223,11 @@ export type TypeToDataType<T extends Type> = {
     [Type.Interval]: type.Interval;
     [Type.IntervalDayTime]: type.IntervalDayTime;
     [Type.IntervalYearMonth]: type.IntervalYearMonth;
+    [Type.Duration]: type.Duration;
+    [Type.DurationSecond]: type.DurationSecond;
+    [Type.DurationMillisecond]: type.DurationMillisecond;
+    [Type.DurationMicrosecond]: type.DurationMicrosecond;
+    [Type.DurationNanosecond]: type.DurationNanosecond;
     [Type.Map]: type.Map_;
     [Type.List]: type.List;
     [Type.Struct]: type.Struct;
@@ -270,6 +276,11 @@ type TypeToBuilder<T extends Type = any, TNull = any> = {
     [Type.Interval]: IntervalBuilder<any, TNull>;
     [Type.IntervalDayTime]: IntervalDayTimeBuilder<TNull>;
     [Type.IntervalYearMonth]: IntervalYearMonthBuilder<TNull>;
+    [Type.Duration]: DurationBuilder<any, TNull>;
+    [Type.DurationSecond]: DurationBuilder<any, TNull>;
+    [Type.DurationMillisecond]: DurationMillisecondBuilder<TNull>;
+    [Type.DurationMicrosecond]: DurationMicrosecondBuilder<TNull>;
+    [Type.DurationNanosecond]: DurationNanosecondBuilder<TNull>;
     [Type.Map]: MapBuilder<any, any, TNull>;
     [Type.List]: ListBuilder<any, TNull>;
     [Type.Struct]: StructBuilder<any, TNull>;
@@ -318,6 +329,11 @@ type DataTypeToBuilder<T extends DataType = any, TNull = any> = {
     [Type.Interval]: T extends type.Interval ? IntervalBuilder<T, TNull> : never;
     [Type.IntervalDayTime]: T extends type.IntervalDayTime ? IntervalDayTimeBuilder<TNull> : never;
     [Type.IntervalYearMonth]: T extends type.IntervalYearMonth ? IntervalYearMonthBuilder<TNull> : never;
+    [Type.Duration]: T extends type.Duration ? DurationBuilder<T, TNull>: never;
+    [Type.DurationSecond]: T extends type.DurationSecond ? DurationSecondBuilder<TNull> : never;
+    [Type.DurationMillisecond]: T extends type.DurationMillisecond ? DurationMillisecondBuilder<TNull> : never;
+    [Type.DurationMicrosecond]: T extends type.DurationMicrosecond ? DurationMicrosecondBuilder<TNull>: never;
+    [Type.DurationNanosecond]: T extends type.DurationNanosecond ? DurationNanosecondBuilder<TNull>: never;
     [Type.Map]: T extends type.Map_ ? MapBuilder<T['keyType'], T['valueType'], TNull> : never;
     [Type.List]: T extends type.List ? ListBuilder<T['valueType'], TNull> : never;
     [Type.Struct]: T extends type.Struct ? StructBuilder<T['dataTypes'], TNull> : never;
diff --git a/js/src/ipc/metadata/json.ts b/js/src/ipc/metadata/json.ts
index e5995110f0..f1f306730d 100644
--- a/js/src/ipc/metadata/json.ts
+++ b/js/src/ipc/metadata/json.ts
@@ -22,7 +22,7 @@ import {
     DataType, Dictionary, TimeBitWidth,
     Utf8, Binary, Decimal, FixedSizeBinary,
     List, FixedSizeList, Map_, Struct, Union,
-    Bool, Null, Int, Float, Date_, Time, Interval, Timestamp, IntBitWidth, Int32, TKeys,
+    Bool, Null, Int, Float, Date_, Time, Interval, Timestamp, IntBitWidth, Int32, TKeys, Duration,
 } from '../../type.js';
 
 import { DictionaryBatch, RecordBatch, FieldNode, BufferRegion } from './message.js';
@@ -185,6 +185,10 @@ function typeFromJSON(f: any, children?: Field[]): DataType<any> {
             const t = f['type'];
             return new Interval(IntervalUnit[t['unit']] as any);
         }
+        case 'duration': {
+            const t = f['type'];
+            return new Duration(TimeUnit[t['unit']] as any);
+        }
         case 'union': {
             const t = f['type'];
             const [m, ...ms] = (t['mode'] + '').toLowerCase();
diff --git a/js/src/ipc/metadata/message.ts b/js/src/ipc/metadata/message.ts
index 6465d3d064..27c9b92d68 100644
--- a/js/src/ipc/metadata/message.ts
+++ b/js/src/ipc/metadata/message.ts
@@ -36,6 +36,7 @@ import { Date as _Date } from '../../fb/date.js';
 import { Time as _Time } from '../../fb/time.js';
 import { Timestamp as _Timestamp } from '../../fb/timestamp.js';
 import { Interval as _Interval } from '../../fb/interval.js';
+import { Duration as _Duration } from '../../fb/duration.js';
 import { Union as _Union } from '../../fb/union.js';
 import { FixedSizeBinary as _FixedSizeBinary } from '../../fb/fixed-size-binary.js';
 import { FixedSizeList as _FixedSizeList } from '../../fb/fixed-size-list.js';
@@ -57,7 +58,7 @@ import {
     DataType, Dictionary, TimeBitWidth,
     Utf8, Binary, Decimal, FixedSizeBinary,
     List, FixedSizeList, Map_, Struct, Union,
-    Bool, Null, Int, Float, Date_, Time, Interval, Timestamp, IntBitWidth, Int32, TKeys,
+    Bool, Null, Int, Float, Date_, Time, Interval, Timestamp, IntBitWidth, Int32, TKeys, Duration,
 } from '../../type.js';
 
 /**
@@ -466,6 +467,10 @@ function decodeFieldType(f: _Field, children?: Field[]): DataType<any> {
             const t = f.type(new _Interval())!;
             return new Interval(t.unit());
         }
+        case Type['Duration']: {
+            const t = f.type(new _Duration())!;
+            return new Duration(t.unit());
+        }
         case Type['Union']: {
             const t = f.type(new _Union())!;
             return new Union(t.mode(), t.typeIdsArray() || [], children || []);
diff --git a/js/src/type.ts b/js/src/type.ts
index 1dc90c47cb..34bbf45bca 100644
--- a/js/src/type.ts
+++ b/js/src/type.ts
@@ -63,6 +63,7 @@ export abstract class DataType<TType extends Type = Type, TChildren extends Type
     /** @nocollapse */ static isTime(x: any): x is Time_ { return x?.typeId === Type.Time; }
     /** @nocollapse */ static isTimestamp(x: any): x is Timestamp_ { return x?.typeId === Type.Timestamp; }
     /** @nocollapse */ static isInterval(x: any): x is Interval_ { return x?.typeId === Type.Interval; }
+    /** @nocollapse */ static isDuration(x: any): x is Duration { return x?.typeId === Type.Duration; }
     /** @nocollapse */ static isList(x: any): x is List { return x?.typeId === Type.List; }
     /** @nocollapse */ static isStruct(x: any): x is Struct { return x?.typeId === Type.Struct; }
     /** @nocollapse */ static isUnion(x: any): x is Union_ { return x?.typeId === Type.Union; }
@@ -433,6 +434,39 @@ export class IntervalDayTime extends Interval_<Type.IntervalDayTime> { construct
 /** @ignore */
 export class IntervalYearMonth extends Interval_<Type.IntervalYearMonth> { constructor() { super(IntervalUnit.YEAR_MONTH); } }
 
+/** @ignore */
+type Durations = Type.Duration | Type.DurationSecond | Type.DurationMillisecond | Type.DurationMicrosecond | Type.DurationNanosecond;
+/** @ignore */
+export interface Duration<T extends Durations = Durations> extends DataType<T> {
+    TArray: BigInt64Array;
+    TValue: bigint;
+    ArrayType: BigInt64Array;
+}
+
+/** @ignore */
+export class Duration<T extends Durations = Durations> extends DataType<T> {
+    constructor(public readonly unit: TimeUnit) {
+        super();
+    }
+    public get typeId() { return Type.Duration as T; }
+    public toString() { return `Duration<${TimeUnit[this.unit]}>`; }
+    protected static [Symbol.toStringTag] = ((proto: Duration) => {
+        (<any>proto).unit = null;
+        (<any>proto).ArrayType = BigInt64Array;
+        return proto[Symbol.toStringTag] = 'Duration';
+    })(Duration.prototype);
+}
+
+/** @ignore */
+export class DurationSecond extends Duration<Type.DurationSecond> { constructor() { super(TimeUnit.SECOND); }}
+/** @ignore */
+export class DurationMillisecond extends Duration<Type.DurationMillisecond> { constructor() { super(TimeUnit.MILLISECOND); }}
+/** @ignore */
+export class DurationMicrosecond extends Duration<Type.DurationMicrosecond> { constructor() { super(TimeUnit.MICROSECOND); }}
+/** @ignore */
+export class DurationNanosecond extends Duration<Type.DurationNanosecond> { constructor() { super(TimeUnit.NANOSECOND); }}
+
+
 /** @ignore */
 export interface List<T extends DataType = any> extends DataType<Type.List, { [0]: T }> {
     TArray: Array<T>;
diff --git a/js/src/visitor.ts b/js/src/visitor.ts
index 3be50a6d3e..c63640b038 100644
--- a/js/src/visitor.ts
+++ b/js/src/visitor.ts
@@ -16,7 +16,7 @@
 // under the License.
 
 import { Type, Precision, DateUnit, TimeUnit, IntervalUnit, UnionMode } from './enum.js';
-import { DataType, Float, Int, Date_, Interval, Time, Timestamp, Union, } from './type.js';
+import { DataType, Float, Int, Date_, Interval, Time, Timestamp, Union, Duration } from './type.js';
 
 export abstract class Visitor {
     public visitMany(nodes: any[], ...args: any[][]) {
@@ -47,6 +47,7 @@ export abstract class Visitor {
     public visitUnion(_node: any, ..._args: any[]): any { return null; }
     public visitDictionary(_node: any, ..._args: any[]): any { return null; }
     public visitInterval(_node: any, ..._args: any[]): any { return null; }
+    public visitDuration(_node: any, ... _args: any[]): any { return null; }
     public visitFixedSizeList(_node: any, ..._args: any[]): any { return null; }
     public visitMap(_node: any, ..._args: any[]): any { return null; }
 }
@@ -113,6 +114,11 @@ function getVisitFnByTypeId(visitor: Visitor, dtype: Type, throwIfNotFound = tru
         case Type.Interval: fn = visitor.visitInterval; break;
         case Type.IntervalDayTime: fn = visitor.visitIntervalDayTime || visitor.visitInterval; break;
         case Type.IntervalYearMonth: fn = visitor.visitIntervalYearMonth || visitor.visitInterval; break;
+        case Type.Duration: fn = visitor.visitDuration; break;
+        case Type.DurationSecond: fn = visitor.visitDurationSecond || visitor.visitDuration; break;
+        case Type.DurationMillisecond: fn = visitor.visitDurationMillisecond || visitor.visitDuration; break;
+        case Type.DurationMicrosecond: fn = visitor.visitDurationMicrosecond || visitor.visitDuration; break;
+        case Type.DurationNanosecond: fn = visitor.visitDurationNanosecond || visitor.visitDuration; break;
         case Type.FixedSizeList: fn = visitor.visitFixedSizeList; break;
         case Type.Map: fn = visitor.visitMap; break;
     }
@@ -180,6 +186,15 @@ function inferDType<T extends DataType>(type: T): Type {
             }
             // @ts-ignore
             return Type.Interval;
+        case Type.Duration:
+            switch ((type as any as Duration).unit) {
+                case TimeUnit.SECOND: return Type.DurationSecond;
+                case TimeUnit.MILLISECOND: return Type.DurationMillisecond;
+                case TimeUnit.MICROSECOND: return Type.DurationMicrosecond;
+                case TimeUnit.NANOSECOND: return Type.DurationNanosecond;
+            }
+            // @ts-ignore
+            return Type.Duration;
         case Type.Map: return Type.Map;
         case Type.List: return Type.List;
         case Type.Struct: return Type.Struct;
@@ -239,6 +254,11 @@ export interface Visitor {
     visitInterval(node: any, ...args: any[]): any;
     visitIntervalDayTime?(node: any, ...args: any[]): any;
     visitIntervalYearMonth?(node: any, ...args: any[]): any;
+    visitDuration(node: any, ...args: any[]): any;
+    visitDurationSecond(node: any, ...args: any[]): any;
+    visitDurationMillisecond(node: any, ...args: any[]): any;
+    visitDurationMicrosecond(node: any, ...args: any[]): any;
+    visitDurationNanosecond(node: any, ...args: any[]): any;
     visitFixedSizeList(node: any, ...args: any[]): any;
     visitMap(node: any, ...args: any[]): any;
 }
@@ -270,3 +290,8 @@ export interface Visitor {
 (Visitor.prototype as any).visitSparseUnion = null;
 (Visitor.prototype as any).visitIntervalDayTime = null;
 (Visitor.prototype as any).visitIntervalYearMonth = null;
+(Visitor.prototype as any).visitDuration = null;
+(Visitor.prototype as any).visitDurationSecond = null;
+(Visitor.prototype as any).visitDurationMillisecond = null;
+(Visitor.prototype as any).visitDurationMicrosecond = null;
+(Visitor.prototype as any).visitDurationNanosecond = null;
diff --git a/js/src/visitor/builderctor.ts b/js/src/visitor/builderctor.ts
index 9ce9ae4d4a..2d20f2a8ef 100644
--- a/js/src/visitor/builderctor.ts
+++ b/js/src/visitor/builderctor.ts
@@ -30,6 +30,7 @@ import { FixedSizeBinaryBuilder } from '../builder/fixedsizebinary.js';
 import { FixedSizeListBuilder } from '../builder/fixedsizelist.js';
 import { FloatBuilder, Float16Builder, Float32Builder, Float64Builder } from '../builder/float.js';
 import { IntervalBuilder, IntervalDayTimeBuilder, IntervalYearMonthBuilder } from '../builder/interval.js';
+import { DurationBuilder, DurationSecondBuilder, DurationMillisecondBuilder, DurationMicrosecondBuilder, DurationNanosecondBuilder } from '../builder/duration.js';
 import { IntBuilder, Int8Builder, Int16Builder, Int32Builder, Int64Builder, Uint8Builder, Uint16Builder, Uint32Builder, Uint64Builder } from '../builder/int.js';
 import { ListBuilder } from '../builder/list.js';
 import { MapBuilder } from '../builder/map.js';
@@ -91,6 +92,11 @@ export class GetBuilderCtor extends Visitor {
     public visitInterval() { return IntervalBuilder; }
     public visitIntervalDayTime() { return IntervalDayTimeBuilder; }
     public visitIntervalYearMonth() { return IntervalYearMonthBuilder; }
+    public visitDuration() { return DurationBuilder; }
+    public visitDurationSecond() { return DurationSecondBuilder; }
+    public visitDurationMillisecond() { return DurationMillisecondBuilder; }
+    public visitDurationMicrosecond() { return DurationMicrosecondBuilder; }
+    public visistDurationNanosecond() { return DurationNanosecondBuilder; }
     public visitFixedSizeList() { return FixedSizeListBuilder; }
     public visitMap() { return MapBuilder; }
 }
diff --git a/js/src/visitor/bytelength.ts b/js/src/visitor/bytelength.ts
index 862808ad54..72d6148a52 100644
--- a/js/src/visitor/bytelength.ts
+++ b/js/src/visitor/bytelength.ts
@@ -25,7 +25,7 @@ import { TypeToDataType } from '../interfaces.js';
 import { Type, TimeUnit, UnionMode } from '../enum.js';
 import {
     DataType, Dictionary,
-    Float, Int, Date_, Interval, Time, Timestamp,
+    Float, Int, Date_, Interval, Time, Timestamp, Duration,
     Bool, Null, Utf8, Binary, Decimal, FixedSizeBinary,
     List, FixedSizeList, Map_, Struct, Union, DenseUnion, SparseUnion,
 } from '../type.js';
@@ -75,6 +75,9 @@ export class GetByteLengthVisitor extends Visitor {
     public visitInterval(data: Data<Interval>, _: number) {
         return (data.type.unit + 1) * 4;
     }
+    public visitDuration(____: Data<Duration>, _: number) {
+        return 8;
+    }
     public visitStruct(data: Data<Struct>, i: number) {
         return data.children.reduce((total, child) => total + instance.visit(child, i), 0);
     }
diff --git a/js/src/visitor/get.ts b/js/src/visitor/get.ts
index 12f8325470..5aaaedf51a 100644
--- a/js/src/visitor/get.ts
+++ b/js/src/visitor/get.ts
@@ -34,6 +34,7 @@ import {
     Interval, IntervalDayTime, IntervalYearMonth,
     Time, TimeSecond, TimeMillisecond, TimeMicrosecond, TimeNanosecond,
     Timestamp, TimestampSecond, TimestampMillisecond, TimestampMicrosecond, TimestampNanosecond,
+    Duration, DurationSecond, DurationMillisecond, DurationMicrosecond, DurationNanosecond,
     Union, DenseUnion, SparseUnion,
 } from '../type.js';
 
@@ -84,6 +85,11 @@ export interface GetVisitor extends Visitor {
     visitInterval<T extends Interval>(data: Data<T>, index: number): T['TValue'] | null;
     visitIntervalDayTime<T extends IntervalDayTime>(data: Data<T>, index: number): T['TValue'] | null;
     visitIntervalYearMonth<T extends IntervalYearMonth>(data: Data<T>, index: number): T['TValue'] | null;
+    visitDuration<T extends Duration>(data: Data<T>, index: number): T['TValue'] | null;
+    visitDurationSecond<T extends DurationSecond>(data: Data<T>, index: number): T['TValue'] | null;
+    visitDurationMillisecond<T extends DurationMillisecond>(data: Data<T>, index: number): T['TValue'] | null;
+    visitDurationMicrosecond<T extends DurationMicrosecond>(data: Data<T>, index: number): T['TValue'] | null;
+    visitDurationNanosecond<T extends DurationNanosecond>(data: Data<T>, index: number): T['TValue'] | null;
     visitFixedSizeList<T extends FixedSizeList>(data: Data<T>, index: number): T['TValue'] | null;
     visitMap<T extends Map_>(data: Data<T>, index: number): T['TValue'] | null;
 }
@@ -279,6 +285,25 @@ const getIntervalYearMonth = <T extends IntervalYearMonth>({ values }: Data<T>,
     return int32s;
 };
 
+/** @ignore */
+const getDurationSecond = <T extends DurationSecond>({ values }: Data<T>, index: number): T['TValue'] => values[index];
+/** @ignore */
+const getDurationMillisecond = <T extends DurationMillisecond>({ values }: Data<T>, index: number): T['TValue'] => values[index];
+/** @ignore */
+const getDurationMicrosecond = <T extends DurationMicrosecond>({ values }: Data<T>, index: number): T['TValue'] => values[index];
+/** @ignore */
+const getDurationNanosecond = <T extends DurationNanosecond>({ values }: Data<T>, index: number): T['TValue'] => values[index];
+/* istanbul ignore next */
+/** @ignore */
+const getDuration = <T extends Duration>(data: Data<T>, index: number): T['TValue'] => {
+    switch (data.type.unit) {
+        case TimeUnit.SECOND: return getDurationSecond(data as Data<DurationSecond>, index);
+        case TimeUnit.MILLISECOND: return getDurationMillisecond(data as Data<DurationMillisecond>, index);
+        case TimeUnit.MICROSECOND: return getDurationMicrosecond(data as Data<DurationMicrosecond>, index);
+        case TimeUnit.NANOSECOND: return getDurationNanosecond(data as Data<DurationNanosecond>, index);
+    }
+};
+
 /** @ignore */
 const getFixedSizeList = <T extends FixedSizeList>(data: Data<T>, index: number): T['TValue'] => {
     const { stride, children } = data;
@@ -328,6 +353,11 @@ GetVisitor.prototype.visitDictionary = wrapGet(getDictionary);
 GetVisitor.prototype.visitInterval = wrapGet(getInterval);
 GetVisitor.prototype.visitIntervalDayTime = wrapGet(getIntervalDayTime);
 GetVisitor.prototype.visitIntervalYearMonth = wrapGet(getIntervalYearMonth);
+GetVisitor.prototype.visitDuration = wrapGet(getDuration);
+GetVisitor.prototype.visitDurationSecond = wrapGet(getDurationSecond);
+GetVisitor.prototype.visitDurationMillisecond = wrapGet(getDurationMillisecond);
+GetVisitor.prototype.visitDurationMicrosecond = wrapGet(getDurationMicrosecond);
+GetVisitor.prototype.visitDurationNanosecond = wrapGet(getDurationNanosecond);
 GetVisitor.prototype.visitFixedSizeList = wrapGet(getFixedSizeList);
 GetVisitor.prototype.visitMap = wrapGet(getMap);
 
diff --git a/js/src/visitor/indexof.ts b/js/src/visitor/indexof.ts
index 654134c6df..28dcff20d3 100644
--- a/js/src/visitor/indexof.ts
+++ b/js/src/visitor/indexof.ts
@@ -31,6 +31,7 @@ import {
     Interval, IntervalDayTime, IntervalYearMonth,
     Time, TimeSecond, TimeMillisecond, TimeMicrosecond, TimeNanosecond,
     Timestamp, TimestampSecond, TimestampMillisecond, TimestampMicrosecond, TimestampNanosecond,
+    Duration, DurationSecond, DurationMillisecond, DurationMicrosecond, DurationNanosecond,
     Union, DenseUnion, SparseUnion,
 } from '../type.js';
 
@@ -81,6 +82,11 @@ export interface IndexOfVisitor extends Visitor {
     visitInterval<T extends Interval>(data: Data<T>, value: T['TValue'] | null, index?: number): number;
     visitIntervalDayTime<T extends IntervalDayTime>(data: Data<T>, value: T['TValue'] | null, index?: number): number;
     visitIntervalYearMonth<T extends IntervalYearMonth>(data: Data<T>, value: T['TValue'] | null, index?: number): number;
+    visitDuration<T extends Duration>(data: Data<T>, value: T['TValue'] | null, index?: number): number;
+    visitDurationSecond<T extends DurationSecond>(data: Data<T>, value: T['TValue'] | null, index?: number): number;
+    visitDurationMillisecond<T extends DurationMillisecond>(data: Data<T>, value: T['TValue'] | null, index?: number): number;
+    visitDurationMicrosecond<T extends DurationMicrosecond>(data: Data<T>, value: T['TValue'] | null, index?: number): number;
+    visitDurationNanosecond<T extends DurationNanosecond>(data: Data<T>, value: T['TValue'] | null, index?: number): number;
     visitFixedSizeList<T extends FixedSizeList>(data: Data<T>, value: T['TValue'] | null, index?: number): number;
     visitMap<T extends Map_>(data: Data<T>, value: T['TValue'] | null, index?: number): number;
 }
@@ -191,6 +197,11 @@ IndexOfVisitor.prototype.visitDictionary = indexOfValue;
 IndexOfVisitor.prototype.visitInterval = indexOfValue;
 IndexOfVisitor.prototype.visitIntervalDayTime = indexOfValue;
 IndexOfVisitor.prototype.visitIntervalYearMonth = indexOfValue;
+IndexOfVisitor.prototype.visitDuration = indexOfValue;
+IndexOfVisitor.prototype.visitDurationSecond = indexOfValue;
+IndexOfVisitor.prototype.visitDurationMillisecond = indexOfValue;
+IndexOfVisitor.prototype.visitDurationMicrosecond = indexOfValue;
+IndexOfVisitor.prototype.visitDurationNanosecond = indexOfValue;
 IndexOfVisitor.prototype.visitFixedSizeList = indexOfValue;
 IndexOfVisitor.prototype.visitMap = indexOfValue;
 
diff --git a/js/src/visitor/iterator.ts b/js/src/visitor/iterator.ts
index 48021a78e8..e38bb90769 100644
--- a/js/src/visitor/iterator.ts
+++ b/js/src/visitor/iterator.ts
@@ -28,6 +28,7 @@ import {
     Interval, IntervalDayTime, IntervalYearMonth,
     Time, TimeSecond, TimeMillisecond, TimeMicrosecond, TimeNanosecond,
     Timestamp, TimestampSecond, TimestampMillisecond, TimestampMicrosecond, TimestampNanosecond,
+    Duration, DurationSecond, DurationMillisecond, DurationMicrosecond, DurationNanosecond,
     Union, DenseUnion, SparseUnion,
 } from '../type.js';
 import { ChunkedIterator } from '../util/chunk.js';
@@ -79,6 +80,11 @@ export interface IteratorVisitor extends Visitor {
     visitInterval<T extends Interval>(vector: Vector<T>): IterableIterator<T['TValue'] | null>;
     visitIntervalDayTime<T extends IntervalDayTime>(vector: Vector<T>): IterableIterator<T['TValue'] | null>;
     visitIntervalYearMonth<T extends IntervalYearMonth>(vector: Vector<T>): IterableIterator<T['TValue'] | null>;
+    visitDuration<T extends Duration>(vector: Vector<T>): IterableIterator<T['TValue'] | null>;
+    visitDurationSecond<T extends DurationSecond>(vector: Vector<T>): IterableIterator<T['TValue'] | null>;
+    visitDurationMillisecond<T extends DurationMillisecond>(vector: Vector<T>): IterableIterator<T['TValue'] | null>;
+    visitDurationMicrosecond<T extends DurationMicrosecond>(vector: Vector<T>): IterableIterator<T['TValue'] | null>;
+    visitDurationNanosecond<T extends DurationNanosecond>(vector: Vector<T>): IterableIterator<T['TValue'] | null>;
     visitFixedSizeList<T extends FixedSizeList>(vector: Vector<T>): IterableIterator<T['TValue'] | null>;
     visitMap<T extends Map_>(vector: Vector<T>): IterableIterator<T['TValue'] | null>;
 }
@@ -177,6 +183,11 @@ IteratorVisitor.prototype.visitDictionary = vectorIterator;
 IteratorVisitor.prototype.visitInterval = vectorIterator;
 IteratorVisitor.prototype.visitIntervalDayTime = vectorIterator;
 IteratorVisitor.prototype.visitIntervalYearMonth = vectorIterator;
+IteratorVisitor.prototype.visitDuration = vectorIterator;
+IteratorVisitor.prototype.visitDurationSecond = vectorIterator;
+IteratorVisitor.prototype.visitDurationMillisecond = vectorIterator;
+IteratorVisitor.prototype.visitDurationMicrosecond = vectorIterator;
+IteratorVisitor.prototype.visitDurationNanosecond = vectorIterator;
 IteratorVisitor.prototype.visitFixedSizeList = vectorIterator;
 IteratorVisitor.prototype.visitMap = vectorIterator;
 
diff --git a/js/src/visitor/jsontypeassembler.ts b/js/src/visitor/jsontypeassembler.ts
index d83edfc24f..6e6cfb0741 100644
--- a/js/src/visitor/jsontypeassembler.ts
+++ b/js/src/visitor/jsontypeassembler.ts
@@ -63,6 +63,9 @@ export class JSONTypeAssembler extends Visitor {
     public visitInterval<T extends type.Interval>({ typeId, unit }: T) {
         return { 'name': ArrowType[typeId].toLowerCase(), 'unit': IntervalUnit[unit] };
     }
+    public visitDuration<T extends type.Duration>({ typeId, unit }: T) {
+        return { 'name': ArrowType[typeId].toLocaleLowerCase(), 'unit': TimeUnit[unit]};
+    }
     public visitList<T extends type.List>({ typeId }: T) {
         return { 'name': ArrowType[typeId].toLowerCase() };
     }
diff --git a/js/src/visitor/jsonvectorassembler.ts b/js/src/visitor/jsonvectorassembler.ts
index 7a617f4afe..55a6b4e2ea 100644
--- a/js/src/visitor/jsonvectorassembler.ts
+++ b/js/src/visitor/jsonvectorassembler.ts
@@ -26,7 +26,7 @@ import { UnionMode, DateUnit, TimeUnit } from '../enum.js';
 import { BitIterator, getBit, getBool } from '../util/bit.js';
 import {
     DataType,
-    Float, Int, Date_, Interval, Time, Timestamp, Union,
+    Float, Int, Date_, Interval, Time, Timestamp, Union, Duration,
     Bool, Null, Utf8, Binary, Decimal, FixedSizeBinary, List, FixedSizeList, Map_, Struct, IntArray,
 } from '../type.js';
 
@@ -52,6 +52,7 @@ export interface JSONVectorAssembler extends Visitor {
     visitStruct<T extends Struct>(data: Data<T>): { children: any[] };
     visitUnion<T extends Union>(data: Data<T>): { children: any[]; TYPE_ID: number[] };
     visitInterval<T extends Interval>(data: Data<T>): { DATA: number[] };
+    visitDuration<T extends Duration>(data: Data<T>): { DATA: string[] };
     visitFixedSizeList<T extends FixedSizeList>(data: Data<T>): { children: any[] };
     visitMap<T extends Map_>(data: Data<T>): { children: any[] };
 }
@@ -146,6 +147,9 @@ export class JSONVectorAssembler extends Visitor {
     public visitInterval<T extends Interval>(data: Data<T>) {
         return { 'DATA': [...data.values] };
     }
+    public visitDuration<T extends Duration>(data: Data<T>) {
+        return { 'DATA': [...bigNumsToStrings(data.values, 2)]};
+    }
     public visitFixedSizeList<T extends FixedSizeList>(data: Data<T>) {
         return {
             'children': this.visitMany(data.type.children, data.children)
diff --git a/js/src/visitor/set.ts b/js/src/visitor/set.ts
index c2d4319911..1a0eddc556 100644
--- a/js/src/visitor/set.ts
+++ b/js/src/visitor/set.ts
@@ -32,6 +32,7 @@ import {
     Interval, IntervalDayTime, IntervalYearMonth,
     Time, TimeSecond, TimeMillisecond, TimeMicrosecond, TimeNanosecond,
     Timestamp, TimestampSecond, TimestampMillisecond, TimestampMicrosecond, TimestampNanosecond,
+    Duration, DurationSecond, DurationMillisecond, DurationMicrosecond, DurationNanosecond,
     Union, DenseUnion, SparseUnion,
 } from '../type.js';
 
@@ -82,6 +83,11 @@ export interface SetVisitor extends Visitor {
     visitInterval<T extends Interval>(data: Data<T>, index: number, value: T['TValue']): void;
     visitIntervalDayTime<T extends IntervalDayTime>(data: Data<T>, index: number, value: T['TValue']): void;
     visitIntervalYearMonth<T extends IntervalYearMonth>(data: Data<T>, index: number, value: T['TValue']): void;
+    visitDuration<T extends Duration>(data: Data<T>, index: number, value: T['TValue']): void;
+    visitDurationSecond<T extends DurationSecond>(data: Data<T>, index: number, value: T['TValue']): void;
+    visitDurationMillisecond<T extends DurationMillisecond>(data: Data<T>, index: number, value: T['TValue']): void;
+    visitDurationMicrosecond<T extends DurationMicrosecond>(data: Data<T>, index: number, value: T['TValue']): void;
+    visitDurationNanosecond<T extends DurationNanosecond>(data: Data<T>, index: number, value: T['TValue']): void;
     visitFixedSizeList<T extends FixedSizeList>(data: Data<T>, index: number, value: T['TValue']): void;
     visitMap<T extends Map_>(data: Data<T>, index: number, value: T['TValue']): void;
 }
@@ -308,6 +314,26 @@ export const setIntervalDayTime = <T extends IntervalDayTime>({ values }: Data<T
 /** @ignore */
 export const setIntervalYearMonth = <T extends IntervalYearMonth>({ values }: Data<T>, index: number, value: T['TValue']): void => { values[index] = (value[0] * 12) + (value[1] % 12); };
 
+/** @ignore */
+export const setDurationSecond = <T extends DurationSecond>({ values }: Data<T>, index: number, value: T['TValue']): void => { values[index] = value; };
+/** @ignore */
+export const setDurationMillisecond = <T extends DurationMillisecond>({ values }: Data<T>, index: number, value: T['TValue']): void => { values[index] = value; };
+/** @ignore */
+export const setDurationMicrosecond = <T extends DurationMicrosecond>({ values }: Data<T>, index: number, value: T['TValue']): void => { values[index] = value; };
+/** @ignore */
+export const setDurationNanosecond = <T extends DurationNanosecond>({ values }: Data<T>, index: number, value: T['TValue']): void => { values[index] = value; };
+/* istanbul ignore next */
+/** @ignore */
+export const setDuration = <T extends Duration>(data: Data<T>, index: number, value: T['TValue']): void => {
+    switch (data.type.unit) {
+        case TimeUnit.SECOND: return setDurationSecond(data as Data<DurationSecond>, index, value as DurationSecond['TValue']);
+        case TimeUnit.MILLISECOND: return setDurationMillisecond(data as Data<DurationMillisecond>, index, value as DurationMillisecond['TValue']);
+        case TimeUnit.MICROSECOND: return setDurationMicrosecond(data as Data<DurationMicrosecond>, index, value as DurationMicrosecond['TValue']);
+        case TimeUnit.NANOSECOND: return setDurationNanosecond(data as Data<DurationNanosecond>, index, value as DurationNanosecond['TValue']);
+    }
+};
+
+
 /** @ignore */
 const setFixedSizeList = <T extends FixedSizeList>(data: Data<T>, index: number, value: T['TValue']): void => {
     const { stride } = data;
@@ -364,6 +390,11 @@ SetVisitor.prototype.visitDictionary = wrapSet(setDictionary);
 SetVisitor.prototype.visitInterval = wrapSet(setIntervalValue);
 SetVisitor.prototype.visitIntervalDayTime = wrapSet(setIntervalDayTime);
 SetVisitor.prototype.visitIntervalYearMonth = wrapSet(setIntervalYearMonth);
+SetVisitor.prototype.visitDuration = wrapSet(setDuration);
+SetVisitor.prototype.visitDurationSecond = wrapSet(setDurationSecond);
+SetVisitor.prototype.visitDurationMillisecond = wrapSet(setDurationMillisecond);
+SetVisitor.prototype.visitDurationMicrosecond = wrapSet(setDurationMicrosecond);
+SetVisitor.prototype.visitDurationNanosecond = wrapSet(setDurationNanosecond);
 SetVisitor.prototype.visitFixedSizeList = wrapSet(setFixedSizeList);
 SetVisitor.prototype.visitMap = wrapSet(setMap);
 
diff --git a/js/src/visitor/typeassembler.ts b/js/src/visitor/typeassembler.ts
index c84e3930f6..c2262d2053 100644
--- a/js/src/visitor/typeassembler.ts
+++ b/js/src/visitor/typeassembler.ts
@@ -32,6 +32,7 @@ import { Date } from '../fb/date.js';
 import { Time } from '../fb/time.js';
 import { Timestamp } from '../fb/timestamp.js';
 import { Interval } from '../fb/interval.js';
+import { Duration } from '../fb/duration.js';
 import { List } from '../fb/list.js';
 import { Struct_ as Struct } from '../fb/struct-.js';
 import { Union } from '../fb/union.js';
@@ -109,6 +110,11 @@ export class TypeAssembler extends Visitor {
         Interval.addUnit(b, node.unit);
         return Interval.endInterval(b);
     }
+    public visitDuration<T extends type.Duration>(node: T, b: Builder) {
+        Duration.startDuration(b);
+        Duration.addUnit(b, node.unit);
+        return Duration.endDuration(b);
+    }
     public visitList<T extends type.List>(_node: T, b: Builder) {
         List.startList(b);
         return List.endList(b);
diff --git a/js/src/visitor/typecomparator.ts b/js/src/visitor/typecomparator.ts
index a77c402096..1de8e218da 100644
--- a/js/src/visitor/typecomparator.ts
+++ b/js/src/visitor/typecomparator.ts
@@ -28,6 +28,7 @@ import {
     Interval, IntervalDayTime, IntervalYearMonth,
     Time, TimeSecond, TimeMillisecond, TimeMicrosecond, TimeNanosecond,
     Timestamp, TimestampSecond, TimestampMillisecond, TimestampMicrosecond, TimestampNanosecond,
+    Duration, DurationSecond, DurationMillisecond, DurationMicrosecond, DurationNanosecond,
     Union, DenseUnion, SparseUnion,
 } from '../type.js';
 
@@ -77,6 +78,11 @@ export interface TypeComparator extends Visitor {
     visitInterval<T extends Interval>(type: T, other?: DataType | null): other is T;
     visitIntervalDayTime<T extends IntervalDayTime>(type: T, other?: DataType | null): other is T;
     visitIntervalYearMonth<T extends IntervalYearMonth>(type: T, other?: DataType | null): other is T;
+    visitDuration<T extends Duration>(type: T, other?: DataType | null): other is T;
+    visitDurationSecond<T extends DurationSecond>(type: T, other?: DataType | null): other is T;
+    visitDurationMillisecond<T extends DurationMillisecond>(type: T, other?: DataType | null): other is T;
+    visitDurationMicrosecond<T extends DurationMicrosecond>(type: T, other?: DataType | null): other is T;
+    visitDurationNanosecond<T extends DurationNanosecond>(type: T, other?: DataType | null): other is T;
     visitFixedSizeList<T extends FixedSizeList>(type: T, other?: DataType | null): other is T;
     visitMap<T extends Map_>(type: T, other?: DataType | null): other is T;
 }
@@ -202,6 +208,13 @@ function compareInterval<T extends Interval>(type: T, other?: DataType | null):
     );
 }
 
+function compareDuration<T extends Duration>(type: T, other?: DataType | null): other is T {
+    return (type === other) || (
+        compareConstructor(type, other) &&
+        type.unit === other.unit
+    );
+}
+
 function compareFixedSizeList<T extends FixedSizeList>(type: T, other?: DataType | null): other is T {
     return (type === other) || (
         compareConstructor(type, other) &&
@@ -261,6 +274,11 @@ TypeComparator.prototype.visitDictionary = compareDictionary;
 TypeComparator.prototype.visitInterval = compareInterval;
 TypeComparator.prototype.visitIntervalDayTime = compareInterval;
 TypeComparator.prototype.visitIntervalYearMonth = compareInterval;
+TypeComparator.prototype.visitDuration = compareDuration;
+TypeComparator.prototype.visitDurationSecond = compareDuration;
+TypeComparator.prototype.visitDurationMillisecond = compareDuration;
+TypeComparator.prototype.visitDurationMicrosecond = compareDuration;
+TypeComparator.prototype.visitDurationNanosecond = compareDuration;
 TypeComparator.prototype.visitFixedSizeList = compareFixedSizeList;
 TypeComparator.prototype.visitMap = compareMap;
 
diff --git a/js/src/visitor/typector.ts b/js/src/visitor/typector.ts
index c825a61dba..077f66592f 100644
--- a/js/src/visitor/typector.ts
+++ b/js/src/visitor/typector.ts
@@ -74,6 +74,11 @@ export class GetDataTypeConstructor extends Visitor {
     public visitInterval() { return type.Interval; }
     public visitIntervalDayTime() { return type.IntervalDayTime; }
     public visitIntervalYearMonth() { return type.IntervalYearMonth; }
+    public visitDuration() { return type.Duration; }
+    public visitDurationSecond() { return type.DurationSecond; }
+    public visitDurationMillisecond() { return type.DurationMillisecond; }
+    public visitDurationMicrosecond() { return type.DurationMicrosecond; }
+    public visitDurationNanosecond() { return type.DurationNanosecond; }
     public visitFixedSizeList() { return type.FixedSizeList; }
     public visitMap() { return type.Map_; }
 }
diff --git a/js/src/visitor/vectorassembler.ts b/js/src/visitor/vectorassembler.ts
index dbf778c4c3..949463272e 100644
--- a/js/src/visitor/vectorassembler.ts
+++ b/js/src/visitor/vectorassembler.ts
@@ -26,7 +26,7 @@ import { packBools, truncateBitmap } from '../util/bit.js';
 import { BufferRegion, FieldNode } from '../ipc/metadata/message.js';
 import {
     DataType, Dictionary,
-    Float, Int, Date_, Interval, Time, Timestamp, Union,
+    Float, Int, Date_, Interval, Time, Timestamp, Union, Duration,
     Bool, Null, Utf8, Binary, Decimal, FixedSizeBinary, List, FixedSizeList, Map_, Struct,
 } from '../type.js';
 
@@ -51,6 +51,7 @@ export interface VectorAssembler extends Visitor {
     visitStruct<T extends Struct>(data: Data<T>): this;
     visitUnion<T extends Union>(data: Data<T>): this;
     visitInterval<T extends Interval>(data: Data<T>): this;
+    visitDuration<T extends Duration>(data: Data<T>): this;
     visitFixedSizeList<T extends FixedSizeList>(data: Data<T>): this;
     visitMap<T extends Map_>(data: Data<T>): this;
 }
@@ -195,7 +196,7 @@ function assembleBoolVector<T extends Bool>(this: VectorAssembler, data: Data<T>
 }
 
 /** @ignore */
-function assembleFlatVector<T extends Int | Float | FixedSizeBinary | Date_ | Timestamp | Time | Decimal | Interval>(this: VectorAssembler, data: Data<T>) {
+function assembleFlatVector<T extends Int | Float | FixedSizeBinary | Date_ | Timestamp | Time | Decimal | Interval | Duration>(this: VectorAssembler, data: Data<T>) {
     return addBuffer.call(this, data.values.subarray(0, data.length * data.stride));
 }
 
@@ -243,5 +244,6 @@ VectorAssembler.prototype.visitList = assembleListVector;
 VectorAssembler.prototype.visitStruct = assembleNestedVector;
 VectorAssembler.prototype.visitUnion = assembleUnion;
 VectorAssembler.prototype.visitInterval = assembleFlatVector;
+VectorAssembler.prototype.visitDuration = assembleFlatVector;
 VectorAssembler.prototype.visitFixedSizeList = assembleListVector;
 VectorAssembler.prototype.visitMap = assembleListVector;
diff --git a/js/src/visitor/vectorloader.ts b/js/src/visitor/vectorloader.ts
index cb4bc28292..db34edad9a 100644
--- a/js/src/visitor/vectorloader.ts
+++ b/js/src/visitor/vectorloader.ts
@@ -115,6 +115,9 @@ export class VectorLoader extends Visitor {
     public visitInterval<T extends type.Interval>(type: T, { length, nullCount } = this.nextFieldNode()) {
         return makeData({ type, length, nullCount, nullBitmap: this.readNullBitmap(type, nullCount), data: this.readData(type) });
     }
+    public visitDuration<T extends type.Duration>(type: T, { length, nullCount } = this.nextFieldNode()) {
+        return makeData({ type, length, nullCount, nullBitmap: this.readNullBitmap(type, nullCount), data: this.readData(type) });
+    }
     public visitFixedSizeList<T extends type.FixedSizeList>(type: T, { length, nullCount } = this.nextFieldNode()) {
         return makeData({ type, length, nullCount, nullBitmap: this.readNullBitmap(type, nullCount), 'child': this.visit(type.children[0]) });
     }
@@ -157,7 +160,7 @@ export class JSONVectorLoader extends VectorLoader {
         const { sources } = this;
         if (DataType.isTimestamp(type)) {
             return toArrayBufferView(Uint8Array, Int64.convertArray(sources[offset] as string[]));
-        } else if ((DataType.isInt(type) || DataType.isTime(type)) && type.bitWidth === 64) {
+        } else if ((DataType.isInt(type) || DataType.isTime(type)) && type.bitWidth === 64 || DataType.isDuration(type)) {
             return toArrayBufferView(Uint8Array, Int64.convertArray(sources[offset] as string[]));
         } else if (DataType.isDate(type) && type.unit === DateUnit.MILLISECOND) {
             return toArrayBufferView(Uint8Array, Int64.convertArray(sources[offset] as string[]));
diff --git a/js/test/data/tables.ts b/js/test/data/tables.ts
index e4d859e0a6..28aed7e4fe 100644
--- a/js/test/data/tables.ts
+++ b/js/test/data/tables.ts
@@ -30,7 +30,8 @@ const valueVectorGeneratorNames = [
     'float16', 'float32', 'float64', 'utf8', 'binary', 'fixedSizeBinary', 'dateDay', 'dateMillisecond',
     'timestampSecond', 'timestampMillisecond', 'timestampMicrosecond', 'timestampNanosecond',
     'timeSecond', 'timeMillisecond', 'timeMicrosecond', 'timeNanosecond', 'decimal',
-    'dictionary', 'intervalDayTime', 'intervalYearMonth'
+    'dictionary', 'intervalDayTime', 'intervalYearMonth',
+    'durationSecond', 'durationMillisecond', 'durationMicrosecond', 'durationNanosecond',
 ];
 
 const vectorGeneratorNames = [...valueVectorGeneratorNames, ...listVectorGeneratorNames, ...nestedVectorGeneratorNames];
diff --git a/js/test/generate-test-data.ts b/js/test/generate-test-data.ts
index a03b22c54c..15fb715a31 100644
--- a/js/test/generate-test-data.ts
+++ b/js/test/generate-test-data.ts
@@ -36,6 +36,7 @@ import {
     Union, DenseUnion, SparseUnion,
     Dictionary,
     Interval, IntervalDayTime, IntervalYearMonth,
+    Duration, DurationSecond, DurationMillisecond, DurationMicrosecond, DurationNanosecond,
     FixedSizeList,
     Map_,
     DateUnit, TimeUnit, UnionMode,
@@ -58,6 +59,7 @@ interface TestDataVectorGenerator extends Visitor {
     visit<T extends Time>(type: T, length?: number, nullCount?: number): GeneratedVector<T>;
     visit<T extends Decimal>(type: T, length?: number, nullCount?: number): GeneratedVector<T>;
     visit<T extends Interval>(type: T, length?: number, nullCount?: number): GeneratedVector<T>;
+    visit<T extends Duration>(type: T, length?: number, nullCount?: number): GeneratedVector<T>;
     visit<T extends List>(type: T, length?: number, nullCount?: number, child?: Vector): GeneratedVector<T>;
     visit<T extends FixedSizeList>(type: T, length?: number, nullCount?: number, child?: Vector): GeneratedVector<T>;
     visit<T extends Dictionary>(type: T, length?: number, nullCount?: number, dictionary?: Vector): GeneratedVector<T>;
@@ -84,6 +86,7 @@ interface TestDataVectorGenerator extends Visitor {
     visitUnion: typeof generateUnion;
     visitDictionary: typeof generateDictionary;
     visitInterval: typeof generateInterval;
+    visitDuration: typeof generateDuration;
     visitFixedSizeList: typeof generateFixedSizeList;
     visitMap: typeof generateMap;
 }
@@ -108,6 +111,7 @@ TestDataVectorGenerator.prototype.visitStruct = generateStruct;
 TestDataVectorGenerator.prototype.visitUnion = generateUnion;
 TestDataVectorGenerator.prototype.visitDictionary = generateDictionary;
 TestDataVectorGenerator.prototype.visitInterval = generateInterval;
+TestDataVectorGenerator.prototype.visitDuration = generateDuration;
 TestDataVectorGenerator.prototype.visitFixedSizeList = generateFixedSizeList;
 TestDataVectorGenerator.prototype.visitMap = generateMap;
 
@@ -230,11 +234,15 @@ export const sparseUnion = (length = 100, nullCount = Math.trunc(length * 0.2),
 export const dictionary = <T extends DataType = Utf8, TKey extends TKeys = Int32>(length = 100, nullCount = Math.trunc(length * 0.2), dict: T = <any>new Utf8(), keys: TKey = <any>new Int32()) => vectorGenerator.visit(new Dictionary(dict, keys), length, nullCount);
 export const intervalDayTime = (length = 100, nullCount = Math.trunc(length * 0.2)) => vectorGenerator.visit(new IntervalDayTime(), length, nullCount);
 export const intervalYearMonth = (length = 100, nullCount = Math.trunc(length * 0.2)) => vectorGenerator.visit(new IntervalYearMonth(), length, nullCount);
+export const durationSecond = (length = 100, nullCount = Math.trunc(length * 0.2)) => vectorGenerator.visit(new DurationSecond(), length, nullCount);
+export const durationMillisecond = (length = 100, nullCount = Math.trunc(length * 0.2)) => vectorGenerator.visit(new DurationMillisecond(), length, nullCount);
+export const durationMicrosecond = (length = 100, nullCount = Math.trunc(length * 0.2)) => vectorGenerator.visit(new DurationMicrosecond(), length, nullCount);
+export const durationNanosecond = (length = 100, nullCount = Math.trunc(length * 0.2)) => vectorGenerator.visit(new DurationNanosecond(), length, nullCount);
 export const fixedSizeList = (length = 100, nullCount = Math.trunc(length * 0.2), listSize = 2, child = defaultListChild) => vectorGenerator.visit(new FixedSizeList(listSize, child), length, nullCount);
 export const map = <TKey extends DataType = any, TValue extends DataType = any>(length = 100, nullCount = Math.trunc(length * 0.2), child: Field<Struct<{ key: TKey; value: TValue }>> = <any>defaultMapChild()) => vectorGenerator.visit(new Map_<TKey, TValue>(child), length, nullCount);
 
 export const vecs = {
-    null_, bool, int8, int16, int32, int64, uint8, uint16, uint32, uint64, float16, float32, float64, utf8, binary, fixedSizeBinary, dateDay, dateMillisecond, timestampSecond, timestampMillisecond, timestampMicrosecond, timestampNanosecond, timeSecond, timeMillisecond, timeMicrosecond, timeNanosecond, decimal, list, struct, denseUnion, sparseUnion, dictionary, intervalDayTime, intervalYearMonth, fixedSizeList, map
+    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, durationSecond, durationMillisecond, durationMicrosecond, durationNanosecond
 } as { [k: string]: (...args: any[]) => any };
 
 function generateNull<T extends Null>(this: TestDataVectorGenerator, type: T, length = 100): GeneratedVector<T> {
@@ -421,6 +429,16 @@ function generateInterval<T extends Interval>(this: TestDataVectorGenerator, typ
     return { values, vector: new Vector([makeData({ type, length, nullCount, nullBitmap, data })]) };
 }
 
+function generateDuration<T extends Duration>(this: TestDataVectorGenerator, type: T, length = 100, nullCount = Math.trunc(length * 0.2)): GeneratedVector<T> {
+    const nullBitmap = createBitmap(length, nullCount);
+    const multiple = type.unit === TimeUnit.NANOSECOND ? 1000000000 :
+        type.unit === TimeUnit.MICROSECOND ? 1000000 :
+            type.unit === TimeUnit.MILLISECOND ? 1000 : 1;
+    const values: bigint[] = [];
+    const data = createTime64(length, nullBitmap, multiple, values);
+    return { values: () => values, vector: new Vector([makeData({ type, length, nullCount, nullBitmap, data })]) };
+}
+
 function generateList<T extends List>(this: TestDataVectorGenerator, type: T, length = 100, nullCount = Math.trunc(length * 0.2), child = this.visit(type.children[0].type, length * 3, nullCount * 3)): GeneratedVector<T> {
     const childVec = child.vector;
     const nullBitmap = createBitmap(length, nullCount);
diff --git a/js/test/unit/builders/builder-tests.ts b/js/test/unit/builders/builder-tests.ts
index a73183a7a5..b261e4f815 100644
--- a/js/test/unit/builders/builder-tests.ts
+++ b/js/test/unit/builders/builder-tests.ts
@@ -64,6 +64,10 @@ describe('Generated Test Data', () => {
     describe('DictionaryBuilder', () => { validateBuilder(generate.dictionary); });
     describe('IntervalDayTimeBuilder', () => { validateBuilder(generate.intervalDayTime); });
     describe('IntervalYearMonthBuilder', () => { validateBuilder(generate.intervalYearMonth); });
+    describe('DurationSecondBuilder', () => { validateBuilder(generate.durationSecond); });
+    describe('DurationMillisecondBuilder', () => { validateBuilder(generate.durationMillisecond); });
+    describe('DurationMicrosecondBuilder', () => { validateBuilder(generate.durationMicrosecond); });
+    describe('DurationNanosecondBuilder', () => { validateBuilder(generate.durationNanosecond); });
     describe('FixedSizeListBuilder', () => { validateBuilder(generate.fixedSizeList); });
     describe('MapBuilder', () => { validateBuilder(generate.map); });
 });
diff --git a/js/test/unit/generated-data-tests.ts b/js/test/unit/generated-data-tests.ts
index 90cf0d598a..d64c7c188d 100644
--- a/js/test/unit/generated-data-tests.ts
+++ b/js/test/unit/generated-data-tests.ts
@@ -58,6 +58,10 @@ describe('Generated Test Data', () => {
     describe('Dictionary', () => { validateVector(generate.dictionary()); });
     describe('IntervalDayTime', () => { validateVector(generate.intervalDayTime()); });
     describe('IntervalYearMonth', () => { validateVector(generate.intervalYearMonth()); });
+    describe('DurationSecond', () => { validateVector(generate.durationSecond()); });
+    describe('DurationMillisecond', () => { validateVector(generate.durationMillisecond()); });
+    describe('DurationMicrosecond', () => { validateVector(generate.durationMicrosecond()); });
+    describe('DurationNanosecond', () => { validateVector(generate.durationNanosecond()); });
     describe('FixedSizeList', () => { validateVector(generate.fixedSizeList()); });
     describe('Map', () => { validateVector(generate.map()); });
 });
diff --git a/js/test/unit/visitor-tests.ts b/js/test/unit/visitor-tests.ts
index 645fcc60f8..8a7ba1ed77 100644
--- a/js/test/unit/visitor-tests.ts
+++ b/js/test/unit/visitor-tests.ts
@@ -25,6 +25,7 @@ import {
     Interval, IntervalDayTime, IntervalYearMonth,
     Time, TimeSecond, TimeMillisecond, TimeMicrosecond, TimeNanosecond,
     Timestamp, TimestampSecond, TimestampMillisecond, TimestampMicrosecond, TimestampNanosecond,
+    Duration, DurationSecond, DurationMillisecond, DurationMicrosecond, DurationNanosecond,
     Union, DenseUnion, SparseUnion,
 } from 'apache-arrow';
 
@@ -46,6 +47,7 @@ class BasicVisitor extends Visitor {
     public visitUnion<T extends Union>(type: T) { return (this.type = type); }
     public visitDictionary<T extends Dictionary>(type: T) { return (this.type = type); }
     public visitInterval<T extends Interval>(type: T) { return (this.type = type); }
+    public visitDuration<T extends Duration>(type: T) { return (this.type = type); }
     public visitFixedSizeList<T extends FixedSizeList>(type: T) { return (this.type = type); }
     public visitMap<T extends Map_>(type: T) { return (this.type = type); }
 }
@@ -86,6 +88,10 @@ class FeatureVisitor extends Visitor {
     public visitDictionary<T extends Dictionary>(type: T) { return (this.type = type); }
     public visitIntervalDayTime<T extends IntervalDayTime>(type: T) { return (this.type = type); }
     public visitIntervalYearMonth<T extends IntervalYearMonth>(type: T) { return (this.type = type); }
+    public visitDurationSecond<T extends DurationSecond>(type: T) { return (this.type = type); }
+    public visitDurationMillisecond<T extends DurationMillisecond>(type: T) { return (this.type = type); }
+    public visitDurationMicrosecond<T extends DurationMicrosecond>(type: T) { return (this.type = type); }
+    public visitDurationNanosecond<T extends DurationNanosecond>(type: T) { return (this.type = type); }
     public visitFixedSizeList<T extends FixedSizeList>(type: T) { return (this.type = type); }
     public visitMap<T extends Map_>(type: T) { return (this.type = type); }
 }
@@ -109,6 +115,7 @@ describe('Visitor', () => {
         test(`visits Union types`, () => validateBasicVisitor(new Union(0, [] as any[], [] as any[])));
         test(`visits Dictionary types`, () => validateBasicVisitor(new Dictionary(null as any, null as any)));
         test(`visits Interval types`, () => validateBasicVisitor(new Interval(0)));
+        test(`visits Duration types`, () => validateBasicVisitor(new Duration(0)));
         test(`visits FixedSizeList types`, () => validateBasicVisitor(new FixedSizeList(2, null as any)));
         test(`visits Map types`, () => validateBasicVisitor(new Map_(new Field('', new Struct<{ key: Utf8; value: Int }>([
             new Field('key', new Utf8()), new Field('value', new Int8())
@@ -158,6 +165,10 @@ describe('Visitor', () => {
         test(`visits IntervalDayTime types`, () => validateFeatureVisitor(new IntervalDayTime()));
         test(`visits IntervalYearMonth types`, () => validateFeatureVisitor(new IntervalYearMonth()));
         test(`visits FixedSizeList types`, () => validateFeatureVisitor(new FixedSizeList(2, null as any)));
+        test(`visits DurationSecond types`, () => validateFeatureVisitor(new DurationSecond()));
+        test(`visits DurationMillisecond types`, () => validateFeatureVisitor(new DurationMillisecond()));
+        test(`visits DurationMicrosecond types`, () => validateFeatureVisitor(new DurationMicrosecond()));
+        test(`visits DurationNanosecond types`, () => validateFeatureVisitor(new DurationNanosecond()));
         test(`visits Map types`, () => validateFeatureVisitor(new Map_(new Field('', new Struct<{ key: Utf8; value: Int }>([
             new Field('key', new Utf8()), new Field('value', new Int8())
         ] as any[])))));