diff --git a/javascript/.gitignore b/javascript/.gitignore
new file mode 100644
index 00000000000..3b97e3ab957
--- /dev/null
+++ b/javascript/.gitignore
@@ -0,0 +1,4 @@
+lib/*_generated.js
+dist
+node_modules
+typings
diff --git a/javascript/README.md b/javascript/README.md
new file mode 100644
index 00000000000..98ef75674ed
--- /dev/null
+++ b/javascript/README.md
@@ -0,0 +1,50 @@
+
+
+### Installation
+
+From this directory, run:
+
+``` bash
+$ npm install # pull dependencies
+$ tsc # build typescript
+$ webpack # bundle for the browser
+```
+
+### Usage
+The library is designed to be used with node.js or in the browser, this repository contains examples of both.
+
+#### Node
+Import the arrow module:
+
+``` js
+var arrow = require("arrow.js");
+```
+
+See [bin/arrow_schema.js](bin/arrow_schema.js) and [bin/arrow2csv.js](bin/arrow2csv.js) for usage examples.
+
+#### Browser
+Include `dist/arrow-bundle.js` in a `` tag:
+``` html
+
+```
+See [examples/read_file.html](examples/read_file.html) for a usage example - or try it out now at [theneuralbit.github.io/arrow](http://theneuralbit.github.io/arrow)
+
+### API
+##### `arrow.loadSchema(buffer)`
+Returns a JSON representation of the file's Arrow schema.
+
+##### `arrow.loadVectors(buffer)`
+Returns a dictionary of `Vector` objects, one for each column, indexed by the column's name.
+Vector objects have, at minimum, a `get(i)` method and a `length` attribute.
diff --git a/javascript/bin/arrow2csv.js b/javascript/bin/arrow2csv.js
new file mode 100755
index 00000000000..48df2f96483
--- /dev/null
+++ b/javascript/bin/arrow2csv.js
@@ -0,0 +1,47 @@
+#! /usr/bin/env node
+
+// 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.
+
+var fs = require('fs')
+var process = require('process');
+var loadVectors = require('../dist/arrow.js').loadVectors;
+var program = require('commander');
+
+function list (val) {
+ return val.split(',');
+}
+
+program
+ .version('0.1.0')
+ .usage('[options] ')
+ .option('-s --schema ', 'A comma-separated list of column names', list)
+ .parse(process.argv);
+
+if (!program.schema) {
+ program.outputHelp();
+ process.exit(1);
+}
+
+var buf = fs.readFileSync(process.argv[process.argv.length - 1]);
+var vectors = loadVectors(buf);
+
+for (var i = 0; i < vectors[program.schema[0]].length; i += 1|0) {
+ console.log(program.schema.map(function (field) {
+ return '' + vectors[field].get(i);
+ }).join(','));
+}
diff --git a/javascript/bin/arrow_schema.js b/javascript/bin/arrow_schema.js
new file mode 100755
index 00000000000..704477863bf
--- /dev/null
+++ b/javascript/bin/arrow_schema.js
@@ -0,0 +1,25 @@
+#! /usr/bin/env node
+
+// 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.
+
+var fs = require('fs');
+var process = require('process');
+var loadSchema = require('../dist/arrow.js').loadSchema;
+
+var buf = fs.readFileSync(process.argv[process.argv.length - 1]);
+console.log(JSON.stringify(loadSchema(buf), null, '\t'));
diff --git a/javascript/examples/read_file.html b/javascript/examples/read_file.html
new file mode 100644
index 00000000000..02b6f0805c7
--- /dev/null
+++ b/javascript/examples/read_file.html
@@ -0,0 +1,79 @@
+
+
+
+
+
+
+ arrow.js browser test
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
diff --git a/javascript/lib/Arrow_generated.d.ts b/javascript/lib/Arrow_generated.d.ts
new file mode 100644
index 00000000000..1f5b4547a47
--- /dev/null
+++ b/javascript/lib/Arrow_generated.d.ts
@@ -0,0 +1,5 @@
+export var org: {
+ apache: {
+ arrow: any
+ }
+}
diff --git a/javascript/lib/arrow.ts b/javascript/lib/arrow.ts
new file mode 100644
index 00000000000..0762885aef8
--- /dev/null
+++ b/javascript/lib/arrow.ts
@@ -0,0 +1,201 @@
+// 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 { flatbuffers } from 'flatbuffers';
+import { org } from './Arrow_generated';
+var arrow = org.apache.arrow;
+import { vectorFromField, Vector } from './types';
+
+export function loadVectors(buf) {
+ var fileLength = buf.length, bb, footerLengthOffset, footerLength,
+ footerOffset, footer, schema, field, type, type_str, i,
+ len, rb_metas, rb_meta, rtrn, recordBatchBlock, recordBatchBlocks = [];
+ var vectors : Vector[] = [];
+
+ bb = new flatbuffers.ByteBuffer(buf);
+
+ footer = _loadFooter(bb);
+
+ schema = footer.schema();
+
+ for (i = 0, len = schema.fieldsLength(); i < len; i += 1|0) {
+ field = schema.fields(i);
+ vectors.push(vectorFromField(field));
+ }
+
+ for (i = 0; i < footer.recordBatchesLength(); i += 1|0) {
+ recordBatchBlock = footer.recordBatches(i);
+ recordBatchBlocks.push({
+ offset: recordBatchBlock.offset(),
+ metaDataLength: recordBatchBlock.metaDataLength(),
+ bodyLength: recordBatchBlock.bodyLength(),
+ })
+ }
+
+ loadBuffersIntoVectors(recordBatchBlocks, bb, vectors);
+ var rtrn : any = {};
+ for (var i : any = 0; i < vectors.length; i += 1|0) {
+ rtrn[vectors[i].name] = vectors[i]
+ }
+ return rtrn;
+}
+
+export function loadSchema(buf) {
+ var footer = _loadFooter(new flatbuffers.ByteBuffer(buf));
+ var schema = footer.schema();
+
+ return parseSchema(schema);
+}
+
+function _loadFooter(bb) {
+ var fileLength: number = bb.bytes_.length;
+
+ if (fileLength < MAGIC.length*2 + 4) {
+ console.error("file too small " + fileLength);
+ return;
+ }
+
+ if (!_checkMagic(bb.bytes_, 0)) {
+ console.error("missing magic bytes at beginning of file")
+ return;
+ }
+
+ if (!_checkMagic(bb.bytes_, fileLength - MAGIC.length)) {
+ console.error("missing magic bytes at end of file")
+ return;
+ }
+
+ var footerLengthOffset: number = fileLength - MAGIC.length - 4;
+ bb.setPosition(footerLengthOffset);
+ var footerLength: number = Int64FromByteBuffer(bb, footerLengthOffset)
+
+ if (footerLength <= 0 || footerLength + MAGIC.length*2 + 4 > fileLength) {
+ console.log("Invalid footer length: " + footerLength)
+ }
+
+ var footerOffset: number = footerLengthOffset - footerLength;
+ bb.setPosition(footerOffset);
+ var footer = arrow.flatbuf.Footer.getRootAsFooter(bb);
+
+ return footer;
+}
+
+function Int64FromByteBuffer(bb, offset) {
+ return ((bb.bytes_[offset + 3] & 255) << 24) |
+ ((bb.bytes_[offset + 2] & 255) << 16) |
+ ((bb.bytes_[offset + 1] & 255) << 8) |
+ ((bb.bytes_[offset] & 255));
+}
+
+
+var MAGIC_STR = "ARROW1";
+var MAGIC = new Uint8Array(MAGIC_STR.length);
+for (var i = 0; i < MAGIC_STR.length; i += 1|0) {
+ MAGIC[i] = MAGIC_STR.charCodeAt(i);
+}
+
+function _checkMagic(buf, index) {
+ for (var i = 0; i < MAGIC.length; i += 1|0) {
+ if (MAGIC[i] != buf[index + i]) {
+ return false;
+ }
+ }
+ return true;
+}
+
+var TYPEMAP = {}
+TYPEMAP[arrow.flatbuf.Type.NONE] = "NONE";
+TYPEMAP[arrow.flatbuf.Type.Null] = "Null";
+TYPEMAP[arrow.flatbuf.Type.Int] = "Int";
+TYPEMAP[arrow.flatbuf.Type.FloatingPoint] = "FloatingPoint";
+TYPEMAP[arrow.flatbuf.Type.Binary] = "Binary";
+TYPEMAP[arrow.flatbuf.Type.Utf8] = "Utf8";
+TYPEMAP[arrow.flatbuf.Type.Bool] = "Bool";
+TYPEMAP[arrow.flatbuf.Type.Decimal] = "Decimal";
+TYPEMAP[arrow.flatbuf.Type.Date] = "Date";
+TYPEMAP[arrow.flatbuf.Type.Time] = "Time";
+TYPEMAP[arrow.flatbuf.Type.Timestamp] = "Timestamp";
+TYPEMAP[arrow.flatbuf.Type.Interval] = "Interval";
+TYPEMAP[arrow.flatbuf.Type.List] = "List";
+TYPEMAP[arrow.flatbuf.Type.Struct_] = "Struct";
+TYPEMAP[arrow.flatbuf.Type.Union] = "Union";
+
+var VECTORTYPEMAP = {};
+VECTORTYPEMAP[arrow.flatbuf.VectorType.OFFSET] = 'OFFSET';
+VECTORTYPEMAP[arrow.flatbuf.VectorType.DATA] = 'DATA';
+VECTORTYPEMAP[arrow.flatbuf.VectorType.VALIDITY] = 'VALIDITY';
+VECTORTYPEMAP[arrow.flatbuf.VectorType.TYPE] = 'TYPE';
+
+function parseField(field) {
+ var children = [];
+ for (var i = 0; i < field.childrenLength(); i += 1|0) {
+ children.push(parseField(field.children(i)));
+ }
+
+ var layouts = [];
+ for (var i = 0; i < field.layoutLength(); i += 1|0) {
+ layouts.push(VECTORTYPEMAP[field.layout(i).type()]);
+
+ }
+
+ return {
+ name: field.name(),
+ nullable: field.nullable(),
+ type: TYPEMAP[field.typeType()],
+ children: children,
+ layout: layouts
+ };
+}
+
+function parseSchema(schema) {
+ var result = [];
+ var this_result, type;
+ for (var i = 0, len = schema.fieldsLength(); i < len; i += 1|0) {
+ result.push(parseField(schema.fields(i)));
+ }
+ return result;
+}
+
+function parseBuffer(buffer) {
+ return {
+ offset: buffer.offset(),
+ length: buffer.length()
+ };
+}
+
+function loadBuffersIntoVectors(recordBatchBlocks, bb, vectors : Vector[]) {
+ var fieldNode, recordBatchBlock, recordBatch, numBuffers, bufReader = {index: 0, node_index: 1}, field_ctr = 0;
+ var buffer = bb.bytes_.buffer;
+ var baseOffset = bb.bytes_.byteOffset;
+ for (var i = recordBatchBlocks.length - 1; i >= 0; i -= 1|0) {
+ recordBatchBlock = recordBatchBlocks[i];
+ bb.setPosition(recordBatchBlock.offset.low);
+ recordBatch = arrow.flatbuf.RecordBatch.getRootAsRecordBatch(bb);
+ bufReader.index = 0;
+ bufReader.node_index = 0;
+ numBuffers = recordBatch.buffersLength();
+
+ //console.log('num buffers: ' + recordBatch.buffersLength());
+ //console.log('num nodes: ' + recordBatch.nodesLength());
+
+ while (bufReader.index < numBuffers) {
+ //console.log('Allocating buffers starting at ' + bufReader.index + '/' + numBuffers + ' to field ' + field_ctr);
+ vectors[field_ctr].loadData(recordBatch, buffer, bufReader, baseOffset + recordBatchBlock.offset.low + recordBatchBlock.metaDataLength)
+ field_ctr += 1;
+ }
+ }
+}
diff --git a/javascript/lib/bitarray.ts b/javascript/lib/bitarray.ts
new file mode 100644
index 00000000000..82fff32c194
--- /dev/null
+++ b/javascript/lib/bitarray.ts
@@ -0,0 +1,55 @@
+// 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.
+
+export class BitArray {
+ private view: Uint8Array;
+ constructor(buffer: ArrayBuffer, offset: number, length: number) {
+ //if (ArrayBuffer.isView(buffer)) {
+ // var og_view = buffer;
+ // buffer = buffer.buffer;
+ // offset = og_view.offset;
+ // length = og_view.length/og_view.BYTES_PER_ELEMENT*8;
+ //} else if (buffer instanceof ArrayBuffer) {
+ var offset = offset || 0;
+ var length = length;// || buffer.length*8;
+ //} else if (buffer instanceof Number) {
+ // length = buffer;
+ // buffer = new ArrayBuffer(Math.ceil(length/8));
+ // offset = 0;
+ //}
+
+ this.view = new Uint8Array(buffer, offset, Math.ceil(length/8));
+ }
+
+ get(i) {
+ var index = (i >> 3) | 0; // | 0 converts to an int. Math.floor works too.
+ var bit = i % 8; // i % 8 is just as fast as i & 7
+ return (this.view[index] & (1 << bit)) !== 0;
+ }
+
+ set(i) {
+ var index = (i >> 3) | 0;
+ var bit = i % 8;
+ this.view[index] |= 1 << bit;
+ }
+
+ unset(i) {
+ var index = (i >> 3) | 0;
+ var bit = i % 8;
+ this.view[index] &= ~(1 << bit);
+ }
+}
diff --git a/javascript/lib/types.ts b/javascript/lib/types.ts
new file mode 100644
index 00000000000..bbc75581005
--- /dev/null
+++ b/javascript/lib/types.ts
@@ -0,0 +1,328 @@
+// 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 { BitArray } from './bitarray';
+import { TextDecoder } from 'text-encoding';
+import { org } from './Arrow_generated';
+var arrow = org.apache.arrow;
+
+interface ArrayView {
+ slice(start: number, end: number) : ArrayView
+ toString() : string
+}
+
+export abstract class Vector {
+ name: string;
+ length: number;
+ null_count: number;
+ constructor(name: string) {
+ this.name = name;
+ }
+ /* Access datum at index i */
+ abstract get(i);
+ /* Return array representing data in the range [start, end) */
+ abstract slice(start: number, end: number);
+
+ /* Use recordBatch fieldNodes and Buffers to construct this Vector */
+ public loadData(recordBatch: any, buffer: any, bufReader: any, baseOffset: any) {
+ var fieldNode = recordBatch.nodes(bufReader.node_index);
+ this.length = fieldNode.length();
+ this.null_count = fieldNode.length();
+ bufReader.node_index += 1|0;
+
+ this.loadBuffers(recordBatch, buffer, bufReader, baseOffset);
+ }
+
+ protected abstract loadBuffers(recordBatch: any, buffer: any, bufReader: any, baseOffset: any);
+
+ /* Helper function for loading a VALIDITY buffer (for Nullable types) */
+ static loadValidityBuffer(recordBatch, buffer, bufReader, baseOffset) : BitArray {
+ var buf_meta = recordBatch.buffers(bufReader.index);
+ var offset = baseOffset + buf_meta.offset().low;
+ var length = buf_meta.length().low;
+ bufReader.index += 1|0;
+ return new BitArray(buffer, offset, length*8);
+ }
+
+ /* Helper function for loading an OFFSET buffer */
+ static loadOffsetBuffer(recordBatch, buffer, bufReader, baseOffset) : Int32Array {
+ var buf_meta = recordBatch.buffers(bufReader.index);
+ var offset = baseOffset + buf_meta.offset().low;
+ var length = buf_meta.length().low/Int32Array.BYTES_PER_ELEMENT;
+ bufReader.index += 1|0;
+ return new Int32Array(buffer, offset, length);
+ }
+
+}
+
+class SimpleVector extends Vector {
+ protected dataView: T;
+ private TypedArray: {new(buffer: any, offset: number, length: number) : T, BYTES_PER_ELEMENT: number};
+
+ constructor (TypedArray: {new(buffer: any, offset: number, length: number): T, BYTES_PER_ELEMENT: number}, name: string) {
+ super(name);
+ this.TypedArray = TypedArray;
+ }
+
+ get(i) {
+ return this.dataView[i];
+ }
+
+ loadBuffers(recordBatch, buffer, bufReader, baseOffset) {
+ this.dataView = this.loadDataBuffer(recordBatch, buffer, bufReader, baseOffset);
+ }
+
+ loadDataBuffer(recordBatch, buffer, bufReader, baseOffset) : T {
+ var buf_meta = recordBatch.buffers(bufReader.index);
+ var offset = baseOffset + buf_meta.offset().low;
+ var length = buf_meta.length().low/this.TypedArray.BYTES_PER_ELEMENT;
+ bufReader.index += 1|0;
+ return new this.TypedArray(buffer, offset, length);
+ }
+
+ getDataView() {
+ return this.dataView;
+ }
+
+ toString() {
+ return this.dataView.toString();
+ }
+
+ slice(start, end) {
+ return this.dataView.slice(start, end);
+ }
+}
+
+class NullableSimpleVector extends SimpleVector {
+ private validityView: BitArray;
+
+ get(i: number) {
+ if (this.validityView.get(i)) return this.dataView[i];
+ else return null
+ }
+
+ loadBuffers(recordBatch, buffer, bufReader, baseOffset) {
+ this.validityView = Vector.loadValidityBuffer(recordBatch, buffer, bufReader, baseOffset);
+ this.dataView = this.loadDataBuffer(recordBatch, buffer, bufReader, baseOffset);
+ }
+
+}
+
+class Uint8Vector extends SimpleVector { constructor(name: string) { super(Uint8Array, name); }; }
+class Uint16Vector extends SimpleVector { constructor(name: string) { super(Uint16Array, name); }; }
+class Uint32Vector extends SimpleVector { constructor(name: string) { super(Uint32Array, name); }; }
+class Int8Vector extends SimpleVector { constructor(name: string) { super(Uint8Array, name); }; }
+class Int16Vector extends SimpleVector { constructor(name: string) { super(Uint16Array, name); }; }
+class Int32Vector extends SimpleVector { constructor(name: string) { super(Uint32Array, name); }; }
+class Float32Vector extends SimpleVector { constructor(name: string) { super(Float32Array, name); }; }
+class Float64Vector extends SimpleVector { constructor(name: string) { super(Float64Array, name); }; }
+
+class NullableUint8Vector extends NullableSimpleVector { constructor(name: string) { super(Uint8Array, name); }; }
+class NullableUint16Vector extends NullableSimpleVector { constructor(name: string) { super(Uint16Array, name); }; }
+class NullableUint32Vector extends NullableSimpleVector { constructor(name: string) { super(Uint32Array, name); }; }
+class NullableInt8Vector extends NullableSimpleVector { constructor(name: string) { super(Uint8Array, name); }; }
+class NullableInt16Vector extends NullableSimpleVector { constructor(name: string) { super(Uint16Array, name); }; }
+class NullableInt32Vector extends NullableSimpleVector { constructor(name: string) { super(Uint32Array, name); }; }
+class NullableFloat32Vector extends NullableSimpleVector { constructor(name: string) { super(Float32Array, name); }; }
+class NullableFloat64Vector extends NullableSimpleVector { constructor(name: string) { super(Float64Array, name); }; }
+
+class Utf8Vector extends SimpleVector {
+ protected offsetView: Int32Array;
+ static decoder: TextDecoder = new TextDecoder('utf8');
+
+ constructor(name: string) {
+ super(Uint8Array, name);
+ }
+
+ loadBuffers(recordBatch, buffer, bufReader, baseOffset) {
+ this.offsetView = Vector.loadOffsetBuffer(recordBatch, buffer, bufReader, baseOffset);
+ this.dataView = this.loadDataBuffer(recordBatch, buffer, bufReader, baseOffset);
+ }
+
+ get(i) {
+ return Utf8Vector.decoder.decode
+ (this.dataView.slice(this.offsetView[i], this.offsetView[i + 1]));
+ }
+
+ slice(start: number, end: number) {
+ var rtrn: string[] = [];
+ for (var i: number = start; i < end; i += 1|0) {
+ rtrn.push(this.get(i));
+ }
+ return rtrn;
+ }
+}
+
+class NullableUtf8Vector extends Utf8Vector {
+ private validityView: BitArray;
+
+ loadBuffers(recordBatch, buffer, bufReader, baseOffset) {
+ this.validityView = Vector.loadValidityBuffer(recordBatch, buffer, bufReader, baseOffset);
+ this.offsetView = Vector.loadOffsetBuffer(recordBatch, buffer, bufReader, baseOffset);
+ this.dataView = this.loadDataBuffer(recordBatch, buffer, bufReader, baseOffset);
+ }
+
+ get(i) {
+ if (!this.validityView.get(i)) return null;
+ return super.get(i);
+ }
+}
+
+// Nested Types
+class ListVector extends Uint32Vector {
+ private dataVector: Vector;
+
+ constructor(name, dataVector : Vector) {
+ super(name);
+ this.dataVector = dataVector;
+ }
+
+ loadBuffers(recordBatch, buffer, bufReader, baseOffset) {
+ super.loadBuffers(recordBatch, buffer, bufReader, baseOffset);
+ this.dataVector.loadData(recordBatch, buffer, bufReader, baseOffset);
+ this.length -= 1;
+ }
+
+ get(i) {
+ var offset = super.get(i)
+ if (offset === null) {
+ return null;
+ }
+ var next_offset = super.get(i + 1)
+ return this.dataVector.slice(offset, next_offset)
+ }
+
+ toString() {
+ return "length: " + (this.length);
+ }
+
+ slice(start : number, end : number) { return []; };
+}
+
+class NullableListVector extends ListVector {
+ private validityView: BitArray;
+
+ loadBuffers(recordBatch, buffer, bufReader, baseOffset) {
+ this.validityView = Vector.loadValidityBuffer(recordBatch, buffer, bufReader, baseOffset);
+ super.loadBuffers(recordBatch, buffer, bufReader, baseOffset);
+ }
+
+ get(i) {
+ if (!this.validityView.get(i)) return null;
+ return super.get(i);
+ }
+}
+
+class StructVector extends Vector {
+ private validityView: BitArray;
+ private vectors : Vector[];
+ constructor(name: string, vectors: Vector[]) {
+ super(name);
+ this.vectors = vectors;
+ }
+
+ loadBuffers(recordBatch, buffer, bufReader, baseOffset) {
+ this.validityView = Vector.loadValidityBuffer(recordBatch, buffer, bufReader, baseOffset);
+ this.vectors.forEach((v: Vector) => v.loadData(recordBatch, buffer, bufReader, baseOffset));
+ }
+
+ get(i : number) {
+ if (!this.validityView.get(i)) return null;
+ return this.vectors.map((v: Vector) => v.get(i));
+ }
+
+ slice(start : number, end : number) {
+ var rtrn = [];
+ for (var i: number = start; i < end; i += 1|0) {
+ rtrn.push(this.get(i));
+ }
+ return rtrn;
+ }
+}
+
+class DateVector extends SimpleVector {
+ constructor (name: string) {
+ super(Uint32Array, name);
+ }
+
+ get (i) {
+ return new Date(super.get(2*i+1)*Math.pow(2,32) + super.get(2*i));
+ }
+}
+
+class NullableDateVector extends DateVector {
+ private validityView: BitArray;
+
+ loadBuffers(recordBatch, buffer, bufReader, baseOffset) {
+ this.validityView = Vector.loadValidityBuffer(recordBatch, buffer, bufReader, baseOffset);
+ super.loadBuffers(recordBatch, buffer, bufReader, baseOffset);
+ }
+
+ get (i) {
+ if (!this.validityView.get(i)) return null;
+ return super.get(i);
+ }
+}
+
+var BASIC_TYPES = [arrow.flatbuf.Type.Int, arrow.flatbuf.Type.FloatingPoint, arrow.flatbuf.Type.Utf8, arrow.flatbuf.Type.Date];
+
+export function vectorFromField(field) : Vector {
+ var typeType = field.typeType();
+ if (BASIC_TYPES.indexOf(typeType) >= 0) {
+ var type = field.typeType();
+ if (type === arrow.flatbuf.Type.Int) {
+ type = field.type(new arrow.flatbuf.Int());
+ var VectorConstructor : {new(string) : Vector};
+ if (type.isSigned()) {
+ if (type.bitWidth() == 32)
+ VectorConstructor = field.nullable() ? NullableInt32Vector : Int32Vector;
+ else if (type.bitWidth() == 16)
+ VectorConstructor = field.nullable() ? NullableInt16Vector : Int16Vector;
+ else if (type.bitWidth() == 8)
+ VectorConstructor = field.nullable() ? NullableInt8Vector : Int8Vector;
+ } else {
+ if (type.bitWidth() == 32)
+ VectorConstructor = field.nullable() ? NullableUint32Vector : Uint32Vector;
+ else if (type.bitWidth() == 16)
+ VectorConstructor = field.nullable() ? NullableUint16Vector : Uint16Vector;
+ else if (type.bitWidth() == 8)
+ VectorConstructor = field.nullable() ? NullableUint8Vector : Uint8Vector;
+ }
+ } else if (type === arrow.flatbuf.Type.FloatingPoint) {
+ type = field.type(new arrow.flatbuf.FloatingPoint());
+ if (type.precision() == arrow.flatbuf.Precision.SINGLE)
+ VectorConstructor = field.nullable() ? NullableFloat32Vector : Float32Vector;
+ else if (type.precision() == arrow.flatbuf.Precision.DOUBLE)
+ VectorConstructor = field.nullable() ? NullableFloat64Vector : Float64Vector;
+ } else if (type === arrow.flatbuf.Type.Utf8) {
+ VectorConstructor = field.nullable() ? NullableUtf8Vector : Utf8Vector;
+ } else if (type === arrow.flatbuf.Type.Date) {
+ VectorConstructor = field.nullable() ? NullableDateVector : DateVector;
+ }
+
+ return new VectorConstructor(field.name());
+ } else if (typeType === arrow.flatbuf.Type.List) {
+ var dataVector = vectorFromField(field.children(0));
+ return field.nullable() ? new NullableListVector(field.name(), dataVector) : new ListVector(field.name(), dataVector);
+ } else if (typeType === arrow.flatbuf.Type.Struct_) {
+ var vectors : Vector[] = [];
+ for (var i : number = 0; i < field.childrenLength(); i += 1|0) {
+ vectors.push(vectorFromField(field.children(i)));
+ }
+ return new StructVector(field.name(), vectors);
+ }
+}
diff --git a/javascript/package.json b/javascript/package.json
new file mode 100644
index 00000000000..b1e583b7d9d
--- /dev/null
+++ b/javascript/package.json
@@ -0,0 +1,19 @@
+{
+ "name": "arrow",
+ "version": "0.0.0",
+ "description": "",
+ "main": "dist/arrow.js",
+ "scripts": {
+ "postinstall": "./postinstall.sh",
+ "test": "echo \"Error: no test specified\" && exit 1"
+ },
+ "author": "",
+ "license": "Apache-2.0",
+ "devDependencies": {
+ "flatbuffers": "^1.5.0",
+ "text-encoding": "^0.6.4"
+ },
+ "dependencies": {
+ "commander": "^2.9.0"
+ }
+}
diff --git a/javascript/postinstall.sh b/javascript/postinstall.sh
new file mode 100755
index 00000000000..1e6622fa4f2
--- /dev/null
+++ b/javascript/postinstall.sh
@@ -0,0 +1,18 @@
+#!/bin/bash
+
+# Licensed 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. See accompanying LICENSE file.
+
+echo "Compiling flatbuffer schemas..."
+#flatc -o lib --js ../format/Message.fbs ../format/File.fbs
+flatc -o lib --js ../format/*.fbs
+cat lib/*_generated.js > lib/Arrow_generated.js
diff --git a/javascript/tsconfig.json b/javascript/tsconfig.json
new file mode 100644
index 00000000000..89c31ef85a1
--- /dev/null
+++ b/javascript/tsconfig.json
@@ -0,0 +1,14 @@
+{
+ "compilerOptions": {
+ "outDir": "./dist/",
+ "allowJs": true,
+ "target": "es5",
+ "module": "commonjs",
+ "moduleResolution": "node"
+ },
+ "include": [
+ "typings/index.d.ts",
+ "lib/*.js",
+ "lib/*.ts"
+ ]
+}
diff --git a/javascript/webpack.config.js b/javascript/webpack.config.js
new file mode 100644
index 00000000000..a0ed56370f6
--- /dev/null
+++ b/javascript/webpack.config.js
@@ -0,0 +1,21 @@
+// Licensed 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. See accompanying LICENSE file.
+
+module.exports = {
+ entry: './dist/arrow.js',
+ output: {
+ path: __dirname + '/dist',
+ filename: 'arrow-bundle.js',
+ libraryTarget: 'var',
+ library: 'arrow'
+ }
+};