diff --git a/Package.swift b/Package.swift index 17db745..d92f2d0 100644 --- a/Package.swift +++ b/Package.swift @@ -20,13 +20,11 @@ import PackageDescription let package = Package( name: "Arrow", platforms: [ - .macOS(.v15) + .macOS(.v26) ], products: [ - .library( - name: "Arrow", - targets: ["Arrow"] - ) + .library(name: "Arrow", targets: ["Arrow"]), + .library(name: "ArrowIPC", targets: ["ArrowIPC"]), ], dependencies: [ .package( @@ -46,9 +44,13 @@ let package = Package( from: "1.29.0" ), .package( - url: "https://github.com/apple/swift-binary-parsing", + url: "https://github.com/apple/swift-binary-parsing.git", from: "0.0.1" ), + .package( + url: "https://github.com/swiftlang/swift-subprocess.git", + branch: "main" + ), ], targets: [ .target( @@ -81,16 +83,11 @@ let package = Package( // build: .unsafeFlags(["-warnings-as-errors"]) ] ), - // .target( - // name: "ArrowC", - // swiftSettings: [ - // // build: .unsafeFlags(["-warnings-as-errors"]) - // ] - // ), .target( name: "ArrowFlight", dependencies: [ "Arrow", + "ArrowIPC", .product(name: "GRPC", package: "grpc-swift"), .product(name: "SwiftProtobuf", package: "swift-protobuf"), ], @@ -111,7 +108,11 @@ let package = Package( ), .testTarget( name: "ArrowIPCTests", - dependencies: ["Arrow", "ArrowIPC"], + dependencies: [ + "Arrow", + "ArrowIPC", + .product(name: "Subprocess", package: "swift-subprocess"), + ], resources: [ .copy("Resources/") ], diff --git a/README.md b/README.md index ef7d9b6..11ba67e 100644 --- a/README.md +++ b/README.md @@ -1,33 +1,59 @@ # Swift Arrow -![Swift 6.2](https://img.shields.io/badge/Swift-6.2-orange?style=for-the-badge&logo=swift&logoColor=white) - A Swift implementation of Apache Arrow, the universal columnar format for fast data interchange and in-memory analytics. -This is a **work in progress**. Do not use in production. Progress is fast however, expect a beta in December. +The in-memory contiguous buffers allow constant-time random access to large, structured and strongly-typed datasets. + +## Project status: + +IPC serialization / deserialization has been tested against the Arrow integration testing JSON files, using the following strategy: + +1. Read the [Arrow cpp21 generated files](https://github.com/apache/arrow-testing/tree/master/data/arrow-ipc-stream/integration/cpp-21.0.0) into memory. +2. Encode the results to Codable & Equatable structs that can read and write the [test data format.](https://arrow.apache.org/docs/format/Integration.html#json-test-data-format). +3. Read the test JSON into the same Codable & Equatable struct and compare with the deserialized results,using Swift equality. This + +IPC serialization uses the same methodology, except a serialization-deserialization round-trip to/from Arrow IPC is performed prior to step 2, i.e. the results under test have been deserialized from IPC, re-serialized to IPC and deserialized again before being compared to the JSON. + +The following types are fully supported: + +* Primitive types: boolean, int8, int16, int32, int64, uint8, uint16, uint32, uint64, float16, float32, float64. +* Temporal types: timestamp, date32, date64, time32, time64, duration. +* Variable length types: binary and string, plus their fixed-width equivalents +* Nested and recursively nested types: lists and structs, structs of lists etc. +* Maps: Represented as list of key-values, which is spec compliant, however the public API will change. +* Binary views: binaryView and utf8View. + +All binary arrays (variable, fixed and view) can be accessed via BinaryArrayProtocol. The same applies to StringArrayProtocol. ## Array interface Arrow arrays are backed by a standard memory layout: https://arrow.apache.org/docs/format/Columnar.html -In Swift-Arrow, every array conforms to: +In Swift-Arrow, every array has the following type-erased capabilities: ```swift -public protocol ArrowArrayProtocol { - associatedtype ItemType - subscript(_ index: Int) -> ItemType? { get } +public protocol AnyArrowArrayProtocol: Sendable { var offset: Int { get } var length: Int { get } + var nullCount: Int { get } func slice(offset: Int, length: Int) -> Self func any(at index: Int) -> Any? + var bufferSizes: [Int] { get } + var buffers: [ArrowBufferProtocol] { get } } ``` -The in-memory contiguous buffers allow constant-time random access. +Every array also supports typed access: -Every Arrow array supports nullable elements. This is encoded as an optional bit-packed validity buffer aka null array aka bitfield. -In psuedocode, bitfield[index] == 0 means null or invalid, and bitfield[index] == 1 means not null or valid. +```swift +public protocol ArrowArrayProtocol: AnyArrowArrayProtocol { + associatedtype ItemType + subscript(_ index: Int) -> ItemType? { get } +} +``` + +Every Arrow array supports nullable elements. This is encoded as an optional bit-packed validity buffer. Fixed-width types are encoded back-to-back, with placeholder values for nulls. For example the array: ```swift @@ -101,19 +127,20 @@ typealias ArrowArrayUtf8 = ArrowArrayVariable< > `` - ## Relationship to Arrow-Swift -This project is based on Arrow-Swift, the official Swift implementation of Apache Arrow. The decision was made to at least temporarily operate independently of the Apache Software Foundation (ASF). Currently there are no active ASF maintaners with knowledge of Swift, and the only [Apache approved CI for Swift](https://github.com/apache/infrastructure-actions/blob/main/approved_patterns.yml) is [setup-swift which is unmaintained](https://github.com/swift-actions/setup-swift/issues), leading to intermittent CI failures. This has led to delays in much-needed fixes being implemented. +This project is based on Arrow-Swift, the official Swift implementation of Apache Arrow. The decision was made to at least temporarily operate independently of the Apache Software Foundation (ASF) to improve development velocity. The intention is to continue contributing to the official Apache-Swift repository, however changes can be iterated on more quickly here. Original source: https://github.com/apache/arrow-swift Changes made since forking Arrow-Swift: +* `ArrowType` has been moved from a class hierarchy to an enum to improve usability and concurrency support. +* IPC is now fully zero-copy, whereas previously file data were copied to pointer-backed arrays. +* Gold-standard IPC tests have been added. * CI uses the swiftlang workflows: https://github.com/swiftlang/github-workflows -* `ArrowType` has been moved from a class hierarchy to an enum to improve concurrency support. * Tests have been migrated to Swift Testing. -* A migration from reference to value types, where appropriate, has begun. * A DockerFile for compiling ArrowFlight protocol buffers and grpc classes is provided. -* C export has been made Swift 6 compatible through MainActor annotations. This is a workaround. +* C import/export has been removed. + diff --git a/Scripts/readArrowIPC.py b/Scripts/readArrowIPC.py new file mode 100644 index 0000000..83e59ce --- /dev/null +++ b/Scripts/readArrowIPC.py @@ -0,0 +1,20 @@ +# Useful for debugging IPC writing issues. +import pyarrow as pa +import sys + +print(f"PyArrow version: {pa.__version__}") + +try: + with open(sys.argv[1], 'rb') as f: + reader = pa.ipc.open_file(f) + print(f"Schema: {reader.schema}") + print(f"Num batches: {reader.num_record_batches}") + + for i in range(reader.num_record_batches): + batch = reader.get_batch(i) + print(f"Batch {i}: {batch.num_rows} rows, {batch.num_columns} columns") + + print("✓ File read successfully") +except Exception as e: + print(f"✗ Error: {e}") + sys.exit(1) diff --git a/Sources/Arrow/Array/Array.swift b/Sources/Arrow/Array/Array.swift index c4d427c..4aac340 100644 --- a/Sources/Arrow/Array/Array.swift +++ b/Sources/Arrow/Array/Array.swift @@ -14,7 +14,8 @@ import Foundation -public protocol AnyArrowArrayProtocol { +/// The type-independent Arrow array capabilities. +public protocol AnyArrowArrayProtocol: Sendable { var offset: Int { get } var length: Int { get } var nullCount: Int { get } @@ -24,7 +25,10 @@ public protocol AnyArrowArrayProtocol { var buffers: [ArrowBufferProtocol] { get } } -internal protocol ArrowArrayProtocol: AnyArrowArrayProtocol { +/// Typed array conformance. +/// +/// Public access to typed arays is provided via concrete types or individual protocols as appropriate. +public protocol ArrowArrayProtocol: AnyArrowArrayProtocol { associatedtype ItemType subscript(_ index: Int) -> ItemType? { get } } @@ -38,28 +42,33 @@ extension ArrowArrayProtocol { // MARK: Capability protocols. -public protocol StringArrayProtocol { - var length: Int { get } +/// A type which provides access to arrays of utf8 encoded `String`, with opaque offset types. +/// +/// The underlying array may be `String` or `LargeString`. +public protocol StringArrayProtocol: AnyArrowArrayProtocol { subscript(index: Int) -> String? { get } } extension ArrowArrayVariable: StringArrayProtocol where ItemType == String {} +extension ArrowArrayBinaryView: StringArrayProtocol where ItemType == String {} -protocol BinaryArrayProtocol: ArrowArrayProtocol where ItemType == Data {} +/// A type which provides access to arrays of `Data`, with opaque offset types. +/// +/// The underlying array may have fixed or variable-length items. +protocol BinaryArrayProtocol: AnyArrowArrayProtocol { + subscript(index: Int) -> Data? { get } +} extension ArrowArrayFixedSizeBinary: BinaryArrayProtocol {} extension ArrowArrayVariable: BinaryArrayProtocol where ItemType == Data, OffsetType: FixedWidthInteger & SignedInteger {} +extension ArrowArrayBinaryView: BinaryArrayProtocol where ItemType == Data {} -protocol Utf8ArrayProtocol: ArrowArrayProtocol where ItemType == String {} -extension ArrowArrayVariable: Utf8ArrayProtocol -where ItemType == String, OffsetType: FixedWidthInteger & SignedInteger {} - -public protocol ListArrayProtocol { - var length: Int { get } +public protocol ListArrayProtocol: AnyArrowArrayProtocol { var values: AnyArrowArrayProtocol { get } subscript(index: Int) -> AnyArrowArrayProtocol? { get } } extension ArrowListArray: ListArrayProtocol {} extension ArrowFixedSizeListArray: ListArrayProtocol {} +// TODO: Add large lists. // MARK: Array implementations. @@ -243,6 +252,7 @@ public struct ArrowArrayVariable< let startOffset = offsetsBuffer[offsetIndex] let endOffset = offsetsBuffer[offsetIndex + 1] + precondition(endOffset >= startOffset, "Corrupted Arrow data") return valueBuffer.loadVariable( at: Int(startOffset), arrayLength: Int(endOffset - startOffset) @@ -315,11 +325,9 @@ public struct ArrowArrayDate64: ArrowArrayProtocol { } ///// An Arrow list array which may be nested arbitrarily. -public struct ArrowListArray: ArrowArrayProtocol -where - OffsetsBuffer: FixedWidthBufferProtocol, - OffsetsBuffer.ElementType: FixedWidthInteger & SignedInteger -{ +public struct ArrowListArray< + OffsetType: FixedWidthInteger & SignedInteger +>: ArrowArrayProtocol { public let offset: Int public let length: Int public var bufferSizes: [Int] { @@ -331,14 +339,14 @@ where public var nullCount: Int { nullBuffer.nullCount } let nullBuffer: NullBuffer - let offsetsBuffer: OffsetsBuffer + let offsetsBuffer: any FixedWidthBufferProtocol public let values: AnyArrowArrayProtocol public init( offset: Int = 0, length: Int, nullBuffer: NullBuffer, - offsetsBuffer: OffsetsBuffer, + offsetsBuffer: any FixedWidthBufferProtocol, values: AnyArrowArrayProtocol ) { self.offset = offset diff --git a/Sources/Arrow/Array/ArrowArrayBinaryView.swift b/Sources/Arrow/Array/ArrowArrayBinaryView.swift new file mode 100644 index 0000000..0d4b1d2 --- /dev/null +++ b/Sources/Arrow/Array/ArrowArrayBinaryView.swift @@ -0,0 +1,102 @@ +// Copyright 2025 The Columnar Swift Contributors +// +// 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. + +public struct ArrowArrayBinaryView: ArrowArrayProtocol +{ + public let offset: Int + public let length: Int + private let nullBuffer: NullBuffer + private let viewsBuffer: any FixedWidthBufferProtocol + let dataBuffers: [any VariableLengthBufferProtocol] + + public var bufferSizes: [Int] { + [nullBuffer.length, viewsBuffer.length] + dataBuffers.map { $0.length } + } + + public var buffers: [ArrowBufferProtocol] { + [nullBuffer, viewsBuffer] + dataBuffers.map { $0 as ArrowBufferProtocol } + } + + public var nullCount: Int { nullBuffer.nullCount } + + public init>( + offset: Int = 0, + length: Int, + nullBuffer: NullBuffer, + viewsBuffer: Views, + dataBuffers: [any VariableLengthBufferProtocol] + ) { + self.offset = offset + self.length = length + self.nullBuffer = nullBuffer + self.viewsBuffer = viewsBuffer + self.dataBuffers = dataBuffers + } + + public subscript(index: Int) -> ItemType? { + let offsetIndex = self.offset + index + guard self.nullBuffer.isSet(offsetIndex) else { + return nil + } + + let view = viewsBuffer[offsetIndex] + + if view.isInline { + // Fast path: data is inline + return view.withInlineData { dataSpan in + dataSpan.withUnsafeBufferPointer { buffer in + ItemType(buffer) + } + } + } else { + // Referenced data + let bufferIndex = Int(view.bufferIndex) + let offset = Int(view.offset) + + precondition( + bufferIndex >= 0 && bufferIndex < dataBuffers.count, + "Invalid buffer index") + + let dataBuffer = dataBuffers[bufferIndex] + return dataBuffer.loadVariable( + at: offset, + arrayLength: Int(view.length) + ) + } + } + + public func slice(offset: Int, length: Int) -> Self { + // True zero-copy: just adjust offset/length, share all buffers + .init( + offset: offset, + length: length, + nullBuffer: nullBuffer, + viewsBuffer: viewsBuffer, + dataBuffers: dataBuffers + ) + } + + /// Compact the array by copying referenced data into fewer buffers. + public func compact() -> Self { + // TODO: Implement compaction strategy + // For now, just return self + self + } + + /// Get buffer utilization statistics. + public func bufferStats() -> [(bufferIndex: Int, utilization: Double)] { + // TODO: Track which views reference which buffers + [] + } +} diff --git a/Sources/Arrow/Array/ArrowArrayBinaryViewBuilder.swift b/Sources/Arrow/Array/ArrowArrayBinaryViewBuilder.swift new file mode 100644 index 0000000..051e224 --- /dev/null +++ b/Sources/Arrow/Array/ArrowArrayBinaryViewBuilder.swift @@ -0,0 +1,110 @@ +// Copyright 2025 The Columnar Swift Contributors +// +// 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. + +/// A builder for Arrow arrays holding binary view types. +public class ArrayBuilderBinaryView: AnyArrayBuilder { + public typealias ArrayType = ArrowArrayBinaryView + + public var length: Int + let nullBuilder: NullBufferBuilder + let viewsBuilder: FixedWidthBufferBuilder + var dataBuffers: [VariableLengthTypeBufferBuilder] + var currentDataBuffer: VariableLengthTypeBufferBuilder + + private let inlineThreshold = 12 + // Limit buffer size to avoid huge buffers + private let maxBufferSize: Int + + public init(maxBufferSize: Int = 2 * 1024 * 1024) { + self.length = 0 + self.nullBuilder = NullBufferBuilder() + self.viewsBuilder = FixedWidthBufferBuilder() + self.dataBuffers = [] + self.currentDataBuffer = VariableLengthTypeBufferBuilder() + self.maxBufferSize = maxBufferSize + } + + public func append(_ value: Element) { + length += 1 + nullBuilder.appendValid(true) + + let data = value.data + + if data.count <= self.inlineThreshold { + // Store inline + let view = BinaryView.inline(data.span) + viewsBuilder.append(view) + } else { + // Check if we need a new buffer + if currentDataBuffer.length + data.count > maxBufferSize { + // Finalize current buffer and start a new one + dataBuffers.append(currentDataBuffer) + currentDataBuffer = VariableLengthTypeBufferBuilder() + } + + // Ensure capacity + let requiredCapacity = currentDataBuffer.length + data.count + if requiredCapacity > currentDataBuffer.capacity { + var newCapacity = currentDataBuffer.capacity + while newCapacity < requiredCapacity { + newCapacity *= 2 + } + currentDataBuffer.increaseCapacity(to: newCapacity) + } + + let offset = currentDataBuffer.length + currentDataBuffer.append(data) + + // Extract prefix (first 4 bytes) + let prefix: UInt32 = data.withUnsafeBytes { bytes in + guard bytes.count >= 4 else { return 0 } + return bytes.loadUnaligned(as: UInt32.self) + } + + let view = BinaryView.referenced( + length: Int32(data.count), + prefix: prefix, + bufferIndex: Int32(dataBuffers.count), // Current buffer index + offset: Int32(offset) + ) + viewsBuilder.append(view) + } + } + + public func appendNull() { + length += 1 + nullBuilder.appendValid(false) + // Append a zero-length inline view for null + let emptyView = BinaryView() + viewsBuilder.append(emptyView) + } + + public func finish() -> ArrayType { + // Add current buffer to the list if it has data + if currentDataBuffer.length > 0 { + dataBuffers.append(currentDataBuffer) + } + + // Finish all data buffers + let finishedDataBuffers = dataBuffers.map { $0.finish() } + + return ArrayType( + offset: 0, + length: length, + nullBuffer: nullBuilder.finish(), + viewsBuffer: viewsBuilder.finish(), + dataBuffers: finishedDataBuffers + ) + } +} diff --git a/Sources/Arrow/Array/ArrowColumn.swift b/Sources/Arrow/Array/ArrowColumn.swift new file mode 100644 index 0000000..63e8529 --- /dev/null +++ b/Sources/Arrow/Array/ArrowColumn.swift @@ -0,0 +1,37 @@ +// Copyright 2025 The Apache Software Foundation +// Copyright 2025 The Columnar Swift Contributors +// +// 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. + +public class ArrowColumn { + let dataHolder: any ChunkedArrayProtocol + public let field: ArrowField + public var length: Int { self.dataHolder.length } + public var nullCount: Int { self.dataHolder.nullCount } + + public var name: String { field.name } + + public init(_ field: ArrowField, chunked: any ChunkedArrayProtocol) { + self.field = field + self.dataHolder = chunked + } + + public func data() throws(ArrowError) -> ChunkedArray { + if let holder = self.dataHolder as? ChunkedArray { + return holder + } else { + throw .init( + .runtimeError("Could not cast array holder to chunked array.")) + } + } +} diff --git a/Sources/Arrow/Array/ArrowTable.swift b/Sources/Arrow/Array/ArrowTable.swift new file mode 100644 index 0000000..7c2ca71 --- /dev/null +++ b/Sources/Arrow/Array/ArrowTable.swift @@ -0,0 +1,192 @@ +// Copyright 2025 The Apache Software Foundation +// Copyright 2025 The Columnar Swift Contributors +// +// 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. + +import Foundation + +public class ArrowTable { + public let schema: ArrowSchema + public let rowCount: Int + public let columns: [ArrowColumn] + public init(_ schema: ArrowSchema, columns: [ArrowColumn]) { + self.schema = schema + self.columns = columns + self.rowCount = columns[0].length + } + + /// Create an ArrowTable from a 'RecordBatch' list. + /// - Parameter recordBatches: The record batches. + /// - Returns: An `ArrowResult` holding an `ArrowTable` on success. + /// - Throws: an `ArrowError` if arrays have no elements or if elements have mismatched types. + public static func from( + recordBatches: [RecordBatch] + ) throws(ArrowError) -> ArrowTable { + if recordBatches.isEmpty { + throw .init(.arrayHasNoElements) + } + var holders: [[AnyArrowArrayProtocol]] = [] + let schema = recordBatches[0].schema + for recordBatch in recordBatches { + for index in 0.. ArrowColumn { + // Dispatch based on the field's type, not the first holder + switch field.type { + case .int8: + return try makeTypedColumn(field, arrays, type: Int8.self) + case .int16: + return try makeTypedColumn(field, arrays, type: Int16.self) + case .int32: + return try makeTypedColumn(field, arrays, type: Int32.self) + case .int64: + return try makeTypedColumn(field, arrays, type: Int64.self) + case .uint8: + return try makeTypedColumn(field, arrays, type: UInt8.self) + case .uint16: + return try makeTypedColumn(field, arrays, type: UInt16.self) + case .uint32: + return try makeTypedColumn(field, arrays, type: UInt32.self) + case .uint64: + return try makeTypedColumn(field, arrays, type: UInt64.self) + case .float16: + return try makeTypedColumn(field, arrays, type: Float16.self) + case .float32: + return try makeTypedColumn(field, arrays, type: Float.self) + case .float64: + return try makeTypedColumn(field, arrays, type: Double.self) + case .binary: + return try makeTypedColumn(field, arrays, type: Data.self) + case .utf8: + return try makeTypedColumn(field, arrays, type: String.self) + case .boolean: + return try makeTypedColumn(field, arrays, type: Bool.self) + case .date32, .date64: + return try makeTypedColumn(field, arrays, type: Date.self) + // TODO: make a fuzzer to make sure all types are hit + + case .strct, .list: + return ArrowColumn( + field, + chunked: try AnyChunkedArray(arrays) + ) + default: + throw .init(.unknownType("Unsupported type: \(field.type)")) + } + } + + private static func makeTypedColumn( + _ field: ArrowField, + _ holders: [AnyArrowArrayProtocol], + type: T.Type + ) throws(ArrowError) -> ArrowColumn { + var arrays: [any ArrowArrayProtocol] = [] + for holder in holders { + guard let array = holder as? (any ArrowArrayProtocol) else { + throw .init( + .runtimeError( + "Array type mismatch: expected \(T.self) for field \(field.name)" + )) + } + arrays.append(array) + } + return ArrowColumn( + field, + chunked: try ChunkedArray(arrays) + ) + } + + public class Builder { + let schemaBuilder = ArrowSchema.Builder() + var columns: [ArrowColumn] = [] + + public init() {} + + // @discardableResult + // public func addColumn( + // _ fieldName: String, + // arrowArray: any ArrowArray + // ) throws -> Builder { + // self.addColumn(fieldName, chunked: try ChunkedArrayX([arrowArray])) + // } + + // @discardableResult + // public func addColumn( + // _ fieldName: String, + // chunked: ChunkedArray + // ) -> Builder { + // let field = ArrowField( + // name: fieldName, + // dataType: chunked.type, + // isNullable: chunked.nullCount != 0 + // ) + // self.schemaBuilder.addField(field) + // let column = ArrowColumn(field, chunked: chunked) + // self.columns.append(column) + // return self + // } + + @discardableResult + public func addColumn( + _ field: ArrowField, + arrowArray: any ArrowArrayProtocol + ) throws -> Builder { + self.schemaBuilder.addField(field) + let holder = try ChunkedArray([arrowArray]) + self.columns.append(ArrowColumn(field, chunked: holder)) + return self + } + + @discardableResult + public func addColumn( + _ field: ArrowField, + chunked: ChunkedArray + ) -> Builder { + self.schemaBuilder.addField(field) + let column = ArrowColumn(field, chunked: chunked) + self.columns.append(column) + return self + } + + @discardableResult + public func addColumn(_ column: ArrowColumn) -> Builder { + self.schemaBuilder.addField(column.field) + self.columns.append(column) + return self + } + + public func finish() -> ArrowTable { + ArrowTable(self.schemaBuilder.finish(), columns: self.columns) + } + } +} diff --git a/Sources/Arrow/Array/Builder.swift b/Sources/Arrow/Array/Builder.swift index 1913f40..c099b88 100644 --- a/Sources/Arrow/Array/Builder.swift +++ b/Sources/Arrow/Array/Builder.swift @@ -14,7 +14,7 @@ import Foundation -protocol AnyArrayBuilder { +public protocol AnyArrayBuilder { associatedtype ArrayType: ArrowArrayProtocol var length: Int { get } func appendNull() @@ -24,7 +24,7 @@ protocol AnyArrayBuilder { } extension AnyArrayBuilder { - func appendAny(_ value: Any) { + public func appendAny(_ value: Any) { guard let x = value as? ArrayType.ItemType else { fatalError( "Type mismatch: \(type(of: self)) expects \(ArrayType.self), got \(type(of: value))" @@ -39,7 +39,7 @@ public class ArrayBuilderBoolean: AnyArrayBuilder { public typealias ArrayType = ArrowArrayBoolean - var length: Int + public var length: Int let nullBuilder: NullBufferBuilder let valueBuilder: NullBufferBuilder @@ -122,7 +122,7 @@ public class ArrayBuilderFixedSizedBinary: { public typealias ArrayType = ArrowArrayFixedSizeBinary - var length: Int + public var length: Int let byteWidth: Int let nullBuilder: NullBufferBuilder let valueBuilder: VariableLengthTypeBufferBuilder @@ -172,12 +172,13 @@ public class ArrayBuilderFixedSizedBinary: /// A builder for Arrow arrays holding variable length types. public class ArrayBuilderVariableLength< - Element: VariableLength, OffsetType: FixedWidthInteger & SignedInteger + Element: VariableLength, + OffsetType: FixedWidthInteger & SignedInteger >: AnyArrayBuilder { public typealias ArrayType = ArrowArrayVariable - var length: Int + public var length: Int let nullBuilder: NullBufferBuilder let offsetsBuilder: FixedWidthBufferBuilder let valueBuilder: VariableLengthTypeBufferBuilder @@ -262,7 +263,7 @@ public struct ArrayBuilderDate64: AnyArrayBuilder { public init() {} - var length: Int { + public var length: Int { builder.length } @@ -289,9 +290,9 @@ public typealias ArrayBuilderTime64 = ArrayBuilderFixedWidth /// A builder for Arrow arrays holding Timestamp values. public typealias ArrayBuilderTimestamp = ArrayBuilderFixedWidth -class ArrayBuilderList { +public class ArrayBuilderList { - typealias ArrayType = ArrowListArray> + typealias ArrayType = ArrowListArray var length: Int let nullBuilder: NullBufferBuilder diff --git a/Sources/Arrow/Array/ChunkedArray.swift b/Sources/Arrow/Array/ChunkedArray.swift new file mode 100644 index 0000000..d1d3329 --- /dev/null +++ b/Sources/Arrow/Array/ChunkedArray.swift @@ -0,0 +1,135 @@ +// Copyright 2025 The Apache Software Foundation +// Copyright 2025 The Columnar Swift Contributors +// +// 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. + +public protocol ChunkedArrayProtocol { + var length: Int { get } + var nullCount: Int { get } +} + +public class ChunkedArray: ChunkedArrayProtocol { + public let arrays: [any ArrowArrayProtocol] + public let nullCount: Int + public let length: Int + + public init(_ arrays: [any ArrowArrayProtocol]) throws(ArrowError) { + if arrays.count == 0 { + throw ArrowError(.arrayHasNoElements) + } + + var len: Int = 0 + var nullCount: Int = 0 + for array in arrays { + len += array.length + nullCount += array.nullCount + } + + self.arrays = arrays + self.length = len + self.nullCount = nullCount + } + + public subscript(_ index: Int) -> T? { + if arrays.count == 0 { + return nil + } + var localIndex = index + var arrayIndex = 0 + var len: Int = arrays[arrayIndex].length + while localIndex > (len - 1) { + arrayIndex += 1 + if arrayIndex > arrays.count { + return nil + } + localIndex -= len + len = arrays[arrayIndex].length + } + return arrays[arrayIndex][localIndex] + } + + public func asString(_ index: Int) -> String { + guard let value = self[index] else { + return "" + } + return "\(value)" + } +} + +/// A type-erased chunked array, suitable for complex types. +public final class AnyChunkedArray: ChunkedArrayProtocol { + private let arrays: [any AnyArrowArrayProtocol] + public let nullCount: Int + public let length: Int + + // Cached chunk boundaries + private let chunkOffsets: [Int] + + public init(_ arrays: [any AnyArrowArrayProtocol]) throws(ArrowError) { + guard !arrays.isEmpty else { + throw ArrowError(.arrayHasNoElements) + } + + var len: Int = 0 + var nullCount: Int = 0 + var offsets: [Int] = [0] + + for array in arrays { + len += array.length + nullCount += array.nullCount + offsets.append(len) + } + + self.arrays = arrays + self.length = len + self.nullCount = nullCount + self.chunkOffsets = offsets + } + + public subscript(_ index: Int) -> Any? { + guard index >= 0, index < length else { + return nil + } + // Binary search to find the right chunk + var low = 0 + var high = arrays.count - 1 + + while low <= high { + let mid = (low + high) / 2 + let chunkStart = chunkOffsets[mid] + let chunkEnd = chunkOffsets[mid + 1] + + if index < chunkStart { + high = mid - 1 + } else if index >= chunkEnd { + low = mid + 1 + } else { + // Found the right chunk + let localIndex = index - chunkStart + return arrays[mid].any(at: localIndex) + } + } + return nil + } + + public func any(at index: Int) -> Any? { + self[index] + } + + public func asString(_ index: Int) -> String { + guard let value = self[index] else { + return "" + } + return "\(value)" + } +} diff --git a/Sources/Arrow/ArrowArray.swift b/Sources/Arrow/ArrowArray.swift deleted file mode 100644 index d938f93..0000000 --- a/Sources/Arrow/ArrowArray.swift +++ /dev/null @@ -1,414 +0,0 @@ -// Copyright 2025 The Apache Software Foundation -// Copyright 2025 The Columnar Swift Contributors -// -// 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. - -import ArrowC -import Foundation - -/// A type-erased ArrowArray. -public protocol AnyArrowArray { - var type: ArrowType { get } - var length: UInt { get } - var nullCount: UInt { get } - var arrowData: ArrowData { get } - var bufferData: [Data] { get } // TODO: remove - var bufferDataSizes: [Int] { get } // TODO: remove - func asAny(_ index: UInt) -> Any? - func asString(_ index: UInt) -> String - func setCArrayPtr(_ cArrayPtr: UnsafePointer?) -} - -/// The interface for Arrow array types. -public protocol ArrowArray: AnyArrowArray { - associatedtype ItemType - init(_ arrowData: ArrowData) throws(ArrowError) - subscript(_ index: UInt) -> ItemType? { get } -} - -public class ArrowArrayBase: ArrowArray { - - public var arrowData: ArrowData - public var cArrayPtr: UnsafePointer? = nil - - required public init(_ arrowData: ArrowData) throws(ArrowError) { - self.arrowData = arrowData - } - - public subscript(_ index: UInt) -> T? { - fatalError("Base class is abstract.") - } - - public func asString(_ index: UInt) -> String { - guard let value = self[index] else { - return "" - } - return "\(value)" - } - - public func asAny(_ index: UInt) -> Any? { - self[index] - } - - public func setCArrayPtr(_ cArrayPtr: UnsafePointer?) { - self.cArrayPtr = cArrayPtr - } - - deinit { - if let cArrayPtr { - ArrowCImporter.release(cArrayPtr) - } - } -} - -extension ArrowArrayBase { - public var nullCount: UInt { - arrowData.nullCount - } - - public var length: UInt { - arrowData.length - } - - public var type: ArrowType { - arrowData.type - } - - // TODO: Remove - public var bufferData: [Data] { - arrowData.bufferData - } - - // TODO: Remove - public var bufferDataSizes: [Int] { - arrowData.bufferDataSizes - } - - public func isNull(at index: UInt) throws(ArrowError) -> Bool { - if index >= self.length { - throw .outOfBounds(index: Int64(index)) - } - return arrowData.isNull(index) - } -} - -// MARK: Fixed Arrays - -public class FixedArray: ArrowArrayBase where T: BitwiseCopyable { - - public override subscript(_ index: UInt) -> ItemType? { - if arrowData.isNull(index) { - return nil - } - let value: ItemType = arrowData.load(at: index) - return value - } -} - -public class StringArray: ArrowArrayBase { - - public override subscript(_ index: UInt) -> String? { - if self.arrowData.isNull(index) { - return nil - } - let offsetBuffer: OffsetsBuffer = arrowData.offsets - let (startIndex, endIndex) = offsetBuffer.offsets(at: Int(index)) - let arrayLength = Int(endIndex - startIndex) - let value: String = self.arrowData.loadVariable( - at: Int(startIndex), arrayLength: arrayLength) - return value - } -} - -public class BoolArray: ArrowArrayBase { - - public override subscript(_ index: UInt) -> Bool? { - if self.arrowData.isNull(index) { - return nil - } - return arrowData.isNullValue(at: index) - } -} - -public class Date32Array: ArrowArrayBase { - - public override subscript(_ index: UInt) -> Date? { - if self.arrowData.isNull(index) { - return nil - } - let milliseconds: UInt32 = arrowData.load(at: index) - return Date(timeIntervalSince1970: TimeInterval(milliseconds * 86400)) - } -} - -public class Date64Array: ArrowArrayBase { - - public override subscript(_ index: UInt) -> Date? { - if self.arrowData.isNull(index) { - return nil - } - - let milliseconds: UInt64 = self.arrowData.load(at: index) - return Date(timeIntervalSince1970: TimeInterval(milliseconds / 1000)) - } -} - -public typealias Time64Array = FixedArray - -public typealias Time32Array = FixedArray - -public class TimestampArray: FixedArray { - - public struct FormattingOptions: Equatable { - public var dateFormat: String = "yyyy-MM-dd HH:mm:ss.SSS" - public var locale: Locale = .current - public var includeTimezone: Bool = true - public var fallbackToRaw: Bool = true - - public init( - dateFormat: String = "yyyy-MM-dd HH:mm:ss.SSS", - locale: Locale = .current, - includeTimezone: Bool = true, - fallbackToRaw: Bool = true - ) { - self.dateFormat = dateFormat - self.locale = locale - self.includeTimezone = includeTimezone - self.fallbackToRaw = fallbackToRaw - } - - public static func == (lhs: FormattingOptions, rhs: FormattingOptions) - -> Bool - { - lhs.dateFormat == rhs.dateFormat - && lhs.locale.identifier == rhs.locale.identifier - && lhs.includeTimezone == rhs.includeTimezone - && lhs.fallbackToRaw == rhs.fallbackToRaw - } - } - - private var cachedFormatter: DateFormatter? - private var cachedOptions: FormattingOptions? - - public func formattedDate( - at index: UInt, - options: FormattingOptions = FormattingOptions() - ) -> String? { - guard let timestamp = self[index] else { return nil } - - guard case .timestamp(let timeUnit, let timezone) = self.arrowData.type - else { - return options.fallbackToRaw ? "\(timestamp)" : nil - } - - let date = dateFromTimestamp(timestamp, unit: timeUnit) - - if cachedFormatter == nil || cachedOptions != options { - let formatter = DateFormatter() - formatter.dateFormat = options.dateFormat - formatter.locale = options.locale - if options.includeTimezone, let timezone { - formatter.timeZone = TimeZone(identifier: timezone) - } - cachedFormatter = formatter - cachedOptions = options - } - return cachedFormatter?.string(from: date) - } - - private func dateFromTimestamp( - _ timestamp: Int64, - unit: TimeUnit - ) -> Date { - let timeInterval: TimeInterval - switch unit { - case .second: - timeInterval = TimeInterval(timestamp) - case .millisecond: - timeInterval = TimeInterval(timestamp) / 1_000 - case .microsecond: - timeInterval = TimeInterval(timestamp) / 1_000_000 - case .nanosecond: - timeInterval = TimeInterval(timestamp) / 1_000_000_000 - } - return Date(timeIntervalSince1970: timeInterval) - } - - public override func asString(_ index: UInt) -> String { - if let formatted = formattedDate(at: index) { - return formatted - } else { - return "\(self[index] ?? 0)" - } - } -} - -public class BinaryArray: ArrowArrayBase { - - public struct Options { - public var printAsHex = false - public var printEncoding: String.Encoding = .utf8 - } - - public var options = Options() - - public override subscript(_ index: UInt) -> Data? { - if self.arrowData.isNull(index) { - return nil - } - - let (startIndex, endIndex) = arrowData.offsets.offsets(at: Int(index)) - - let arrayLength = Int(endIndex - startIndex) - - let data: Data = self.arrowData.loadVariable( - at: Int(startIndex), arrayLength: arrayLength) - return data - } - - public override func asString(_ index: UInt) -> String { - guard let data = self[index] else { return "" } - if options.printAsHex { - return data.hexEncodedString() - } else { - if let string = String(data: data, encoding: options.printEncoding) { - return string - } else { - return "" - } - } - } -} - -public class NestedArray: ArrowArrayBase<[Any?]> { - - private var children: [AnyArrowArray]? - - public required init( - _ arrowData: ArrowData - ) throws(ArrowError) { - try super.init(arrowData) - - switch arrowData.type { - case .list(let field): - guard arrowData.children.count == 1 else { - throw ArrowError.invalid("List array must have exactly one child") - } - self.children = [ - try ArrowArrayLoader.loadArray( - field.type, - with: arrowData.children[0] - ) - ] - case .strct(let _): - var fields: [AnyArrowArray] = [] - for child in arrowData.children { - fields.append( - try ArrowArrayLoader.loadArray(child.type, with: child) - ) - } - self.children = fields - default: - throw .invalid( - "NestedArray only supports list and struct types, got: \(arrowData.type)" - ) - } - } - - public override subscript(_ index: UInt) -> [Any?]? { - if self.arrowData.isNull(index) { - return nil - } - guard let children = self.children else { - return nil - } - switch arrowData.type { - case .list(let _): - guard let values = children.first else { return nil } - - let (startIndex, endIndex) = arrowData.offsets.offsets(at: Int(index)) - var items: [Any?] = [] - for i in startIndex.. String { - switch arrowData.type { - case .list(let _): - if self.arrowData.isNull(index) { - return "null" - } - guard let list = self[index] else { - return "null" - } - var output = "[" - for (i, item) in list.enumerated() { - if i > 0 { - output.append(",") - } - switch item { - case nil: - output.append("null") - case let asStringItem as AnyArrowArray: - output.append(asStringItem.asString(0)) - case let someItem?: - output.append("\(someItem)") - } - } - output.append("]") - return output - case .strct(let _): - if self.arrowData.isNull(index) { - return "" - } - var output = "{" - if let children = self.children { - let parts = children.compactMap { child in - child.asString(index) - } - output.append(parts.joined(separator: ",")) - } - output += "}" - return output - default: - return "" - } - } - - public var fields: [AnyArrowArray]? { - if case .strct(_) = arrowData.type { - return children - } else { - return nil - } - } - - public var values: AnyArrowArray? { - if case .list(_) = arrowData.type { - return children?.first - } else { - return nil - } - } -} diff --git a/Sources/Arrow/ArrowArrayBuilder.swift b/Sources/Arrow/ArrowArrayBuilder.swift deleted file mode 100644 index 23d1914..0000000 --- a/Sources/Arrow/ArrowArrayBuilder.swift +++ /dev/null @@ -1,547 +0,0 @@ -// Copyright 2025 The Apache Software Foundation -// Copyright 2025 The Columnar Swift Contributors -// -// 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. - -import Foundation - -// MARK: Array builder interface. - -/// A type which builds a type-erased `ArrowArray`. -public protocol AnyArrowArrayBuilder { - /// Returns an unparameterised `ArrowArray`. - /// - Returns: The type-erased Arrow array. - func toAnyArrowArray() throws(ArrowError) -> AnyArrowArray - func appendAny(_ val: Any?) -} - -/// A type which can build an `ArrowArray`of `ItemType`. -public protocol ArrowArrayBuilder { - associatedtype BufferBuilder: ArrowBufferBuilder - associatedtype ArrayType: ArrowArray - where ArrayType.ItemType == BufferBuilder.ItemType - - func append(_ vals: BufferBuilder.ItemType?...) - func append(_ vals: [BufferBuilder.ItemType?]) - func append(_ val: BufferBuilder.ItemType?) - func appendAny(_ val: Any?) - func finish() throws(ArrowError) -> ArrayType -} - -internal protocol ArrowArrayBuilderInternal: ArrowArrayBuilder { - var arrowType: ArrowType { get } - var bufferBuilder: BufferBuilder { get } -} - -extension ArrowArrayBuilderInternal { - - var length: UInt { self.bufferBuilder.length } - var capacity: UInt { self.bufferBuilder.capacity } - var nullCount: UInt { self.bufferBuilder.nullCount } - var offset: UInt { self.bufferBuilder.offset } - - public func append(_ vals: BufferBuilder.ItemType?...) { - for val in vals { - self.bufferBuilder.append(val) - } - } - - public func append(_ vals: [BufferBuilder.ItemType?]) { - for val in vals { - self.bufferBuilder.append(val) - } - } - - public func append(_ val: BufferBuilder.ItemType?) { - self.bufferBuilder.append(val) - } - - public func appendAny(_ val: Any?) { - self.bufferBuilder.append(val as? BufferBuilder.ItemType) - } - - /// Returns the byte width of this type if it is a primitive type. - public func stride() -> Int { - self.arrowType.getStride() - } - - public func toAnyArrowArray() throws(ArrowError) -> AnyArrowArray { - try self.finish() - } -} - -// MARK: Base implementation. - -// Note: It would be preferable to move this to a protocol, however -// ListArrayBuilder overrides finish. This is delicate because protocol -// extension method dispatching uses static dispatch, so overrides are not -// called when type erasure is used in nested types. -public class ArrowArrayBuilderBase< - BufferBuilder: ArrowBufferBuilder, - ArrayType: ArrowArray ->: AnyArrowArrayBuilder, ArrowArrayBuilderInternal { - let arrowType: ArrowType - let bufferBuilder: BufferBuilder - - fileprivate init(_ type: ArrowType) throws(ArrowError) { - self.arrowType = type - self.bufferBuilder = BufferBuilder() - } - - public func finish() throws(ArrowError) -> ArrayType { - let buffers = self.bufferBuilder.finish() - let arrowData = ArrowData( - self.arrowType, - buffers: buffers, - nullCount: self.nullCount - ) - let array = try ArrayType(arrowData) - return array - } -} - -/// An array builder for numeric types. -public class NumberArrayBuilder: ArrowArrayBuilderBase< - FixedBufferBuilder, - FixedArray -> -where ItemType: Numeric, ItemType: BitwiseCopyable { - fileprivate convenience init() throws(ArrowError) { - try self.init(try ArrowTypeConverter.infoForNumericType(ItemType.self)) - } -} - -/// A `String` array builder. -public class StringArrayBuilder: ArrowArrayBuilderBase< - VariableBufferBuilder, - StringArray -> -{ - fileprivate convenience init() throws(ArrowError) { - try self.init(.utf8) - } -} - -/// A `Data` array builder. -public class BinaryArrayBuilder: ArrowArrayBuilderBase< - VariableBufferBuilder, - BinaryArray -> -{ - fileprivate convenience init() throws(ArrowError) { - try self.init(.binary) - } -} - -/// A `Bool` array builder. -public class BoolArrayBuilder: ArrowArrayBuilderBase< - BoolBufferBuilder, BoolArray -> -{ - fileprivate convenience init() throws(ArrowError) { - try self.init(.boolean) - } -} - -/// A 32-bit date array builder. -public class Date32ArrayBuilder: ArrowArrayBuilderBase< - Date32BufferBuilder, - Date32Array -> -{ - fileprivate convenience init() throws(ArrowError) { - try self.init(.date32) - } -} - -/// A 64-bit date array builder. -public class Date64ArrayBuilder: ArrowArrayBuilderBase< - Date64BufferBuilder, - Date64Array -> -{ - fileprivate convenience init() throws(ArrowError) { - try self.init(.date64) - } -} - -// A 32-bit elaspsed time builder. -public class Time32ArrayBuilder: ArrowArrayBuilderBase< - FixedBufferBuilder, - Time32Array -> -{ - fileprivate convenience init(_ unit: TimeUnit) throws(ArrowError) { - try self.init(.time32(unit)) - } -} - -// A 64-bit elaspsed time builder. -public class Time64ArrayBuilder: ArrowArrayBuilderBase< - FixedBufferBuilder, - Time64Array -> -{ - fileprivate convenience init(_ unit: TimeUnit) throws(ArrowError) { - try self.init(.time64(unit)) - } -} - -// A Timestamp array builder. -public class TimestampArrayBuilder: ArrowArrayBuilderBase< - FixedBufferBuilder, - TimestampArray -> -{ - fileprivate convenience init( - _ unit: TimeUnit, timezone: String? = nil - ) throws(ArrowError) { - try self.init(.timestamp(unit, timezone)) - } -} - -// MARK: Struct array builder. - -/// Builds an array of structs. -public class StructArrayBuilder: ArrowArrayBuilderBase< - StructBufferBuilder, - NestedArray -> -{ - let builders: [any AnyArrowArrayBuilder] - let fields: [ArrowField] - public init( - _ fields: [ArrowField], - builders: [any AnyArrowArrayBuilder] - ) throws(ArrowError) { - self.fields = fields - self.builders = builders - try super.init(.strct(fields)) - self.bufferBuilder.initializeTypeInfo(fields) - } - - public init(_ fields: [ArrowField]) throws(ArrowError) { - self.fields = fields - var builders: [any AnyArrowArrayBuilder] = [] - for field in fields { - builders.append( - try ArrowArrayBuilders.loadBuilder(arrowType: field.type)) - } - self.builders = builders - try super.init(.strct(fields)) - } - - public func append(_ values: [Any?]?) { - self.bufferBuilder.append(values) - if let anyValues = values { - for index in 0.. ArrayType { - let buffers = self.bufferBuilder.finish() - var childData: [ArrowData] = [] - for builder in self.builders { - childData.append(try builder.toAnyArrowArray().arrowData) - } - let arrowData = ArrowData( - self.arrowType, buffers: buffers, - children: childData, - nullCount: self.nullCount, - length: self.length) - let structArray = try NestedArray(arrowData) - return structArray - } -} - -// MARK: List array builder. - -/// Builds a `NestedArray`containing lists of `ItemType`. -/// -/// Both lists and items in lists are nullablie. -public class ListArrayBuilder: ArrowArrayBuilderBase< - ListBufferBuilder, - NestedArray -> -{ - let valueBuilder: any AnyArrowArrayBuilder - - public override init(_ elementType: ArrowType) throws(ArrowError) { - guard case .list(let field) = elementType else { - throw .invalid("Expected a field with type .list") - } - self.valueBuilder = try ArrowArrayBuilders.loadBuilder( - arrowType: field.type - ) - try super.init(elementType) - } - - // Overrides the protocol extension. - // Swift currently provides no marker for this. - public func append(_ values: [Any?]?) { - self.bufferBuilder.append(values) - if let vals = values { - for val in vals { - self.valueBuilder.appendAny(val) - } - } - } - - public override func finish() throws(ArrowError) -> ArrayType { - let buffers = self.bufferBuilder.finish() - let childData = try valueBuilder.toAnyArrowArray().arrowData - let arrowData = ArrowData( - self.arrowType, - buffers: buffers, - children: [childData], - nullCount: self.nullCount, - length: self.length - ) - return try NestedArray(arrowData) - } -} - -public enum ArrowArrayBuilders { - public static func builder( - for builderType: Any.Type - ) throws(ArrowError) -> AnyArrowArrayBuilder { - if builderType == Int8.self || builderType == Int8?.self { - return try ArrowArrayBuilders.loadNumberArrayBuilder() - as NumberArrayBuilder - } else if builderType == Int16.self || builderType == Int16?.self { - return try ArrowArrayBuilders.loadNumberArrayBuilder() - as NumberArrayBuilder - } else if builderType == Int32.self || builderType == Int32?.self { - return try ArrowArrayBuilders.loadNumberArrayBuilder() - as NumberArrayBuilder - } else if builderType == Int64.self || builderType == Int64?.self { - return try ArrowArrayBuilders.loadNumberArrayBuilder() - as NumberArrayBuilder - } else if builderType == Float.self || builderType == Float?.self { - return try ArrowArrayBuilders.loadNumberArrayBuilder() - as NumberArrayBuilder - } else if builderType == UInt8.self || builderType == UInt8?.self { - return try ArrowArrayBuilders.loadNumberArrayBuilder() - as NumberArrayBuilder - } else if builderType == UInt16.self || builderType == UInt16?.self { - return try ArrowArrayBuilders.loadNumberArrayBuilder() - as NumberArrayBuilder - } else if builderType == UInt32.self || builderType == UInt32?.self { - return try ArrowArrayBuilders.loadNumberArrayBuilder() - as NumberArrayBuilder - } else if builderType == UInt64.self || builderType == UInt64?.self { - return try ArrowArrayBuilders.loadNumberArrayBuilder() - as NumberArrayBuilder - } else if builderType == Double.self || builderType == Double?.self { - return try ArrowArrayBuilders.loadNumberArrayBuilder() - as NumberArrayBuilder - } else if builderType == String.self || builderType == String?.self { - return try ArrowArrayBuilders.loadStringArrayBuilder() - } else if builderType == Bool.self || builderType == Bool?.self { - return try ArrowArrayBuilders.loadBoolArrayBuilder() - } else if builderType == Date.self || builderType == Date?.self { - return try ArrowArrayBuilders.loadDate64ArrayBuilder() - } else { - throw .invalid("Invalid type for builder: \(builderType)") - } - } - - public static func isValidBuilderType(_ type: T.Type) -> Bool { - type == Int8?.self || type == Int16?.self || type == Int32?.self - || type == Int64?.self - || type == UInt8?.self || type == UInt16?.self || type == UInt32?.self - || type == UInt64?.self - || type == String?.self || type == Double?.self || type == Float?.self - || type == Date?.self - || type == Bool?.self || type == Bool.self || type == Int8.self - || type == Int16.self - || type == Int32.self || type == Int64.self || type == UInt8.self - || type == UInt16.self - || type == UInt32.self || type == UInt64.self || type == String.self - || type == Double.self - || type == Float.self || type == Date.self - } - - public static func structArrayBuilderForType( - _ obj: T - ) throws -> StructArrayBuilder { - let mirror = Mirror(reflecting: obj) - var builders: [AnyArrowArrayBuilder] = [] - var fields: [ArrowField] = [] - for (property, value) in mirror.children { - guard let propertyName = property else { - continue - } - let builderType = type(of: value) - let arrowType = try ArrowTypeConverter.infoForType(builderType) - fields.append( - ArrowField( - name: propertyName, - dataType: arrowType, - isNullable: true - ) - ) - builders.append(try loadBuilder(arrowType: arrowType)) - } - return try StructArrayBuilder(fields, builders: builders) - } - - public static func loadBuilder( - arrowType: ArrowType - ) throws(ArrowError) -> AnyArrowArrayBuilder { - switch arrowType { - case .uint8: - return try loadNumberArrayBuilder() as NumberArrayBuilder - case .uint16: - return try loadNumberArrayBuilder() as NumberArrayBuilder - case .uint32: - return try loadNumberArrayBuilder() as NumberArrayBuilder - case .uint64: - return try loadNumberArrayBuilder() as NumberArrayBuilder - case .int8: - return try loadNumberArrayBuilder() as NumberArrayBuilder - case .int16: - return try loadNumberArrayBuilder() as NumberArrayBuilder - case .int32: - return try loadNumberArrayBuilder() as NumberArrayBuilder - case .int64: - return try loadNumberArrayBuilder() as NumberArrayBuilder - case .float16: - return try loadNumberArrayBuilder() as NumberArrayBuilder - case .float64: - return try loadNumberArrayBuilder() as NumberArrayBuilder - case .float32: - return try loadNumberArrayBuilder() as NumberArrayBuilder - case .utf8: - return try StringArrayBuilder() - case .boolean: - return try BoolArrayBuilder() - case .binary: - return try BinaryArrayBuilder() - case .date32: - return try Date32ArrayBuilder() - case .date64: - return try Date64ArrayBuilder() - case .time32(let unit): - return try Time32ArrayBuilder(unit) - case .time64(let unit): - return try Time64ArrayBuilder(unit) - case .timestamp(let unit, _): - return try TimestampArrayBuilder(unit) - case .strct(let fields): - return try StructArrayBuilder(fields) - case .list(_): - return try ListArrayBuilder(arrowType) - default: - throw ArrowError.unknownType( - "Builder not found for arrow type: \(arrowType)" - ) - } - } - - public static func loadNumberArrayBuilder() throws(ArrowError) - -> NumberArrayBuilder - { - let type = T.self - if type == Int8.self { - return try NumberArrayBuilder() - } else if type == Int16.self { - return try NumberArrayBuilder() - } else if type == Int32.self { - return try NumberArrayBuilder() - } else if type == Int64.self { - return try NumberArrayBuilder() - } else if type == UInt8.self { - return try NumberArrayBuilder() - } else if type == UInt16.self { - return try NumberArrayBuilder() - } else if type == UInt32.self { - return try NumberArrayBuilder() - } else if type == UInt64.self { - return try NumberArrayBuilder() - } else if type == Float.self { - return try NumberArrayBuilder() - } else if type == Double.self { - return try NumberArrayBuilder() - } else { - throw ArrowError.unknownType("Type is invalid for NumberArrayBuilder") - } - } - - public static func loadStringArrayBuilder() throws(ArrowError) - -> StringArrayBuilder - { - try StringArrayBuilder() - } - - public static func loadBoolArrayBuilder() throws(ArrowError) - -> BoolArrayBuilder - { - try BoolArrayBuilder() - } - - public static func loadDate32ArrayBuilder() throws(ArrowError) - -> Date32ArrayBuilder - { - try Date32ArrayBuilder() - } - - public static func loadDate64ArrayBuilder() throws(ArrowError) - -> Date64ArrayBuilder - { - try Date64ArrayBuilder() - } - - public static func loadBinaryArrayBuilder() throws(ArrowError) - -> BinaryArrayBuilder - { - try BinaryArrayBuilder() - } - - public static func loadTime32ArrayBuilder( - _ unit: TimeUnit - ) throws(ArrowError) -> Time32ArrayBuilder { - try Time32ArrayBuilder(unit) - } - - public static func loadTime64ArrayBuilder( - _ unit: TimeUnit - ) throws(ArrowError) -> Time64ArrayBuilder { - try Time64ArrayBuilder(unit) - } - - public static func loadTimestampArrayBuilder( - _ unit: TimeUnit, - timezone: String? = nil - ) throws -> TimestampArrayBuilder { - try TimestampArrayBuilder(unit, timezone: timezone) - } - - public static func loadStructArrayBuilder( - _ fields: [ArrowField] - ) throws(ArrowError) -> StructArrayBuilder { - try StructArrayBuilder(fields) - } - - public static func loadListArrayBuilder( - _ elementType: ArrowType - ) throws(ArrowError) -> ListArrayBuilder { - try ListArrayBuilder(elementType) - } -} diff --git a/Sources/Arrow/ArrowArrayLoader.swift b/Sources/Arrow/ArrowArrayLoader.swift deleted file mode 100644 index 9fb9415..0000000 --- a/Sources/Arrow/ArrowArrayLoader.swift +++ /dev/null @@ -1,68 +0,0 @@ -// Copyright 2025 The Apache Software Foundation -// Copyright 2025 The Columnar-Swift Contributors -// -// 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. - -// TODO: Duplicated -struct ArrowArrayLoader { - public static func loadArray( - _ arrowType: ArrowType, - with arrowData: ArrowData - ) throws(ArrowError) -> any AnyArrowArray { - switch arrowType { - case .int8: - return try FixedArray(arrowData) - case .int16: - return try FixedArray(arrowData) - case .int32: - return try FixedArray(arrowData) - case .int64: - return try FixedArray(arrowData) - case .uint8: - return try FixedArray(arrowData) - case .uint16: - return try FixedArray(arrowData) - case .uint32: - return try FixedArray(arrowData) - case .uint64: - return try FixedArray(arrowData) - case .float64: - return try FixedArray(arrowData) - case .float32: - return try FixedArray(arrowData) - case .date32: - return try Date32Array(arrowData) - case .date64: - return try Date64Array(arrowData) - case .time32: - return try Time32Array(arrowData) - case .time64: - return try Time64Array(arrowData) - case .timestamp: - return try TimestampArray(arrowData) - case .utf8: - return try StringArray(arrowData) - case .boolean: - return try BoolArray(arrowData) - case .binary: - return try BinaryArray(arrowData) - case .strct(let _): - return try NestedArray(arrowData) - case .list(let _): - return try NestedArray(arrowData) - default: - throw .invalid("Array not found for type: \(arrowType)") - } - } - -} diff --git a/Sources/Arrow/ArrowBuffer.swift b/Sources/Arrow/ArrowBuffer.swift deleted file mode 100644 index a517ed8..0000000 --- a/Sources/Arrow/ArrowBuffer.swift +++ /dev/null @@ -1,109 +0,0 @@ -// Copyright 2025 The Apache Software Foundation -// Copyright 2025 The Columnar Swift Contributors -// -// 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. - -import Foundation - -public class ArrowBuffer { - static let minLength: UInt = 1 << 5 - static let maxLength = UInt.max - fileprivate(set) var length: UInt - let capacity: UInt - public let rawPointer: UnsafeMutableRawPointer - let isMemoryOwner: Bool - - init( - length: UInt, - capacity: UInt, - rawPointer: UnsafeMutableRawPointer, - isMemoryOwner: Bool = true - ) { - self.length = length - self.capacity = capacity - self.rawPointer = rawPointer - self.isMemoryOwner = isMemoryOwner - } - - deinit { - if isMemoryOwner { - self.rawPointer.deallocate() - } - } - - func append(to data: inout Data) { - let ptr = UnsafePointer(rawPointer.assumingMemoryBound(to: UInt8.self)) - data.append(ptr, count: Int(capacity)) - } - - static func createEmptyBuffer() -> ArrowBuffer { - ArrowBuffer( - length: 0, - capacity: 0, - rawPointer: UnsafeMutableRawPointer.allocate( - byteCount: 0, alignment: .zero)) - } - - static func createBuffer(_ data: [UInt8], length: UInt) -> ArrowBuffer { - let byteCount = UInt(data.count) - let capacity = alignTo64(byteCount) - let rawPointer = allocateArray(byteCount: Int(capacity)) - rawPointer.copyMemory(from: data, byteCount: data.count) - return ArrowBuffer( - length: length, capacity: capacity, rawPointer: rawPointer) - } - - static func createBuffer(_ length: UInt, size: UInt, doAlign: Bool = true) - -> ArrowBuffer - { - let actualLen = max(length, ArrowBuffer.minLength) - let byteCount = size * actualLen - var capacity = byteCount - if doAlign { - capacity = alignTo64(byteCount) - } - - let rawPointer = allocateArray(byteCount: Int(capacity)) - rawPointer.initializeMemory( - as: UInt8.self, repeating: 0, count: Int(capacity)) - return ArrowBuffer( - length: length, capacity: capacity, rawPointer: rawPointer) - } - - static func copyCurrent(_ from: ArrowBuffer, to: inout ArrowBuffer, len: UInt) - { - to.rawPointer.copyMemory(from: from.rawPointer, byteCount: Int(len)) - } - - static func alignTo64(_ length: UInt) -> UInt { - (length + 63) & ~63 - } - - /// Allocates uninitialized memory, defaulting to 64 byte alignment. - /// - /// This memory must be de-allocated manually. It is not bound to a specific type. - /// - /// - Parameters: - /// - byteCount: A positive number of bytes to allocate. - /// - alignment: An integer power of two, to which this memory must be aligned. - /// - Returns: A pointer to a newly allocated but uninitialized region of memory. - private static func allocateArray( - byteCount: Int, - alignment: Int = 64 - ) -> UnsafeMutableRawPointer { - UnsafeMutableRawPointer.allocate( - byteCount: byteCount, - alignment: alignment - ) - } -} diff --git a/Sources/Arrow/ArrowBufferBuilder.swift b/Sources/Arrow/ArrowBufferBuilder.swift deleted file mode 100644 index 68cca2a..0000000 --- a/Sources/Arrow/ArrowBufferBuilder.swift +++ /dev/null @@ -1,511 +0,0 @@ -// Copyright 2025 The Apache Software Foundation -// Copyright 2025 The Columnar-Swift Contributors -// -// 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. - -import Foundation - -public protocol ByteConvertible { - func toData() -> Data -} - -extension String: ByteConvertible { - public func toData() -> Data { - Data(self.utf8) - } -} - -extension Data: ByteConvertible { - public func toData() -> Data { - self - } -} - -public protocol ArrowBufferBuilder { - associatedtype ItemType - var capacity: UInt { get } - var length: UInt { get } - var nullCount: UInt { get } - var offset: UInt { get } - init() - func append(_ newValue: ItemType?) - // func isNull(_ index: UInt) -> Bool - func resize(_ length: UInt) - func finish() -> [ArrowBuffer] -} - -public class BaseBufferBuilder { - var nulls: ArrowBuffer - public var offset: UInt = 0 - public var capacity: UInt { self.nulls.capacity } - public var length: UInt = 0 - public var nullCount: UInt = 0 - - init(_ nulls: ArrowBuffer) { - self.nulls = nulls - } - - public func isNull(_ index: UInt) -> Bool { - self.nulls.length == 0 - || BitUtility.isSet(index + self.offset, buffer: self.nulls) - } - - func resizeLength(_ data: ArrowBuffer, len: UInt = 0) -> UInt { - if len == 0 || len < data.length * 2 { - if data.length == 0 || data.length * 2 < ArrowBuffer.minLength { - return ArrowBuffer.minLength - } - return UInt(data.length * 2) - } - return UInt(len * 2) - } -} - -public class ValuesBufferBuilder: BaseBufferBuilder { - var values: ArrowBuffer - var stride: Int - public override var capacity: UInt { self.values.capacity } - - init( - values: ArrowBuffer, - nulls: ArrowBuffer, - stride: Int = MemoryLayout.stride - ) { - self.stride = stride - self.values = values - super.init(nulls) - } -} - -// TODO: look at potential for typed memory allocation -/// Builds buffers of fixed-width types. -public class FixedBufferBuilder: ValuesBufferBuilder, ArrowBufferBuilder -where T: Numeric { - public typealias ItemType = T - private let defaultVal: ItemType = 0 - - public required init() { - let values = ArrowBuffer.createBuffer(0, size: UInt(MemoryLayout.stride)) - let nulls = ArrowBuffer.createBuffer( - 0, - size: UInt(MemoryLayout.stride) - ) - super.init(values: values, nulls: nulls) - } - - public func append(_ newValue: ItemType?) { - let index = UInt(self.length) - let byteIndex = self.stride * Int(index) - self.length += 1 - if length > self.values.length { - self.resize(length) - } - - if let val = newValue { - BitUtility.setBit(index + self.offset, buffer: self.nulls) - self.values.rawPointer.advanced(by: byteIndex).storeBytes( - of: val, as: T.self) - } else { - self.nullCount += 1 - BitUtility.clearBit(index + self.offset, buffer: self.nulls) - self.values.rawPointer.advanced(by: byteIndex).storeBytes( - of: defaultVal, as: T.self) - } - } - - public func resize(_ length: UInt) { - if length > self.values.length { - let resizeLength = resizeLength(self.values) - var values = ArrowBuffer.createBuffer( - resizeLength, size: UInt(MemoryLayout.size)) - var nulls = ArrowBuffer.createBuffer( - resizeLength / 8 + 1, size: UInt(MemoryLayout.size)) - ArrowBuffer.copyCurrent( - self.values, to: &values, len: self.values.capacity) - ArrowBuffer.copyCurrent(self.nulls, to: &nulls, len: self.nulls.capacity) - self.values = values - self.nulls = nulls - } - } - - public func finish() -> [ArrowBuffer] { - let length = self.length - var values = ArrowBuffer.createBuffer( - length, - size: UInt(MemoryLayout.size) - ) - var nulls = ArrowBuffer.createBuffer( - length / 8 + 1, - size: UInt(MemoryLayout.size) - ) - ArrowBuffer.copyCurrent(self.values, to: &values, len: values.capacity) - ArrowBuffer.copyCurrent(self.nulls, to: &nulls, len: nulls.capacity) - return [nulls, values] - } -} - -public class BoolBufferBuilder: ValuesBufferBuilder, ArrowBufferBuilder { - public typealias ItemType = Bool - public required init() { - let values = ArrowBuffer.createBuffer( - 0, size: UInt(MemoryLayout.stride)) - let nulls = ArrowBuffer.createBuffer( - 0, size: UInt(MemoryLayout.stride)) - super.init(values: values, nulls: nulls) - } - - public func append(_ newValue: ItemType?) { - let index = UInt(self.length) - self.length += 1 - if (length / 8) > self.values.length { - self.resize(length) - } - - if newValue != nil { - BitUtility.setBit(index + self.offset, buffer: self.nulls) - if newValue == true { - BitUtility.setBit(index + self.offset, buffer: self.values) - } else { - BitUtility.clearBit(index + self.offset, buffer: self.values) - } - - } else { - self.nullCount += 1 - BitUtility.clearBit(index + self.offset, buffer: self.nulls) - BitUtility.clearBit(index + self.offset, buffer: self.values) - } - } - - public func resize(_ length: UInt) { - if (length / 8) > self.values.length { - let resizeLength = resizeLength(self.values) - var values = ArrowBuffer.createBuffer( - resizeLength, size: UInt(MemoryLayout.size)) - var nulls = ArrowBuffer.createBuffer( - resizeLength, size: UInt(MemoryLayout.size)) - ArrowBuffer.copyCurrent( - self.values, to: &values, len: self.values.capacity) - ArrowBuffer.copyCurrent(self.nulls, to: &nulls, len: self.nulls.capacity) - self.values = values - self.nulls = nulls - } - } - - public func finish() -> [ArrowBuffer] { - let length = self.length - var values = ArrowBuffer.createBuffer( - length, size: UInt(MemoryLayout.size)) - var nulls = ArrowBuffer.createBuffer( - length, size: UInt(MemoryLayout.size)) - ArrowBuffer.copyCurrent(self.values, to: &values, len: values.capacity) - ArrowBuffer.copyCurrent(self.nulls, to: &nulls, len: nulls.capacity) - return [nulls, values] - } -} - -public class VariableBufferBuilder: ValuesBufferBuilder, - ArrowBufferBuilder -where T: ByteConvertible { - public typealias ItemType = T - var offsets: ArrowBuffer - let binaryStride = MemoryLayout.stride - - public required init() { - let values = ArrowBuffer.createBuffer(0, size: UInt(binaryStride)) - let nulls = ArrowBuffer.createBuffer(0, size: UInt(binaryStride)) - self.offsets = ArrowBuffer.createBuffer( - 0, size: UInt(MemoryLayout.stride)) - super.init(values: values, nulls: nulls, stride: binaryStride) - } - - public func append(_ newValue: ItemType?) { - let index = UInt(self.length) - self.length += 1 - let offsetIndex = MemoryLayout.stride * Int(index) - if self.length >= self.offsets.length { - self.resize(UInt(self.offsets.length + 1)) - } - var data: Data - var isNull = false - if let newValue { - data = newValue.toData() - } else { - var nullVal = 0 - isNull = true - data = Data(bytes: &nullVal, count: MemoryLayout.size) - } - var currentIndex: Int32 = 0 - var currentOffset: Int32 = Int32(data.count) - if index > 0 { - currentIndex = self.offsets.rawPointer.advanced(by: offsetIndex).load( - as: Int32.self) - currentOffset += currentIndex - if currentOffset > self.values.length { - self.valueResize(UInt(currentOffset)) - } - } - if isNull { - self.nullCount += 1 - BitUtility.clearBit(index + self.offset, buffer: self.nulls) - } else { - BitUtility.setBit(index + self.offset, buffer: self.nulls) - } - data.withUnsafeBytes { buffer in - UnsafeMutableRawBufferPointer( - start: self.values.rawPointer.advanced(by: Int(currentIndex)), - count: data.count - ).copyBytes(from: buffer) - } - self.offsets.rawPointer.advanced( - by: offsetIndex + MemoryLayout.stride - ) - .storeBytes(of: currentOffset, as: Int32.self) - } - - public func valueResize(_ length: UInt) { - if length > self.values.length { - let resizeLength = resizeLength(self.values, len: length) - var values = ArrowBuffer.createBuffer( - resizeLength, size: UInt(MemoryLayout.size)) - ArrowBuffer.copyCurrent( - self.values, to: &values, len: self.values.capacity) - self.values = values - } - } - - public func resize(_ length: UInt) { - if length > self.offsets.length { - let resizeLength = resizeLength(self.offsets, len: length) - var nulls = ArrowBuffer.createBuffer( - resizeLength / 8 + 1, size: UInt(MemoryLayout.size)) - var offsets = ArrowBuffer.createBuffer( - resizeLength, size: UInt(MemoryLayout.size)) - ArrowBuffer.copyCurrent(self.nulls, to: &nulls, len: self.nulls.capacity) - ArrowBuffer.copyCurrent( - self.offsets, to: &offsets, len: self.offsets.capacity) - self.nulls = nulls - self.offsets = offsets - } - } - - public func finish() -> [ArrowBuffer] { - let length = self.length - var values = ArrowBuffer.createBuffer( - self.values.length, size: UInt(MemoryLayout.size)) - var nulls = ArrowBuffer.createBuffer( - length / 8 + 1, size: UInt(MemoryLayout.size)) - var offsets = ArrowBuffer.createBuffer( - length, size: UInt(MemoryLayout.size)) - ArrowBuffer.copyCurrent(self.values, to: &values, len: values.capacity) - ArrowBuffer.copyCurrent(self.nulls, to: &nulls, len: nulls.capacity) - ArrowBuffer.copyCurrent(self.offsets, to: &offsets, len: offsets.capacity) - return [nulls, offsets, values] - } -} - -public class AbstractWrapperBufferBuilder: ArrowBufferBuilder -where U: Numeric { - public typealias ItemType = T - public var capacity: UInt { self.bufferBuilder.capacity } - public var length: UInt { self.bufferBuilder.length } - public var nullCount: UInt { self.bufferBuilder.nullCount } - public var offset: UInt { self.bufferBuilder.offset } - let bufferBuilder: FixedBufferBuilder - public required init() { - self.bufferBuilder = FixedBufferBuilder() - } - - public func append(_ newValue: ItemType?) { - fatalError("Method is not implemented") - } - - // public func isNull(_ index: UInt) -> Bool { - // self.bufferBuilder.isNull(index) - // } - - public func resize(_ length: UInt) { - self.bufferBuilder.resize(length) - } - - public func finish() -> [ArrowBuffer] { - self.bufferBuilder.finish() - } -} - -public class Date32BufferBuilder: AbstractWrapperBufferBuilder { - public override func append(_ newValue: ItemType?) { - if let val = newValue { - let daysSinceEpoch = Int32(val.timeIntervalSince1970 / 86400) - self.bufferBuilder.append(daysSinceEpoch) - } else { - self.bufferBuilder.append(nil) - } - } -} - -public class Date64BufferBuilder: AbstractWrapperBufferBuilder { - public override func append(_ newValue: ItemType?) { - if let val = newValue { - let daysSinceEpoch = Int64(val.timeIntervalSince1970 * 1000) - self.bufferBuilder.append(daysSinceEpoch) - } else { - self.bufferBuilder.append(nil) - } - } -} - -public final class StructBufferBuilder: BaseBufferBuilder, ArrowBufferBuilder { - public typealias ItemType = [Any?] - var info: ArrowType? - public init() { - let nulls = ArrowBuffer.createBuffer( - 0, size: UInt(MemoryLayout.stride)) - super.init(nulls) - } - - public func initializeTypeInfo(_ fields: [ArrowField]) { - info = .strct(fields) - } - - public func append(_ newValue: [Any?]?) { - let index = UInt(self.length) - self.length += 1 - if self.length > self.nulls.length { - self.resize(length) - } - - if newValue != nil { - BitUtility.setBit(index + self.offset, buffer: self.nulls) - } else { - self.nullCount += 1 - BitUtility.clearBit(index + self.offset, buffer: self.nulls) - } - } - - public func resize(_ length: UInt) { - if length > self.nulls.length { - let resizeLength = resizeLength(self.nulls) - var nulls = ArrowBuffer.createBuffer( - resizeLength / 8 + 1, size: UInt(MemoryLayout.size)) - ArrowBuffer.copyCurrent(self.nulls, to: &nulls, len: self.nulls.capacity) - self.nulls = nulls - } - } - - public func finish() -> [ArrowBuffer] { - let length = self.length - var nulls = ArrowBuffer.createBuffer( - length / 8 + 1, - size: UInt(MemoryLayout.size) - ) - ArrowBuffer.copyCurrent(self.nulls, to: &nulls, len: nulls.capacity) - return [nulls] - } -} - -public class ListBufferBuilder: BaseBufferBuilder, ArrowBufferBuilder { - public typealias ItemType = [Any?] - var offsets: ArrowBuffer - - public required init() { - self.offsets = ArrowBuffer.createBuffer( - 1, - size: UInt(MemoryLayout.stride) - ) - let nulls = ArrowBuffer.createBuffer( - 0, - size: UInt(MemoryLayout.stride) - ) - super.init(nulls) - self.offsets.rawPointer.storeBytes(of: Int32(0), as: Int32.self) - } - - public func append(_ count: Int) { - let index = UInt(self.length) - self.length += 1 - - if length >= self.offsets.length { - self.resize(length + 1) - } - - let offsetIndex = Int(index) * MemoryLayout.stride - let currentOffset = self.offsets.rawPointer.advanced(by: offsetIndex).load( - as: Int32.self) - - BitUtility.setBit(index + self.offset, buffer: self.nulls) - let newOffset = currentOffset + Int32(count) - self.offsets.rawPointer.advanced( - by: offsetIndex + MemoryLayout.stride - ).storeBytes( - of: newOffset, as: Int32.self) - } - - public func append(_ newValue: [Any?]?) { - let index = UInt(self.length) - self.length += 1 - - if self.length >= self.offsets.length { - self.resize(self.length + 1) - } - - let offsetIndex = Int(index) * MemoryLayout.stride - let currentOffset = self.offsets.rawPointer.advanced(by: offsetIndex).load( - as: Int32.self) - - if let vals = newValue { - BitUtility.setBit(index + self.offset, buffer: self.nulls) - let newOffset = currentOffset + Int32(vals.count) - self.offsets.rawPointer.advanced( - by: offsetIndex + MemoryLayout.stride - ).storeBytes( - of: newOffset, as: Int32.self) - } else { - self.nullCount += 1 - BitUtility.clearBit(index + self.offset, buffer: self.nulls) - self.offsets.rawPointer.advanced( - by: offsetIndex + MemoryLayout.stride - ).storeBytes( - of: currentOffset, as: Int32.self) - } - } - - public override func isNull(_ index: UInt) -> Bool { - !BitUtility.isSet(index + self.offset, buffer: self.nulls) - } - - public func resize(_ length: UInt) { - if length > self.offsets.length { - let resizeLength = resizeLength(self.offsets) - var offsets = ArrowBuffer.createBuffer( - resizeLength, size: UInt(MemoryLayout.size)) - var nulls = ArrowBuffer.createBuffer( - resizeLength / 8 + 1, size: UInt(MemoryLayout.size)) - ArrowBuffer.copyCurrent( - self.offsets, to: &offsets, len: self.offsets.capacity) - ArrowBuffer.copyCurrent(self.nulls, to: &nulls, len: self.nulls.capacity) - self.offsets = offsets - self.nulls = nulls - } - } - - public func finish() -> [ArrowBuffer] { - let length = self.length - var nulls = ArrowBuffer.createBuffer( - length / 8 + 1, size: UInt(MemoryLayout.size)) - var offsets = ArrowBuffer.createBuffer( - length + 1, size: UInt(MemoryLayout.size)) - ArrowBuffer.copyCurrent(self.nulls, to: &nulls, len: nulls.capacity) - ArrowBuffer.copyCurrent(self.offsets, to: &offsets, len: offsets.capacity) - return [nulls, offsets] - } -} diff --git a/Sources/Arrow/ArrowCExporter.swift b/Sources/Arrow/ArrowCExporter.swift deleted file mode 100644 index aa8cba6..0000000 --- a/Sources/Arrow/ArrowCExporter.swift +++ /dev/null @@ -1,165 +0,0 @@ -// Copyright 2025 The Apache Software Foundation -// Copyright 2025 The Columnar Swift Contributors -// -// 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. - -import ArrowC -import Atomics -import Foundation - -// The memory used by UnsafeAtomic is not automatically -// reclaimed. Since this value is initialized once -// and used until the program/app is closed it's -// memory will be released on program/app exit -let exportDataCounter: UnsafeAtomic = .create(0) - -public class ArrowCExporter { - private class ExportData { - let id: Int - @MainActor - init() { - id = exportDataCounter.loadThenWrappingIncrement(ordering: .relaxed) - ArrowCExporter.exportedData[id] = self - } - } - - private class ExportSchema: ExportData { - public let arrowTypeNameCstr: UnsafePointer - public let nameCstr: UnsafePointer - private let arrowType: ArrowType - private let name: String - private let arrowTypeName: String - @MainActor - init(_ arrowType: ArrowType, name: String = "") throws { - self.arrowType = arrowType - // keeping the name str to ensure the cstring buffer remains valid - self.name = name - self.arrowTypeName = try arrowType.cDataFormatId - guard let nameCstr = (self.name as NSString).utf8String else { - throw ArrowError.runtimeError("Failed to convert name to C string") - } - self.nameCstr = nameCstr - guard let typeNameCstr = (self.arrowTypeName as NSString).utf8String - else { - throw ArrowError.runtimeError("Failed to convert type name to C string") - } - self.arrowTypeNameCstr = typeNameCstr - super.init() - } - } - - private class ExportArray: ExportData { - private let arrowData: ArrowData - private(set) var data: [UnsafeRawPointer?] = [] - private(set) var buffers: UnsafeMutablePointer - @MainActor - init(_ arrowData: ArrowData) { - // keep a reference to the ArrowData - // obj so the memory doesn't get - // deallocated - self.arrowData = arrowData - // for arrowBuffer in arrowData.buffers { - // self.data.append(arrowBuffer.rawPointer) - // } - for pointer in arrowData.data { - self.data.append(pointer) - } - - self.buffers = UnsafeMutablePointer.allocate( - capacity: self.data.count) - self.buffers.initialize(from: &self.data, count: self.data.count) - super.init() - } - - deinit { - self.buffers.deinitialize(count: self.data.count) - self.buffers.deallocate() - } - } - - @MainActor private static var exportedData: [Int: ExportData] = [:] - public init() {} - - @MainActor public func exportType( - _ cSchema: inout ArrowC.ArrowSchema, arrowType: ArrowType, name: String = "" - ) -> Result { - do { - let exportSchema = try ExportSchema(arrowType, name: name) - cSchema.format = exportSchema.arrowTypeNameCstr - cSchema.name = exportSchema.nameCstr - cSchema.private_data = - UnsafeMutableRawPointer( - mutating: UnsafeRawPointer(bitPattern: exportSchema.id)) - cSchema.release = { (data: UnsafeMutablePointer?) in - guard let data else { - fatalError("Release called with nil Arrow schema pointer.") - } - let arraySchema = data.pointee - let exportId = Int(bitPattern: arraySchema.private_data) - guard ArrowCExporter.exportedData[exportId] != nil else { - fatalError("Export schema not found with id \(exportId)") - } - - // the data associated with this exportSchema object - // which includes the C strings for the format and name - // be deallocated upon removal - ArrowCExporter.exportedData.removeValue(forKey: exportId) - ArrowC.ArrowSwiftClearReleaseSchema(data) - } - } catch { - return .failure(.unknownError("\(error)")) - } - return .success(true) - } - - @MainActor public func exportField( - _ schema: inout ArrowC.ArrowSchema, field: ArrowField - ) -> Result { - exportType(&schema, arrowType: field.type, name: field.name) - } - - @MainActor public func exportArray( - _ cArray: inout ArrowC.ArrowArray, arrowData: ArrowData - ) { - let exportArray = ExportArray(arrowData) - cArray.buffers = exportArray.buffers - cArray.length = Int64(arrowData.length) - cArray.null_count = Int64(arrowData.nullCount) - cArray.n_buffers = Int64(arrowData.bufferCount) - // Swift Arrow does not currently support children or dictionaries - // This will need to be updated once support has been added - cArray.n_children = 0 - cArray.children = nil - cArray.dictionary = nil - cArray.private_data = UnsafeMutableRawPointer( - mutating: UnsafeRawPointer(bitPattern: exportArray.id) - ) - cArray.release = { (data: UnsafeMutablePointer?) in - guard let data else { - fatalError("Release called with nil ArrowArray pointer.") - } - let arrayData = data.pointee - let exportId = Int(bitPattern: arrayData.private_data) - guard ArrowCExporter.exportedData[exportId] != nil else { - fatalError("Export data not found with id \(exportId)") - } - - // the data associated with this exportArray object - // which includes the entire arrowData object - // and the buffers UnsafeMutablePointer[] will - // be deallocated upon removal - ArrowCExporter.exportedData.removeValue(forKey: exportId) - ArrowC.ArrowSwiftClearReleaseArray(data) - } - } -} diff --git a/Sources/Arrow/ArrowCImporter.swift b/Sources/Arrow/ArrowCImporter.swift deleted file mode 100644 index ae47de2..0000000 --- a/Sources/Arrow/ArrowCImporter.swift +++ /dev/null @@ -1,202 +0,0 @@ -// Copyright 2025 The Apache Software Foundation -// Copyright 2025 The Columnar Swift Contributors -// -// 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. - -import ArrowC -import Foundation - -public class ArrowCImporter { - private func appendToBuffer( - _ cBuffer: UnsafeRawPointer?, - arrowBuffers: inout [ArrowBuffer], - length: UInt - ) throws(ArrowError) { - if cBuffer == nil { - // Some implementations may have null buffers. - // The empty buffers are positional placeholders. - arrowBuffers.append(ArrowBuffer.createEmptyBuffer()) - return - } - guard let pointer = UnsafeMutableRawPointer(mutating: cBuffer) else { - throw .invalid("Failed to obtain a pointer to C buffer.") - } - arrowBuffers.append( - ArrowBuffer( - length: length, - capacity: length, - rawPointer: pointer, - isMemoryOwner: false - ) - ) - } - - public init() {} - - public func importType(_ cArrow: String, name: String = "") -> Result< - ArrowField, ArrowError - > { - do { - let type = try ArrowType.fromCDataFormatId(cArrow) - return .success(ArrowField(name: name, dataType: type, isNullable: true)) - } catch { - return .failure( - .invalid("Error occurred while attempting to import type: \(error)")) - } - } - - public func importField(_ cSchema: ArrowC.ArrowSchema) -> Result< - ArrowField, ArrowError - > { - if cSchema.n_children > 0 { - ArrowCImporter.release(cSchema) - return .failure(.invalid("Children currently not supported")) - } else if cSchema.dictionary != nil { - ArrowCImporter.release(cSchema) - return .failure(.invalid("Dictinoary types currently not supported")) - } - - switch importType( - String(cString: cSchema.format), name: String(cString: cSchema.name)) - { - case .success(let field): - ArrowCImporter.release(cSchema) - return .success(field) - case .failure(let err): - ArrowCImporter.release(cSchema) - return .failure(err) - } - } - - public func importArray( - _ cArray: UnsafePointer, - arrowType: ArrowType, - isNullable: Bool = false - ) -> Result { - let arrowField = ArrowField( - name: "", dataType: arrowType, isNullable: isNullable) - return importArray(cArray, arrowField: arrowField) - } - - public func importArray( - _ cArrayPtr: UnsafePointer, - arrowField: ArrowField - ) -> Result { - let cArray = cArrayPtr.pointee - if cArray.null_count < 0 { - ArrowCImporter.release(cArrayPtr) - return .failure(.invalid("Uncomputed null count is not supported")) - } else if cArray.n_children > 0 { - ArrowCImporter.release(cArrayPtr) - return .failure(.invalid("Children currently not supported")) - } else if cArray.dictionary != nil { - ArrowCImporter.release(cArrayPtr) - return .failure(.invalid("Dictionary types currently not supported")) - } else if cArray.offset != 0 { - ArrowCImporter.release(cArrayPtr) - return .failure( - .invalid("Offset of 0 is required but found offset: \(cArray.offset)")) - } - - let arrowType = arrowField.type - let length = UInt(cArray.length) - let nullCount = UInt(cArray.null_count) - var arrowBuffers: [ArrowBuffer] = [] - - if cArray.n_buffers > 0 { - if cArray.buffers == nil { - ArrowCImporter.release(cArrayPtr) - return .failure(.invalid("C array buffers is nil")) - } - - do { - if arrowType.isVariable { - if cArray.n_buffers != 3 { - ArrowCImporter.release(cArrayPtr) - return .failure( - .invalid( - "Variable buffer count expected 3 but found \(cArray.n_buffers)" - )) - } - try appendToBuffer( - cArray.buffers[0], - arrowBuffers: &arrowBuffers, - length: UInt(ceil(Double(length) / 8)) - ) - try appendToBuffer( - cArray.buffers[1], - arrowBuffers: &arrowBuffers, - length: length - ) - guard let buffer1 = cArray.buffers[1] else { - return .failure(.invalid("C array buffer is nil")) - } - let lastOffsetLength = - buffer1 - .advanced(by: Int(length) * MemoryLayout.stride) - .load(as: Int32.self) - try appendToBuffer( - cArray.buffers[2], - arrowBuffers: &arrowBuffers, - length: UInt(lastOffsetLength) - ) - } else { - - if cArray.n_buffers != 2 { - ArrowCImporter.release(cArrayPtr) - return .failure( - .invalid("Expected buffer count 2 but found \(cArray.n_buffers)")) - } - - try appendToBuffer( - cArray.buffers[0], arrowBuffers: &arrowBuffers, - length: UInt(ceil(Double(length) / 8))) - try appendToBuffer( - cArray.buffers[1], arrowBuffers: &arrowBuffers, length: length) - } - } catch { - return .failure(error) - } - } - - switch makeArrayHolder( - arrowField, buffers: arrowBuffers, - nullCount: nullCount, children: nil, rbLength: 0) - { - case .success(let holder): - holder.setCArrayPtr(cArrayPtr) - return .success(holder) - case .failure(let err): - ArrowCImporter.release(cArrayPtr) - return .failure(err) - } - } - - public static func release(_ cArrayPtr: UnsafePointer) { - if cArrayPtr.pointee.release != nil { - let cSchemaMutablePtr = UnsafeMutablePointer( - mutating: cArrayPtr - ) - cArrayPtr.pointee.release(cSchemaMutablePtr) - } - } - - public static func release(_ cSchema: ArrowC.ArrowSchema) { - if cSchema.release != nil { - let cSchemaPtr = UnsafeMutablePointer.allocate( - capacity: 1) - cSchemaPtr.initialize(to: cSchema) - cSchema.release(cSchemaPtr) - } - } -} diff --git a/Sources/Arrow/ArrowData.swift b/Sources/Arrow/ArrowData.swift deleted file mode 100644 index bda48e4..0000000 --- a/Sources/Arrow/ArrowData.swift +++ /dev/null @@ -1,140 +0,0 @@ -// Copyright 2025 The Apache Software Foundation -// -// 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. - -import Foundation - -public struct ArrowData { - - // FIXME: Remove - public var bufferData: [Data] { - buffers.map { buffer in - var data = Data() - buffer.append(to: &data) - return data - } - } - - // FIXME: Remove - public var bufferDataSizes: [Int] { - buffers.map { Int($0.capacity) } - } - - // FIXME: Remove - public var data: [UnsafeMutableRawPointer] { - buffers.map { $0.rawPointer } - } - - // FIXME: Remove - public var bufferCount: Int { - buffers.count - } - - // TODO: Typed accessors - migration - var offsets: OffsetsBuffer { - if !type.isVariable && !type.isNested { - fatalError() - } - return ArrowBufferBackedOffsets(buffers[1]) - } - - // TODO: this should replace nullBuffer - var nulls: BitPackedNullBuffer { - let buffer = buffers[0] - let pointer = buffer.rawPointer.assumingMemoryBound(to: UInt8.self) - return BitPackedNullBuffer( - length: Int(buffer.length), capacity: 0, valueCount: -1, nullCount: -1, - ownsMemory: false, - buffer: pointer) - } - - public let type: ArrowType - public let children: [ArrowData] - public let nullCount: UInt - public let length: UInt - - private let nullBuffer: ArrowBuffer - // FIXME: Remove - private let buffers: [ArrowBuffer] - - init( - _ arrowType: ArrowType, - buffers: [ArrowBuffer], - nullCount: UInt - ) { - self.init( - arrowType, buffers: buffers, - children: [ArrowData](), - nullCount: nullCount, - length: buffers[1].length - ) - } - - init( - _ arrowType: ArrowType, - buffers: [ArrowBuffer], - children: [ArrowData], - nullCount: UInt, - length: UInt - ) { - self.type = arrowType - self.buffers = buffers - self.children = children - self.nullCount = nullCount - self.length = length - self.nullBuffer = buffers[0] - } - - // TODO: Temporary while removing ArrowBuffer - public func load(at index: UInt) -> T where T: BitwiseCopyable { - let valueType = T.self - let byteOffset = type.getStride() * Int(index) - let milliseconds = buffers[1].rawPointer.advanced( - by: byteOffset - ).load(as: valueType) - return milliseconds - } - - // TODO: Temporary while removing ArrowBuffer - func loadVariable( - at startIndex: Int, - arrayLength: Int - ) -> T where T: VariableLength { - let values = buffers[2] - let rawPointer = values.rawPointer.advanced(by: startIndex) - .bindMemory(to: UInt8.self, capacity: arrayLength) - let buffer = UnsafeBufferPointer( - start: rawPointer, count: arrayLength) - return T(buffer) - } - - // TODO: Temporary while removing ArrowBuffer - public func isNull(_ at: UInt) -> Bool { - let a = nulls.length > 0 && !nulls.isSet(Int(at)) - let b = nullBuffer.length > 0 && !BitUtility.isSet(at, buffer: nullBuffer) - if nulls.length != nullBuffer.length { - fatalError("Check new null handling") - } - if a != b { - fatalError("Check new null handling") - } - return a - } - - // TODO: Temporary while removing ArrowBuffer - func isNullValue(at index: UInt) -> Bool { - let valueBuffer = buffers[1] - return BitUtility.isSet(index, buffer: valueBuffer) - } - -} diff --git a/Sources/Arrow/ArrowDecoder.swift b/Sources/Arrow/ArrowDecoder.swift deleted file mode 100644 index e902e27..0000000 --- a/Sources/Arrow/ArrowDecoder.swift +++ /dev/null @@ -1,481 +0,0 @@ -// Copyright 2025 The Apache Software Foundation -// -// 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. - -import Foundation - -public class ArrowDecoder: Decoder { - var rbIndex: UInt = 0 - var singleRBCol: Int = 0 - public var codingPath: [CodingKey] = [] - public var userInfo: [CodingUserInfoKey: Any] = [:] - public let rb: RecordBatchX - public let nameToCol: [String: AnyArrowArray] - public let columns: [AnyArrowArray] - - public init(_ decoder: ArrowDecoder) { - self.userInfo = decoder.userInfo - self.codingPath = decoder.codingPath - self.rb = decoder.rb - self.columns = decoder.columns - self.nameToCol = decoder.nameToCol - self.rbIndex = decoder.rbIndex - } - - public init(_ rb: RecordBatchX) { - self.rb = rb - var colMapping: [String: AnyArrowArray] = [:] - var columns: [AnyArrowArray] = [] - for index in 0..( - _ type: [T: U].Type - ) throws -> [T: U] { - var output: [T: U] = [:] - if rb.columnCount != 2 { - throw ArrowError.invalid( - "RecordBatch column count of 2 is required to decode to map" - ) - } - for index in 0..(_ type: T.Type) throws -> [T] { - var output: [T] = [] - for index in 0..( - keyedBy type: Key.Type - ) -> KeyedDecodingContainer where Key: CodingKey { - let container = ArrowKeyedDecoding(self, codingPath: codingPath) - return KeyedDecodingContainer(container) - } - - public func unkeyedContainer() -> UnkeyedDecodingContainer { - ArrowUnkeyedDecoding(self, codingPath: codingPath) - } - - public func singleValueContainer() -> SingleValueDecodingContainer { - ArrowSingleValueDecoding(self, codingPath: codingPath) - } - - func getCol(_ name: String) throws -> AnyArrowArray { - guard let col = self.nameToCol[name] else { - throw ArrowError.invalid("Column for key \"\(name)\" not found") - } - - return col - } - - func getCol(_ index: Int) throws -> AnyArrowArray { - if index >= self.columns.count { - throw ArrowError.outOfBounds(index: Int64(index)) - } - - return self.columns[index] - } - - func doDecode(_ key: CodingKey) throws -> T? { - let array: AnyArrowArray = try self.getCol(key.stringValue) - return array.asAny(self.rbIndex) as? T - } - - func doDecode(_ col: Int) throws -> T? { - let array: AnyArrowArray = try self.getCol(col) - return array.asAny(self.rbIndex) as? T - } - - func isNull(_ key: CodingKey) throws -> Bool { - let array: AnyArrowArray = try self.getCol(key.stringValue) - return array.asAny(self.rbIndex) == nil - } - - func isNull(_ col: Int) throws -> Bool { - let array: AnyArrowArray = try self.getCol(col) - return array.asAny(self.rbIndex) == nil - } -} - -private struct ArrowUnkeyedDecoding: UnkeyedDecodingContainer { - var codingPath: [CodingKey] - var count: Int? = 0 - var isAtEnd: Bool = false - var currentIndex: Int = 0 - let decoder: ArrowDecoder - - init(_ decoder: ArrowDecoder, codingPath: [CodingKey]) { - self.decoder = decoder - self.codingPath = codingPath - self.count = self.decoder.columns.count - } - - mutating func increment() { - self.currentIndex += 1 - self.isAtEnd = self.currentIndex >= self.count ?? 0 - } - - mutating func decodeNil() throws -> Bool { - defer { increment() } - return try self.decoder.isNull(self.currentIndex) - } - - mutating func decode(_ type: T.Type) throws -> T where T: Decodable { - if type == Int8?.self || type == Int16?.self || type == Int32?.self - || type == Int64?.self - || type == UInt8?.self || type == UInt16?.self || type == UInt32?.self - || type == UInt64?.self - || type == String?.self || type == Double?.self || type == Float?.self - || type == Date?.self - || type == Bool?.self || type == Bool.self || type == Int8.self - || type == Int16.self - || type == Int32.self || type == Int64.self || type == UInt8.self - || type == UInt16.self - || type == UInt32.self || type == UInt64.self || type == String.self - || type == Double.self - || type == Float.self || type == Date.self - { - defer { increment() } - guard let value: T = try self.decoder.doDecode(self.currentIndex) else { - throw ArrowError.invalid("Failed to decode value for \(type)") - } - return value - } else { - throw ArrowError.invalid("Type \(type) is currently not supported") - } - } - - func nestedContainer( - keyedBy type: NestedKey.Type - ) throws -> KeyedDecodingContainer where NestedKey: CodingKey { - throw ArrowError.invalid("Nested decoding is currently not supported.") - } - - func nestedUnkeyedContainer() throws -> UnkeyedDecodingContainer { - throw ArrowError.invalid("Nested decoding is currently not supported.") - } - - func superDecoder() throws -> Decoder { - throw ArrowError.invalid("super decoding is currently not supported.") - } -} - -private struct ArrowKeyedDecoding: - KeyedDecodingContainerProtocol -{ - var codingPath: [CodingKey] = [] - var allKeys: [Key] = [] - let decoder: ArrowDecoder - - init(_ decoder: ArrowDecoder, codingPath: [CodingKey]) { - self.decoder = decoder - self.codingPath = codingPath - } - - func contains(_ key: Key) -> Bool { - self.decoder.nameToCol.keys.contains(key.stringValue) - } - - func decodeNil(forKey key: Key) throws -> Bool { - try self.decoder.isNull(key) - } - - func decode(_ type: Bool.Type, forKey key: Key) throws -> Bool { - guard let value: Bool = try self.decoder.doDecode(key) else { - throw ArrowError.invalid("Failed to decode \(type) for key \(key)") - } - return value - } - - func decode(_ type: String.Type, forKey key: Key) throws -> String { - guard let value: String = try self.decoder.doDecode(key) else { - throw ArrowError.invalid("Failed to decode \(type) for key \(key)") - } - return value - } - - func decode(_ type: Double.Type, forKey key: Key) throws -> Double { - guard let value: Double = try self.decoder.doDecode(key) else { - throw ArrowError.invalid("Failed to decode \(type) for key \(key)") - } - return value - } - - func decode(_ type: Float.Type, forKey key: Key) throws -> Float { - guard let value: Float = try self.decoder.doDecode(key) else { - throw ArrowError.invalid("Failed to decode \(type) for key \(key)") - } - return value - } - - func decode(_ type: Int.Type, forKey key: Key) throws -> Int { - throw ArrowError.invalid( - "Int type is not supported (please use Int8, Int16, Int32 or Int64)") - } - - func decode(_ type: Int8.Type, forKey key: Key) throws -> Int8 { - guard let value: Int8 = try self.decoder.doDecode(key) else { - throw ArrowError.invalid("Failed to decode \(type) for key \(key)") - } - return value - } - - func decode(_ type: Int16.Type, forKey key: Key) throws -> Int16 { - guard let value: Int16 = try self.decoder.doDecode(key) else { - throw ArrowError.invalid("Failed to decode \(type) for key \(key)") - } - return value - } - - func decode(_ type: Int32.Type, forKey key: Key) throws -> Int32 { - guard let value: Int32 = try self.decoder.doDecode(key) else { - throw ArrowError.invalid("Failed to decode \(type) for key \(key)") - } - return value - } - - func decode(_ type: Int64.Type, forKey key: Key) throws -> Int64 { - guard let value: Int64 = try self.decoder.doDecode(key) else { - throw ArrowError.invalid("Failed to decode \(type) for key \(key)") - } - return value - } - - func decode(_ type: UInt.Type, forKey key: Key) throws -> UInt { - throw ArrowError.invalid( - "UInt type is not supported (please use UInt8, UInt16, UInt32 or UInt64)") - } - - func decode(_ type: UInt8.Type, forKey key: Key) throws -> UInt8 { - guard let value: UInt8 = try self.decoder.doDecode(key) else { - throw ArrowError.invalid("Failed to decode \(type) for key \(key)") - } - return value - } - - func decode(_ type: UInt16.Type, forKey key: Key) throws -> UInt16 { - guard let value: UInt16 = try self.decoder.doDecode(key) else { - throw ArrowError.invalid("Failed to decode \(type) for key \(key)") - } - return value - } - - func decode(_ type: UInt32.Type, forKey key: Key) throws -> UInt32 { - guard let value: UInt32 = try self.decoder.doDecode(key) else { - throw ArrowError.invalid("Failed to decode \(type) for key \(key)") - } - return value - } - - func decode(_ type: UInt64.Type, forKey key: Key) throws -> UInt64 { - guard let value: UInt64 = try self.decoder.doDecode(key) else { - throw ArrowError.invalid("Failed to decode \(type) for key \(key)") - } - return value - } - - func decode(_ type: T.Type, forKey key: Key) throws -> T - where T: Decodable { - if ArrowArrayBuilders.isValidBuilderType(type) || type == Date.self { - guard let value: T = try self.decoder.doDecode(key) else { - throw ArrowError.invalid("Failed to decode \(type) for key \(key)") - } - return value - } else { - throw ArrowError.invalid("Type \(type) is currently not supported") - } - } - - func nestedContainer( - keyedBy type: NestedKey.Type, - forKey key: Key - ) throws -> KeyedDecodingContainer where NestedKey: CodingKey { - throw ArrowError.invalid("Nested decoding is currently not supported.") - } - - func nestedUnkeyedContainer(forKey key: Key) throws - -> UnkeyedDecodingContainer - { - throw ArrowError.invalid("Nested decoding is currently not supported.") - } - - func superDecoder() throws -> Decoder { - throw ArrowError.invalid("super decoding is currently not supported.") - } - - func superDecoder(forKey key: Key) throws -> Decoder { - throw ArrowError.invalid("super decoding is currently not supported.") - } -} - -private struct ArrowSingleValueDecoding: SingleValueDecodingContainer { - var codingPath: [CodingKey] = [] - let decoder: ArrowDecoder - - init(_ decoder: ArrowDecoder, codingPath: [CodingKey]) { - self.decoder = decoder - self.codingPath = codingPath - } - - func decodeNil() -> Bool { - do { - return try self.decoder.isNull(self.decoder.singleRBCol) - } catch { - return false - } - } - - func decode(_ type: Bool.Type) throws -> Bool { - guard let value: Bool = try self.decoder.doDecode(self.decoder.singleRBCol) - else { - throw ArrowError.invalid("Failed to decode \(type)") - } - return value - } - - func decode(_ type: String.Type) throws -> String { - guard - let value: String = try self.decoder.doDecode(self.decoder.singleRBCol) - else { - throw ArrowError.invalid("Failed to decode \(type)") - } - return value - } - - func decode(_ type: Double.Type) throws -> Double { - guard - let value: Double = try self.decoder.doDecode(self.decoder.singleRBCol) - else { - throw ArrowError.invalid("Failed to decode \(type)") - } - return value - } - - func decode(_ type: Float.Type) throws -> Float { - guard let value: Float = try self.decoder.doDecode(self.decoder.singleRBCol) - else { - throw ArrowError.invalid("Failed to decode \(type)") - } - return value - } - - func decode(_ type: Int.Type) throws -> Int { - throw ArrowError.invalid( - "Int type is not supported (please use Int8, Int16, Int32 or Int64)") - } - - func decode(_ type: Int8.Type) throws -> Int8 { - guard let value: Int8 = try self.decoder.doDecode(self.decoder.singleRBCol) - else { - throw ArrowError.invalid("Failed to decode \(type)") - } - return value - } - - func decode(_ type: Int16.Type) throws -> Int16 { - guard let value: Int16 = try self.decoder.doDecode(self.decoder.singleRBCol) - else { - throw ArrowError.invalid("Failed to decode \(type)") - } - return value - } - - func decode(_ type: Int32.Type) throws -> Int32 { - guard let value: Int32 = try self.decoder.doDecode(self.decoder.singleRBCol) - else { - throw ArrowError.invalid("Failed to decode \(type)") - } - return value - } - - func decode(_ type: Int64.Type) throws -> Int64 { - guard let value: Int64 = try self.decoder.doDecode(self.decoder.singleRBCol) - else { - throw ArrowError.invalid("Failed to decode \(type)") - } - return value - } - - func decode(_ type: UInt.Type) throws -> UInt { - throw ArrowError.invalid( - "UInt type is not supported (please use UInt8, UInt16, UInt32 or UInt64)") - } - - func decode(_ type: UInt8.Type) throws -> UInt8 { - guard let value: UInt8 = try self.decoder.doDecode(self.decoder.singleRBCol) - else { - throw ArrowError.invalid("Failed to decode \(type)") - } - return value - } - - func decode(_ type: UInt16.Type) throws -> UInt16 { - guard - let value: UInt16 = try self.decoder.doDecode(self.decoder.singleRBCol) - else { - throw ArrowError.invalid("Failed to decode \(type)") - } - return value - } - - func decode(_ type: UInt32.Type) throws -> UInt32 { - guard - let value: UInt32 = try self.decoder.doDecode(self.decoder.singleRBCol) - else { - throw ArrowError.invalid("Failed to decode \(type)") - } - return value - } - - func decode(_ type: UInt64.Type) throws -> UInt64 { - guard - let value: UInt64 = try self.decoder.doDecode(self.decoder.singleRBCol) - else { - throw ArrowError.invalid("Failed to decode \(type)") - } - return value - } - - func decode(_ type: T.Type) throws -> T where T: Decodable { - if ArrowArrayBuilders.isValidBuilderType(type) || type == Date.self { - guard let value: T = try self.decoder.doDecode(self.decoder.singleRBCol) - else { - throw ArrowError.invalid("Failed to decode \(type)") - } - return value - } else { - throw ArrowError.invalid("Type \(type) is currently not supported") - } - } -} diff --git a/Sources/Arrow/ArrowEncoder.swift b/Sources/Arrow/ArrowEncoder.swift deleted file mode 100644 index bf2dce6..0000000 --- a/Sources/Arrow/ArrowEncoder.swift +++ /dev/null @@ -1,470 +0,0 @@ -// Copyright 2025 The Apache Software Foundation -// -// 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. - -import Foundation - -public class ArrowEncoder: Encoder { - public private(set) var builders: [String: AnyArrowArrayBuilder] = [:] - private var byIndex: [String] = [] - public var codingPath: [CodingKey] = [] - public var userInfo: [CodingUserInfoKey: Any] = [:] - var errorMsg: String? - // This is used for Dictionary types. A dictionary type - // will give each key and value their own index so instead - // of having a 2 column RecordBatch you would have - // 2 * length(dictionary) column RecordBatch. Which would not - // be the expected output. - var modForIndex: Int? - - public init() {} - - public init(_ builders: [String: AnyArrowArrayBuilder], byIndex: [String]) { - self.builders = builders - self.byIndex = byIndex - } - - public static func encode(_ data: T) throws -> RecordBatchX { - let encoder = try loadEncoder(data) - try data.encode(to: encoder) - return try encoder.finish() - } - - public static func encode(_ rows: [T]) throws -> RecordBatchX? { - if rows.isEmpty { - return nil - } - - let encoder = try loadEncoder(rows[0]) - for row in rows { - try row.encode(to: encoder) - } - - return try encoder.finish() - } - - static func loadEncoder(_ data: T) throws -> ArrowEncoder { - // this will check if T is a simple built in type - // (UInt, Int, Int8, String, Date, etc...). - if ArrowArrayBuilders.isValidBuilderType(T.self) { - let builders = ["col0": try ArrowArrayBuilders.builder(for: T.self)] - return ArrowEncoder(builders, byIndex: ["col0"]) - } else { - let encoder = ArrowEncoder() - if data is [AnyHashable: Any] { - encoder.modForIndex = 2 - } - return encoder - } - } - - public func finish() throws(ArrowError) -> RecordBatchX { - try throwIfInvalid() - let batchBuilder = RecordBatchX.Builder() - for key in byIndex { - guard let builder = builders[key] else { - throw .invalid("Missing builder for \(key)") - } - batchBuilder.addColumn(key, arrowArray: try builder.toAnyArrowArray()) - } - return try batchBuilder.finish().get() - } - - public func container(keyedBy type: Key.Type) -> KeyedEncodingContainer< - Key - > - where Key: CodingKey { - var container = ArrowKeyedEncoding(self) - container.codingPath = codingPath - return KeyedEncodingContainer(container) - } - - public func unkeyedContainer() -> UnkeyedEncodingContainer { - ArrowUnkeyedEncoding(self, codingPath: self.codingPath) - } - - public func singleValueContainer() -> SingleValueEncodingContainer { - ArrowSingleValueEncoding(self, codingPath: codingPath) - } - - func doEncodeNil(key: CodingKey) throws { - try throwIfInvalid() - guard let builder = builders[key.stringValue] else { - throw ArrowError.invalid("Column not found for key: \(key)") - } - builder.appendAny(nil) - } - - // This is required by the keyed and unkeyed encoders as columns are - // added when the first row of the data is encoded. This is done due - // to limitations in the Swifts Mirror API (ex: it is unable to correctly - // find the type for String? in [Int: String?]) - @discardableResult - func ensureColumnExists( - _ value: T, - key: String - ) throws(ArrowError) -> AnyArrowArrayBuilder { - try throwIfInvalid() - if let builder = builders[key] { - return builder - } - let builder = try ArrowArrayBuilders.builder(for: T.self) - builders[key] = builder - byIndex.append(key) - return builder - } - - func getIndex(_ index: Int) -> Int { - if let mod = self.modForIndex { - return index % mod - } else { - return index - } - } - - func doEncodeNil(_ keyIndex: Int) throws(ArrowError) { - try throwIfInvalid() - let index = self.getIndex(keyIndex) - guard index < builders.count else { - throw ArrowError.outOfBounds(index: Int64(index)) - } - let key = byIndex[index] - guard let builder = builders[key] else { - throw .invalid("Missing builder for key: \(key)") - } - builder.appendAny(nil) - } - - func doEncode(_ value: T, key: CodingKey) throws(ArrowError) { - try throwIfInvalid() - let builder = try ensureColumnExists(value, key: key.stringValue) - builder.appendAny(value) - } - - func doEncode(_ value: T, keyIndex: Int) throws(ArrowError) { - try throwIfInvalid() - let index = self.getIndex(keyIndex) - if index > builders.count { - throw .outOfBounds(index: Int64(index)) - } - if index == builders.count { - try ensureColumnExists(value, key: "col\(index)") - } - let key = byIndex[index] - guard let builder = builders[key] else { - throw .invalid("Missing builder for key: \(key)") - } - builder.appendAny(value) - } - - func throwIfInvalid() throws(ArrowError) { - if let errorMsg = self.errorMsg { - throw ArrowError.invalid(errorMsg) - } - } -} - -private struct ArrowKeyedEncoding: - KeyedEncodingContainerProtocol -{ - var codingPath: [CodingKey] = [] - let encoder: ArrowEncoder - init(_ encoder: ArrowEncoder) { - self.encoder = encoder - } - - // If this method is called on row 0 and the encoder is - // lazily bulding holders then this will produce an error - // as this method does not know what the underlying type - // is for the column. This method is not called for - // nullable types (String?, Int32?, Date?) and the workaround - // for this issue would be to predefine the builders for the - // encoder. (I have only encoutered this issue when allowing - // nullable types at the encode func level which is currently - // not allowed) - mutating func encodeNil(forKey key: Key) throws { - try encoder.doEncodeNil(key: key) - } - - mutating func doEncodeIf(_ value: T?, forKey key: Key) throws { - if value == nil { - try encoder.ensureColumnExists(value, key: key.stringValue) - try encoder.doEncodeNil(key: key) - } else { - try encoder.doEncode(value, key: key) - } - } - - mutating func encode(_ value: Bool, forKey key: Key) throws { - try encoder.doEncode(value, key: key) - } - - mutating func encodeIfPresent(_ value: Bool?, forKey key: Key) throws { - try doEncodeIf(value, forKey: key) - } - - mutating func encode(_ value: String, forKey key: Key) throws { - try encoder.doEncode(value, key: key) - } - - mutating func encodeIfPresent(_ value: String?, forKey key: Key) throws { - try doEncodeIf(value, forKey: key) - } - - mutating func encode(_ value: Double, forKey key: Key) throws { - try encoder.doEncode(value, key: key) - } - - mutating func encodeIfPresent(_ value: Double?, forKey key: Key) throws { - try doEncodeIf(value, forKey: key) - } - - mutating func encode(_ value: Float, forKey key: Key) throws { - try encoder.doEncode(value, key: key) - } - - mutating func encodeIfPresent(_ value: Float?, forKey key: Key) throws { - try doEncodeIf(value, forKey: key) - } - - mutating func encode(_ value: Int, forKey key: Key) throws { - throw ArrowError.invalid( - "Int type is not supported (please use Int8, Int16, Int32 or Int64)") - } - - mutating func encodeIfPresent(_ value: Int?, forKey key: Key) throws { - throw ArrowError.invalid( - "Int type is not supported (please use Int8, Int16, Int32 or Int64)") - } - - mutating func encode(_ value: Int8, forKey key: Key) throws { - try encoder.doEncode(value, key: key) - } - - mutating func encodeIfPresent(_ value: Int8?, forKey key: Key) throws { - try doEncodeIf(value, forKey: key) - } - - mutating func encode(_ value: Int16, forKey key: Key) throws { - try encoder.doEncode(value, key: key) - } - - mutating func encodeIfPresent(_ value: Int16?, forKey key: Key) throws { - try doEncodeIf(value, forKey: key) - } - - mutating func encode(_ value: Int32, forKey key: Key) throws { - try encoder.doEncode(value, key: key) - } - - mutating func encodeIfPresent(_ value: Int32?, forKey key: Key) throws { - try doEncodeIf(value, forKey: key) - } - - mutating func encode(_ value: Int64, forKey key: Key) throws { - try encoder.doEncode(value, key: key) - } - - mutating func encodeIfPresent(_ value: Int64?, forKey key: Key) throws { - try doEncodeIf(value, forKey: key) - } - - mutating func encode(_ value: UInt, forKey key: Key) throws { - throw ArrowError.invalid( - "UInt type is not supported (please use UInt8, UInt16, UInt32 or UInt64)") - } - - mutating func encodeIfPresent(_ value: UInt?, forKey key: Key) throws { - throw ArrowError.invalid( - "UInt type is not supported (please use UInt8, UInt16, UInt32 or UInt64)") - } - - mutating func encode(_ value: UInt8, forKey key: Key) throws { - try encoder.doEncode(value, key: key) - } - - mutating func encodeIfPresent(_ value: UInt8?, forKey key: Key) throws { - try doEncodeIf(value, forKey: key) - } - - mutating func encode(_ value: UInt16, forKey key: Key) throws { - try encoder.doEncode(value, key: key) - } - - mutating func encodeIfPresent(_ value: UInt16?, forKey key: Key) throws { - try doEncodeIf(value, forKey: key) - } - - mutating func encode(_ value: UInt32, forKey key: Key) throws { - try encoder.doEncode(value, key: key) - } - - mutating func encodeIfPresent(_ value: UInt32?, forKey key: Key) throws { - try doEncodeIf(value, forKey: key) - } - - mutating func encode(_ value: UInt64, forKey key: Key) throws { - try encoder.doEncode(value, key: key) - } - - mutating func encodeIfPresent(_ value: UInt64?, forKey key: Key) throws { - try doEncodeIf(value, forKey: key) - } - - mutating func encode(_ value: T, forKey key: Key) throws { - if ArrowArrayBuilders.isValidBuilderType(T.self) { - try encoder.doEncode(value, key: key) - } else { - throw ArrowError.invalid("Type \(T.self) is currently not supported") - } - } - - mutating func encodeIfPresent(_ value: T?, forKey key: Self.Key) throws - where T: Encodable { - if ArrowArrayBuilders.isValidBuilderType(T?.self) { - try doEncodeIf(value, forKey: key) - } else { - throw ArrowError.invalid("Type \(T.self) is currently not supported") - } - } - - // nested container is currently not allowed. This method doesn't throw - // so setting an error mesg that will be throw by the encoder at the next - // method call that throws - mutating func nestedContainer( - keyedBy keyType: NestedKey.Type, - forKey key: Key - ) -> KeyedEncodingContainer { - self.encoder.errorMsg = "Nested decoding is currently not supported." - var container = ArrowKeyedEncoding(self.encoder) - container.codingPath = codingPath - return KeyedEncodingContainer(container) - } - - // nested container is currently not allowed. This method doesn't throw - // so setting an error mesg that will be throw by the encoder at the next - // method call that throws - mutating func nestedUnkeyedContainer(forKey key: Key) - -> UnkeyedEncodingContainer - { - self.encoder.errorMsg = "Nested decoding is currently not supported." - return ArrowUnkeyedEncoding(self.encoder, codingPath: self.codingPath) - } - - // super encoding is currently not allowed. This method doesn't throw - // so setting an error mesg that will be throw by the encoder at the next - // method call that throws - mutating func superEncoder() -> Encoder { - self.encoder.errorMsg = "super encoding is currently not supported." - return self.encoder - } - - // super encoding is currently not allowed. This method doesn't throw - // so setting an error mesg that will be throw by the encoder at the next - // method call that throws - mutating func superEncoder(forKey key: Key) -> Encoder { - self.encoder.errorMsg = "super encoding is currently not supported." - return self.encoder - } -} - -private struct ArrowUnkeyedEncoding: UnkeyedEncodingContainer { - public private(set) var encoder: ArrowEncoder - var codingPath: [CodingKey] = [] - var currentIndex: Int - var count: Int = 0 - - init(_ encoder: ArrowEncoder, codingPath: [CodingKey], currentIndex: Int = 0) - { - self.encoder = encoder - self.currentIndex = currentIndex - } - - mutating func increment() { - self.currentIndex += 1 - } - - // If this method is called on row 0 and the encoder is - // lazily bulding holders then this will produce an error - // as this method does not know what the underlying type - // is for the column. This method is not called for - // nullable types (String?, Int32?, Date?) and the workaround - // for this issue would be to predefine the builders for the - // encoder. (I have only encoutered this issue when allowing - // nullable types at the encode func level which is currently - // not allowed) - mutating func encodeNil() throws { - try encoder.doEncodeNil(self.currentIndex) - } - - mutating func encode(_ value: T) throws where T: Encodable { - let type = T.self - if ArrowArrayBuilders.isValidBuilderType(type) { - defer { increment() } - return try self.encoder.doEncode(value, keyIndex: self.currentIndex) - } else { - throw ArrowError.invalid("Type \(type) is currently not supported") - } - } - - // nested container is currently not allowed. This method doesn't throw - // so setting an error mesg that will be throw by the encoder at the next - // method call that throws - mutating func nestedContainer( - keyedBy keyType: NestedKey.Type - ) -> KeyedEncodingContainer where NestedKey: CodingKey { - self.encoder.errorMsg = "Nested decoding is currently not supported." - var container = ArrowKeyedEncoding(self.encoder) - container.codingPath = codingPath - return KeyedEncodingContainer(container) - } - - // nested container is currently not allowed. This method doesn't throw - // so setting an error mesg that will be throw by the encoder at the next - // method call that throws - mutating func nestedUnkeyedContainer() -> UnkeyedEncodingContainer { - self.encoder.errorMsg = "Nested decoding is currently not supported." - return ArrowUnkeyedEncoding(self.encoder, codingPath: self.codingPath) - } - - // super encoding is currently not allowed. This method doesn't throw - // so setting an error mesg that will be throw by the encoder at the next - // method call that throws - mutating func superEncoder() -> Encoder { - self.encoder.errorMsg = "super encoding is currently not supported." - return self.encoder - } -} - -private struct ArrowSingleValueEncoding: SingleValueEncodingContainer { - public private(set) var encoder: ArrowEncoder - var codingPath: [CodingKey] = [] - - public init(_ encoder: ArrowEncoder, codingPath: [CodingKey]) { - self.encoder = encoder - self.codingPath = codingPath - } - - mutating func encodeNil() throws { - try self.encoder.doEncodeNil(0) - } - - mutating func encode(_ value: T) throws { - if ArrowArrayBuilders.isValidBuilderType(T.self) { - return try self.encoder.doEncode(value, keyIndex: 0) - } else { - throw ArrowError.invalid("Type \(T.self) is currently not supported") - } - } -} -// swiftlint:disable:this file_length diff --git a/Sources/Arrow/ArrowError.swift b/Sources/Arrow/ArrowError.swift index 3df9e7a..9a28491 100644 --- a/Sources/Arrow/ArrowError.swift +++ b/Sources/Arrow/ArrowError.swift @@ -12,14 +12,25 @@ // See the License for the specific language governing permissions and // limitations under the License. -public enum ArrowError: Error { - case none - case unknownType(String) - case runtimeError(String) - case outOfBounds(index: Int64) - case arrayHasNoElements - case unknownError(String) - case notImplemented - case ioError(String) - case invalid(String) +public struct ArrowError: Error { + + public enum ErrorType: Equatable, Sendable { + case none + case unknownType(String) + case runtimeError(String) + case outOfBounds(index: Int64) + case arrayHasNoElements + case unknownError(String) + case notImplemented + case ioError(String) + case invalid(String) + } + + let type: ErrorType + let underlyingError: Error? + + public init(_ type: ErrorType, underlyingError: Error? = nil) { + self.type = type + self.underlyingError = underlyingError + } } diff --git a/Sources/Arrow/ArrowReaderHelper.swift b/Sources/Arrow/ArrowReaderHelper.swift deleted file mode 100644 index 9ecec00..0000000 --- a/Sources/Arrow/ArrowReaderHelper.swift +++ /dev/null @@ -1,259 +0,0 @@ -// Copyright 2025 The Apache Software Foundation -// -// 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. - -import FlatBuffers -import Foundation - -private func makeBinaryHolder( - _ buffers: [ArrowBuffer], - nullCount: UInt -) -> Result { - let arrowType: ArrowType = .binary - let arrowData = ArrowData( - arrowType, - buffers: buffers, - nullCount: nullCount - ) - - do { - let array = try BinaryArray(arrowData) - return .success(array) - } catch { - return .failure(error) - } -} - -private func makeStringHolder( - _ buffers: [ArrowBuffer], - nullCount: UInt -) -> Result { - let arrowType: ArrowType = .utf8 - let arrowData = ArrowData( - arrowType, buffers: buffers, nullCount: nullCount) - - do { - let array = try StringArray(arrowData) - return .success(array) - } catch { - return .failure(error) - } -} - -private func makeBoolHolder( - _ buffers: [ArrowBuffer], - nullCount: UInt -) -> Result { - let arrowData = ArrowData( - .boolean, - buffers: buffers, - nullCount: nullCount - ) - do { - let array = try BoolArray(arrowData) - return .success(array) - } catch { - return .failure(error) - } -} - -private func makeFixedHolder( - _: T.Type, - field: ArrowField, - buffers: [ArrowBuffer], - nullCount: UInt -) -> Result { - let arrowData = ArrowData( - field.type, - buffers: buffers, - nullCount: nullCount - ) - do { - let array = try FixedArray(arrowData) - return .success(array) - } catch { - return .failure(error) - } -} - -func makeNestedHolder( - _ field: ArrowField, - buffers: [ArrowBuffer], - nullCount: UInt, - children: [ArrowData], - rbLength: UInt -) -> Result { - do { - let arrowData = ArrowData( - field.type, - buffers: buffers, - children: children, - nullCount: nullCount, - length: rbLength - ) - return .success(try NestedArray(arrowData)) - } catch { - return .failure(error) - } -} - -func makeArrayHolder( - _ field: FField, - buffers: [ArrowBuffer], - nullCount: UInt, - children: [ArrowData]?, - rbLength: UInt -) -> Result { - do { - let arrowField = try fromProto(field: field) - return makeArrayHolder( - arrowField, - buffers: buffers, - nullCount: nullCount, - children: children, - rbLength: rbLength - ) - } catch { - return .failure(error) - } -} - -func makeArrayHolder( - _ field: ArrowField, - buffers: [ArrowBuffer], - nullCount: UInt, - children: [ArrowData]?, - rbLength: UInt -) -> Result { - - let typeId = field.type - switch typeId { - case .int8: - return - makeFixedHolder( - Int8.self, field: field, buffers: buffers, nullCount: nullCount) - case .int16: - return - makeFixedHolder( - Int16.self, field: field, buffers: buffers, nullCount: nullCount) - case .int32: - return - makeFixedHolder( - Int32.self, field: field, buffers: buffers, nullCount: nullCount) - case .int64: - return - makeFixedHolder( - Int64.self, field: field, buffers: buffers, nullCount: nullCount) - case .uint8: - return - makeFixedHolder( - UInt8.self, field: field, buffers: buffers, nullCount: nullCount) - case .uint16: - return makeFixedHolder( - UInt16.self, field: field, buffers: buffers, nullCount: nullCount) - case .uint32: - return makeFixedHolder( - UInt32.self, field: field, buffers: buffers, nullCount: nullCount) - case .uint64: - return makeFixedHolder( - UInt64.self, field: field, buffers: buffers, nullCount: nullCount) - case .boolean: - return makeBoolHolder(buffers, nullCount: nullCount) - case .float32: - return - makeFixedHolder( - Float.self, field: field, buffers: buffers, nullCount: nullCount) - case .float64: - return - makeFixedHolder( - Double.self, field: field, buffers: buffers, nullCount: nullCount) - case .utf8: - return makeStringHolder(buffers, nullCount: nullCount) - case .binary: - return makeBinaryHolder(buffers, nullCount: nullCount) - case .date32: - let arrowData = ArrowData( - field.type, - buffers: buffers, - nullCount: nullCount - ) - do { - let array = try Date32Array(arrowData) - return .success(array) - } catch { - return .failure(error) - } - case .date64: - let arrowData = ArrowData( - field.type, - buffers: buffers, - nullCount: nullCount - ) - do { - let array = try Date64Array(arrowData) - return .success(array) - } catch { - return .failure(error) - } - case .time32: - let arrowData = ArrowData( - field.type, buffers: buffers, nullCount: nullCount) - do { - let array = try FixedArray(arrowData) - return .success(array) - } catch { - return .failure(error) - } - case .time64: - let arrowData = ArrowData( - field.type, buffers: buffers, nullCount: nullCount) - do { - let array = try FixedArray(arrowData) - return .success(array) - } catch { - return .failure(error) - } - case .timestamp: - let arrowData = ArrowData( - field.type, buffers: buffers, nullCount: nullCount) - do { - let array = try TimestampArray(arrowData) - return .success(array) - } catch { - return .failure(error) - } - case .strct: - guard let children else { - return .failure(.invalid("Expected a struct field to have children")) - } - return makeNestedHolder( - field, buffers: buffers, nullCount: nullCount, children: children, - rbLength: rbLength) - case .list: - guard let children else { - return .failure(.invalid("Expected a list field to have children")) - } - return makeNestedHolder( - field, buffers: buffers, nullCount: nullCount, children: children, - rbLength: rbLength) - default: - return .failure(.unknownType("Type \(typeId) currently not supported")) - } -} - -func getUInt32(_ data: Data, offset: Int) -> UInt32 { - let token = data.withUnsafeBytes { rawBuffer in - rawBuffer.loadUnaligned(fromByteOffset: offset, as: UInt32.self) - } - return token -} diff --git a/Sources/Arrow/ArrowReaderX.swift b/Sources/Arrow/ArrowReaderX.swift deleted file mode 100644 index 22b3668..0000000 --- a/Sources/Arrow/ArrowReaderX.swift +++ /dev/null @@ -1,586 +0,0 @@ -// Copyright 2025 The Apache Software Foundation -// Copyright 2025 The Columnar-Swift Contributors -// -// 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. - -import FlatBuffers -import Foundation - -let fileMarker = Data("ARROW1".utf8) -let continuationMarker = UInt32(0xFFFF_FFFF) - -public struct ArrowReaderX: Sendable { - - func makeBuffer( - _ buffer: FBuffer, - fileData: Data, - length: UInt, - messageOffset: Int64 - ) -> ArrowBuffer { - let startOffset = messageOffset + buffer.offset - let endOffset = startOffset + buffer.length - - // let range = Int(startOffset).. FFieldNode? { - if nodeIndex >= self.recordBatch.nodesCount { return nil } - defer { nodeIndex += 1 } - return self.recordBatch.nodes(at: nodeIndex) - } - - func nextBuffer() -> FBuffer? { - if bufferIndex >= self.recordBatch.buffersCount { return nil } - defer { bufferIndex += 1 } - return self.recordBatch.buffers(at: bufferIndex) - } - - func nextField() -> FField? { - if fieldIndex >= self.schema.fieldsCount { return nil } - defer { fieldIndex += 1 } - return self.schema.fields(at: fieldIndex) - } - - func isDone() -> Bool { - nodeIndex >= self.recordBatch.nodesCount - } - } - - private struct DataLoadInfo { - let fileData: Data - let messageOffset: Int64 - var batchData: RecordBatchData - } - - public class ArrowReaderResult { - fileprivate var messageSchema: FSchema? - public var schema: ArrowSchema? - public var batches: [RecordBatchX] = [] - } - - public init() {} - - private func loadSchema( - _ schema: FSchema - ) -> Result { - let builder = ArrowSchema.Builder() - for index in 0.. Result { - guard let node = loadInfo.batchData.nextNode() else { - return .failure(.invalid("Node not found")) - } - - guard let nullBuffer = loadInfo.batchData.nextBuffer() else { - return .failure(.invalid("Null buffer not found")) - } - - let nullLength = UInt(ceil(Double(node.length) / 8)) - let arrowNullBuffer = makeBuffer( - nullBuffer, fileData: loadInfo.fileData, - length: nullLength, messageOffset: loadInfo.messageOffset) - var children: [ArrowData] = [] - for index in 0.. Result - { - guard let node = loadInfo.batchData.nextNode() else { - return .failure(.invalid("Node not found")) - } - - guard let nullBuffer = loadInfo.batchData.nextBuffer() else { - return .failure(.invalid("Null buffer not found")) - } - - guard let offsetBuffer = loadInfo.batchData.nextBuffer() else { - return .failure(.invalid("Offset buffer not found")) - } - - let nullLength = UInt(ceil(Double(node.length) / 8)) - let arrowNullBuffer = makeBuffer( - nullBuffer, fileData: loadInfo.fileData, length: nullLength, - messageOffset: loadInfo.messageOffset) - let arrowOffsetBuffer = makeBuffer( - offsetBuffer, fileData: loadInfo.fileData, length: UInt(node.length + 1), - messageOffset: loadInfo.messageOffset) - - guard field.childrenCount == 1, let childField = field.children(at: 0) - else { - return .failure(.invalid("List must have exactly one child")) - } - - switch loadField(loadInfo, field: childField) { - case .success(let childHolder): - return makeArrayHolder( - field, - buffers: [arrowNullBuffer, arrowOffsetBuffer], - nullCount: UInt(node.nullCount), - children: [childHolder.arrowData], - rbLength: UInt(loadInfo.batchData.recordBatch.length)) - case .failure(let error): - return .failure(error) - } - } - - private func loadPrimitiveData( - _ loadInfo: DataLoadInfo, - field: FField - ) - -> Result - { - guard let node = loadInfo.batchData.nextNode() else { - return .failure(.invalid("Node not found")) - } - - guard let nullBuffer = loadInfo.batchData.nextBuffer() else { - return .failure(.invalid("Null buffer not found")) - } - - guard let valueBuffer = loadInfo.batchData.nextBuffer() else { - return .failure(.invalid("Value buffer not found")) - } - - let nullLength = UInt(ceil(Double(node.length) / 8)) - let arrowNullBuffer = makeBuffer( - nullBuffer, - fileData: loadInfo.fileData, - length: nullLength, - messageOffset: loadInfo.messageOffset - ) - let arrowValueBuffer = makeBuffer( - valueBuffer, - fileData: loadInfo.fileData, - length: UInt(node.length), - messageOffset: loadInfo.messageOffset - ) - return makeArrayHolder( - field, - buffers: [arrowNullBuffer, arrowValueBuffer], - nullCount: UInt(node.nullCount), - children: nil, - rbLength: UInt(loadInfo.batchData.recordBatch.length) - ) - } - - // MARK: Variable data loading - - private func loadVariableData( - _ loadInfo: DataLoadInfo, - field: FField - ) - -> Result - { - guard let node = loadInfo.batchData.nextNode() else { - return .failure(.invalid("Node not found")) - } - - guard let nullBuffer = loadInfo.batchData.nextBuffer() else { - return .failure(.invalid("Null buffer not found")) - } - - guard let offsetBuffer = loadInfo.batchData.nextBuffer() else { - return .failure(.invalid("Offset buffer not found")) - } - - guard let valueBuffer = loadInfo.batchData.nextBuffer() else { - return .failure(.invalid("Value buffer not found")) - } - - let nullLength = UInt(ceil(Double(node.length) / 8)) - let arrowNullBuffer = makeBuffer( - nullBuffer, fileData: loadInfo.fileData, - length: nullLength, messageOffset: loadInfo.messageOffset) - - let arrowOffsetBuffer = makeBuffer( - offsetBuffer, fileData: loadInfo.fileData, - length: UInt(node.length), messageOffset: loadInfo.messageOffset) - let arrowValueBuffer = makeBuffer( - valueBuffer, fileData: loadInfo.fileData, - length: UInt(node.length), messageOffset: loadInfo.messageOffset) - return makeArrayHolder( - field, buffers: [arrowNullBuffer, arrowOffsetBuffer, arrowValueBuffer], - nullCount: UInt(node.nullCount), children: nil, - rbLength: UInt(loadInfo.batchData.recordBatch.length)) - } - - private func loadField( - _ loadInfo: DataLoadInfo, - field: FField - ) - -> Result - { - switch field.typeType { - case .struct_: - return loadStructData(loadInfo, field: field) - case .list: - return loadListData(loadInfo, field: field) - default: - if isFixedPrimitive(field.typeType) { - return loadPrimitiveData(loadInfo, field: field) - } else { - return loadVariableData(loadInfo, field: field) - } - } - } - - private func loadRecordBatch( - _ recordBatch: FRecordBatch, - schema: FSchema, - arrowSchema: ArrowSchema, - data: Data, - messageEndOffset: Int64 - ) -> Result { - var columns: [AnyArrowArray] = [] - let batchData = RecordBatchData(recordBatch, schema: schema) - let loadInfo = DataLoadInfo( - fileData: data, - messageOffset: messageEndOffset, - batchData: batchData) - while !batchData.isDone() { - guard let field = batchData.nextField() else { - return .failure(.invalid("Field not found")) - } - - let result = loadField(loadInfo, field: field) - switch result { - case .success(let holder): - columns.append(holder) - case .failure(let error): - return .failure(error) - } - } - return .success(RecordBatchX(arrowSchema, columns: columns)) - } - - /// This is for reading the Arrow streaming format. - /// - /// The Arrow streaming format is slightly different from the Arrow File format as it doesn't contain a - /// header and footer. - /// - Parameters: - /// - input: The buffer to read from - /// - useUnalignedBuffers: to be removed. - /// - Returns: An `ArrowReaderResult` If successful, otherwise an `ArrowError`. - public func readStreaming( - _ input: Data, - useUnalignedBuffers: Bool = false - ) -> Result { - let result = ArrowReaderResult() - var offset: Int = 0 - var length = getUInt32(input, offset: offset) - var streamData = input - // TODO: The following assumes message order will populate schemaMessage first - var schemaMessage: FSchema? - while length != 0 { - if length == continuationMarker { - offset += Int(MemoryLayout.size) - length = getUInt32(input, offset: offset) - if length == 0 { - return .success(result) - } - } - offset += Int(MemoryLayout.size) - streamData = input[offset...] - var dataBuffer = ByteBuffer( - data: streamData - ) - let message: FMessage = getRoot(byteBuffer: &dataBuffer) - switch message.headerType { - case .recordbatch: - guard let rbMessage = message.header(type: FRecordBatch.self) else { - return .failure(.invalid("Failed to parse RecordBatch message")) - } - guard let schemaMessage else { - return .failure(.invalid("Schema message not found")) - } - guard let resultSchema = result.schema else { - return .failure(.invalid("Result schema not loaded")) - } - let recordBatchResult = loadRecordBatch( - rbMessage, - schema: schemaMessage, - arrowSchema: resultSchema, - data: input, - messageEndOffset: Int64(offset) + Int64(length) - ) - switch recordBatchResult { - case .success(let recordBatch): - result.batches.append(recordBatch) - case .failure(let error): - return .failure(error) - } - offset += Int(message.bodyLength + Int64(length)) - length = getUInt32(input, offset: offset) - case .schema: - schemaMessage = message.header(type: FSchema.self) - guard let schemaMessage else { - return .failure(.invalid("Schema message not found")) - } - let schemaResult = loadSchema(schemaMessage) - switch schemaResult { - case .success(let schema): - result.schema = schema - case .failure(let error): - return .failure(error) - } - offset += Int(message.bodyLength + Int64(length)) - length = getUInt32(input, offset: offset) - default: - return .failure( - .unknownError("Unhandled header type: \(message.headerType)")) - } - } - return .success(result) - } - - /// This is for reading the Arrow file format. - /// - /// The Arrow file format supports random access. The Arrow file format contains a header and footer - /// around the Arrow streaming format. - /// - Parameters: - /// - fileData: the file content - /// - useUnalignedBuffers: to be removed. - /// - Returns: An `ArrowReaderResult` on success, or an `ArrowError` on failure. - public func readFile( - _ fileData: Data, - useUnalignedBuffers: Bool = false - ) -> Result { - let footerLength = fileData.withUnsafeBytes { rawBuffer in - rawBuffer.loadUnaligned( - fromByteOffset: fileData.count - 4, as: Int32.self) - } - - let result = ArrowReaderResult() - let footerStartOffset = fileData.count - Int(footerLength + 4) - let footerData = fileData[footerStartOffset...] - var footerBuffer = ByteBuffer( - data: footerData, - allowReadingUnalignedBuffers: useUnalignedBuffers) - let footer: FFooter = getRoot(byteBuffer: &footerBuffer) - guard let footerSchema = footer.schema else { - return .failure(.invalid("Missing schema in footer")) - } - let schemaResult = loadSchema(footerSchema) - switch schemaResult { - case .success(let schema): - result.schema = schema - case .failure(let error): - return .failure(error) - } - - for index in 0...size)), - as: UInt32.self) - } - } - - let messageStartOffset = - recordBatch.offset + (Int64(MemoryLayout.size) * messageOffset) - let messageEndOffset = messageStartOffset + Int64(messageLength) - let recordBatchData = fileData[messageStartOffset.. Result { - do { - // TODO: implement alignment checks. - let fileData = try Data(contentsOf: fileURL, options: .mappedIfSafe) - - if !validateFileData(fileData) { - return .failure(.ioError("Not a valid arrow file.")) - } - let data = fileData[.. ArrowReaderResult { - ArrowReaderResult() - } - - public func fromMessage( - _ dataHeader: Data, - dataBody: Data, - result: ArrowReaderResult, - useUnalignedBuffers: Bool = false - ) -> Result { - var mbb = ByteBuffer( - data: dataHeader, - allowReadingUnalignedBuffers: useUnalignedBuffers) - let message: FMessage = getRoot(byteBuffer: &mbb) - switch message.headerType { - case .schema: - guard let sMessage = message.header(type: FSchema.self) else { - return .failure(.unknownError("Expected a schema but found none")) - } - switch loadSchema(sMessage) { - case .success(let schema): - result.schema = schema - result.messageSchema = sMessage - return .success(()) - case .failure(let error): - return .failure(error) - } - case .recordbatch: - guard let rbMessage = message.header(type: FRecordBatch.self) else { - return .failure(.invalid("Expected a RecordBatch but found none")) - } - // TODO: the result used here is also the return type. Ideally is would be constructed once as a struct (same issue as above) - guard let messageSchema = result.messageSchema else { - return .failure( - .invalid("Expected the result to have the messageSchema already")) - } - guard let resultSchema = result.schema else { - return .failure(.invalid("Expected result to have a schema")) - } - let recordBatchResult = loadRecordBatch( - rbMessage, - schema: messageSchema, - arrowSchema: resultSchema, - data: dataBody, - messageEndOffset: 0 - ) - switch recordBatchResult { - case .success(let recordBatch): - result.batches.append(recordBatch) - return .success(()) - case .failure(let error): - return .failure(error) - } - default: - return .failure( - .unknownError("Unhandled header type: \(message.headerType)")) - } - } - - func validateFileData(_ data: Data) -> Bool { - let markerLength = fileMarker.count - let startString = data[..() throws(ArrowError) -> ChunkedArray { - if let holder = self.dataHolder.holder as? ChunkedArray { - return holder - } else { - throw .runtimeError("Could not cast array holder to chunked array.") - } - } -} - -public class ArrowTable { - public let schema: ArrowSchema - public var columnCount: UInt { UInt(self.columns.count) } - public let rowCount: UInt - public let columns: [ArrowColumn] - init(_ schema: ArrowSchema, columns: [ArrowColumn]) { - self.schema = schema - self.columns = columns - self.rowCount = columns[0].length - } - - /// Create an ArrowTable from a 'RecordBatch' list. - /// - Parameter recordBatches: The record batches. - /// - Returns: An `ArrowResult` holding an `ArrowTable` on success, or an`ArrowError` - /// on failure. - public static func from( - recordBatches: [RecordBatchX] - ) -> Result { - if recordBatches.isEmpty { - return .failure(.arrayHasNoElements) - } - var holders: [[AnyArrowArray]] = [] - let schema = recordBatches[0].schema - for recordBatch in recordBatches { - for index in 0.. ArrowColumn { - // Dispatch based on the field's type, not the first holder - switch field.type { - case .int8: - return try makeTypedColumn(field, holders, type: Int8.self) - case .int16: - return try makeTypedColumn(field, holders, type: Int16.self) - case .int32: - return try makeTypedColumn(field, holders, type: Int32.self) - case .int64: - return try makeTypedColumn(field, holders, type: Int64.self) - case .uint8: - return try makeTypedColumn(field, holders, type: UInt8.self) - case .uint16: - return try makeTypedColumn(field, holders, type: UInt16.self) - case .uint32: - return try makeTypedColumn(field, holders, type: UInt32.self) - case .uint64: - return try makeTypedColumn(field, holders, type: UInt64.self) - case .float32: - return try makeTypedColumn(field, holders, type: Float.self) - case .float64: - return try makeTypedColumn(field, holders, type: Double.self) - case .utf8, .binary: - return try makeTypedColumn(field, holders, type: String.self) - case .boolean: - return try makeTypedColumn(field, holders, type: Bool.self) - case .date32, .date64: - return try makeTypedColumn(field, holders, type: Date.self) - // TODO: make a fuzzer to make sure all types are hit - default: - throw ArrowError.unknownType("Unsupported type: \(field.type)") - } - } - - private static func makeTypedColumn( - _ field: ArrowField, - _ holders: [AnyArrowArray], - type: T.Type - ) throws(ArrowError) -> ArrowColumn { - var arrays: [any ArrowArray] = [] - for holder in holders { - guard let array = holder as? (any ArrowArray) else { - throw .runtimeError( - "Array type mismatch: expected \(T.self) for field \(field.name)" - ) - } - arrays.append(array) - } - return ArrowColumn( - field, - chunked: ChunkedArrayHolder(try ChunkedArray(arrays)) - ) - } - - public class Builder { - let schemaBuilder = ArrowSchema.Builder() - var columns: [ArrowColumn] = [] - - public init() {} - - @discardableResult - public func addColumn( - _ fieldName: String, - arrowArray: any ArrowArray - ) throws -> Builder { - self.addColumn(fieldName, chunked: try ChunkedArray([arrowArray])) - } - - @discardableResult - public func addColumn( - _ fieldName: String, - chunked: ChunkedArray - ) -> Builder { - let field = ArrowField( - name: fieldName, - dataType: chunked.type, - isNullable: chunked.nullCount != 0 - ) - self.schemaBuilder.addField(field) - let column = ArrowColumn(field, chunked: ChunkedArrayHolder(chunked)) - self.columns.append(column) - return self - } - - @discardableResult - public func addColumn( - _ field: ArrowField, - arrowArray: any ArrowArray - ) throws -> Builder { - self.schemaBuilder.addField(field) - let holder = ChunkedArrayHolder(try ChunkedArray([arrowArray])) - self.columns.append(ArrowColumn(field, chunked: holder)) - return self - } - - @discardableResult - public func addColumn( - _ field: ArrowField, - chunked: ChunkedArray - ) -> Builder { - self.schemaBuilder.addField(field) - let column = ArrowColumn(field, chunked: ChunkedArrayHolder(chunked)) - self.columns.append(column) - return self - } - - @discardableResult - public func addColumn(_ column: ArrowColumn) -> Builder { - self.schemaBuilder.addField(column.field) - self.columns.append(column) - return self - } - - public func finish() -> ArrowTable { - ArrowTable(self.schemaBuilder.finish(), columns: self.columns) - } - } -} - -public class RecordBatchX { - public let schema: ArrowSchema - public var columnCount: UInt { UInt(self.columns.count) } - public let columns: [AnyArrowArray] - public let length: UInt - - public init(_ schema: ArrowSchema, columns: [AnyArrowArray]) { - self.schema = schema - self.columns = columns - self.length = columns[0].length - } - - public class Builder { - let schemaBuilder = ArrowSchema.Builder() - var columns: [AnyArrowArray] = [] - - public init() {} - - /// Add a column the `RecordBatch` builder. - /// - Parameters: - /// - fieldName: The field name. - /// - arrowArray: The array to add to the reocrd batch. - /// - Returns: The `RecordBatch.Builder` with the array appended and the field added to - /// the schema. If the array contains zero nulls, the field is defined as non-null. - @discardableResult - public func addColumn( - _ fieldName: String, - arrowArray: AnyArrowArray - ) -> Builder { - let field = ArrowField( - name: fieldName, - dataType: arrowArray.type, - isNullable: arrowArray.nullCount != 0 - ) - self.schemaBuilder.addField(field) - self.columns.append(arrowArray) - return self - } - - /// Add a column the `RecordBatch` builder. - /// - Parameters: - /// - field: The field describing the array. - /// - arrowArray: The array to add to the reocrd batch. - /// - Returns: The `RecordBatch.Builder` with the array appended and the field added to - /// the schema. - @discardableResult - public func addColumn( - _ field: ArrowField, - arrowArray: AnyArrowArray - ) -> Builder { - self.schemaBuilder.addField(field) - self.columns.append(arrowArray) - return self - } - - public func finish() -> Result { - if columns.count > 0 { - let columnLength = columns[0].length - for column in columns { - if column.length != columnLength { - return .failure(.runtimeError("Columns have different sizes")) - } - } - } - // Check nullability matches actual data - let schema = self.schemaBuilder.finish() - for (index, field) in schema.fields.enumerated() { - let column = columns[index] - if !field.isNullable && column.nullCount > 0 { - return .failure( - .invalid( - "non-nullable column '\(field.name)' contains \(column.nullCount) null values." - )) - } - } - return .success( - RecordBatchX(self.schemaBuilder.finish(), columns: self.columns) - ) - } - } - - public func data( - for columnIndex: Int - ) throws(ArrowError) -> any ArrowArray { - let arrayHolder = column(columnIndex) - if let array = arrayHolder as? any ArrowArray { - return array - } else { - throw .invalid( - "Could not convert \(arrayHolder) for \(columnIndex)" - ) - } - } - - public func column(_ index: Int) -> AnyArrowArray { - self.columns[index] - } - - public func column(_ name: String) -> AnyArrowArray? { - if let index = self.schema.fieldIndex(name) { - return self.columns[index] - } else { - return nil - } - } -} diff --git a/Sources/Arrow/ArrowType.swift b/Sources/Arrow/ArrowType.swift index 829c9de..04726fd 100644 --- a/Sources/Arrow/ArrowType.swift +++ b/Sources/Arrow/ArrowType.swift @@ -650,6 +650,14 @@ extension ArrowType { } } + @inlinable + public var isBinaryView: Bool { + switch self { + case .binaryView, .utf8View: true + default: false + } + } + /// Returns true if this type is DataType::Null. @inlinable public var isNull: Bool { @@ -917,7 +925,7 @@ extension ArrowType { case .second: return "tts" default: - throw .invalid("\(unit) invalid for Time32.") + throw .init(.invalid("\(unit) invalid for Time32.")) } case .time64(let unit): switch unit { @@ -926,7 +934,7 @@ extension ArrowType { case .nanosecond: return "ttn" default: - throw .invalid("\(unit) invalid for Time64.") + throw .init(.invalid("\(unit) invalid for Time64.")) } case .timestamp(let unit, let timezone): let unitChar: Character = @@ -955,7 +963,7 @@ extension ArrowType { case .list(let field): return "+l" + (try field.type.cDataFormatId) default: - throw .notImplemented + throw .init(.notImplemented) } } } @@ -1000,9 +1008,10 @@ extension ArrowType { } else if from.starts(with: "ts") { let components = from.split(separator: ":", maxSplits: 1) guard let unitPart = components.first, unitPart.count == 3 else { - throw .invalid( - "Invalid timestamp format '\(from)'. Expected format 'ts[s|m|u|n][:timezone]'" - ) + throw .init( + .invalid( + "Invalid timestamp format '\(from)'. Expected format 'ts[s|m|u|n][:timezone]'" + )) } let unitChar = unitPart.suffix(1) let unit: TimeUnit = @@ -1012,9 +1021,10 @@ extension ArrowType { case "u": .microsecond case "n": .nanosecond default: - throw .invalid( - "Unrecognized timestamp unit '\(unitChar)'. Expected 's', 'm', 'u', or 'n'." - ) + throw .init( + .invalid( + "Unrecognized timestamp unit '\(unitChar)'. Expected 's', 'm', 'u', or 'n'." + )) } let timezone = components.count > 1 ? String(components[1]) : nil return .timestamp(unit, timezone) @@ -1023,6 +1033,6 @@ extension ArrowType { } else if from == "u" { return .utf8 } - throw .notImplemented + throw .init(.notImplemented) } } diff --git a/Sources/Arrow/ArrowTypeConverter.swift b/Sources/Arrow/ArrowTypeConverter.swift deleted file mode 100644 index d32e256..0000000 --- a/Sources/Arrow/ArrowTypeConverter.swift +++ /dev/null @@ -1,83 +0,0 @@ -// Copyright 2025 The Apache Software Foundation -// Copyright 2025 The Columnar Swift Contributors -// -// 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. - -import Foundation - -public struct ArrowTypeConverter { - - public static func infoForType( - _ type: Any.Type - ) throws(ArrowError) -> ArrowType { - if type == String.self { - return .utf8 - } else if type == Date.self { - return .date64 - } else if type == Bool.self { - return .boolean - } else if type == Data.self { - return .binary - } else if type == Int8.self { - return .int8 - } else if type == Int16.self { - return .int16 - } else if type == Int32.self { - return .int32 - } else if type == Int64.self { - return .int64 - } else if type == UInt8.self { - return .uint8 - } else if type == UInt16.self { - return .uint16 - } else if type == UInt32.self { - return .uint32 - } else if type == UInt64.self { - return .uint64 - } else if type == Float.self { - return .float32 - } else if type == Double.self { - return .float64 - } else { - throw .invalid("Unsupported type: \(type)") - } - } - - public static func infoForNumericType( - _ type: T.Type - ) throws(ArrowError) -> ArrowType { - if type == Int8.self { - return .int8 - } else if type == Int16.self { - return .int16 - } else if type == Int32.self { - return .int32 - } else if type == Int64.self { - return .int64 - } else if type == UInt8.self { - return .uint8 - } else if type == UInt16.self { - return .uint16 - } else if type == UInt32.self { - return .uint32 - } else if type == UInt64.self { - return .uint64 - } else if type == Float.self { - return .float32 - } else if type == Double.self { - return .float64 - } else { - throw .invalid("Unsupported numeric type: \(type)") - } - } -} diff --git a/Sources/Arrow/ArrowWriterHelper.swift b/Sources/Arrow/ArrowWriterHelper.swift deleted file mode 100644 index f444b36..0000000 --- a/Sources/Arrow/ArrowWriterHelper.swift +++ /dev/null @@ -1,149 +0,0 @@ -// Copyright 2025 The Apache Software Foundation -// -// 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. - -import FlatBuffers -import Foundation - -extension Data { - func hexEncodedString() -> String { - map { String(format: "%02hhx", $0) }.joined() - } -} - -func toFBTypeEnum(_ arrowType: ArrowType) -> Result { - let typeId = arrowType - switch typeId { - case .int8, .int16, .int32, .int64, .uint8, .uint16, .uint32, .uint64: - return .success(FType.int) - case .float16, .float32, .float64: - return .success(FType.floatingpoint) - case .utf8: - return .success(FType.utf8) - case .binary: - return .success(FType.binary) - case .boolean: - return .success(FType.bool) - case .date32, .date64: - return .success(FType.date) - case .time32, .time64: - return .success(FType.time) - case .timestamp: - return .success(FType.timestamp) - case .strct: - return .success(FType.struct_) - default: - return .failure( - .unknownType("Unable to find flatbuf type for Arrow type: \(typeId)") - ) - } -} - -func toFBType( - _ fbb: inout FlatBufferBuilder, - arrowType: ArrowType -) -> Result { - // let infoType = arrowType.info - switch arrowType { - case .int8, .uint8: - return .success( - FInt.createInt(&fbb, bitWidth: 8, isSigned: arrowType == .int8)) - case .int16, .uint16: - return .success( - FInt.createInt(&fbb, bitWidth: 16, isSigned: arrowType == .int16)) - case .int32, .uint32: - return .success( - FInt.createInt(&fbb, bitWidth: 32, isSigned: arrowType == .int32)) - case .int64, .uint64: - return .success( - FInt.createInt(&fbb, bitWidth: 64, isSigned: arrowType == .int64)) - case .float16: - return .success(FFloatingPoint.createFloatingPoint(&fbb, precision: .half)) - case .float32: - return .success( - FFloatingPoint.createFloatingPoint(&fbb, precision: .single)) - case .float64: - return .success( - FFloatingPoint.createFloatingPoint(&fbb, precision: .double)) - case .utf8: - return .success(FUtf8.endUtf8(&fbb, start: FUtf8.startUtf8(&fbb))) - case .binary: - return .success(FBinary.endBinary(&fbb, start: FBinary.startBinary(&fbb))) - case .boolean: - return .success(FBool.endBool(&fbb, start: FBool.startBool(&fbb))) - case .date32: - let startOffset = FDate.startDate(&fbb) - FDate.add(unit: .day, &fbb) - return .success(FDate.endDate(&fbb, start: startOffset)) - case .date64: - let startOffset = FDate.startDate(&fbb) - FDate.add(unit: .millisecond, &fbb) - return .success(FDate.endDate(&fbb, start: startOffset)) - case .time32(let unit): - let startOffset = FTime.startTime(&fbb) - FTime.add(unit: unit == .second ? .second : .millisecond, &fbb) - return .success(FTime.endTime(&fbb, start: startOffset)) - case .time64(let unit): - let startOffset = FTime.startTime(&fbb) - FTime.add(unit: unit == .microsecond ? .microsecond : .nanosecond, &fbb) - return .success(FTime.endTime(&fbb, start: startOffset)) - case .timestamp(let unit, let timezone): - let startOffset = FTimestamp.startTimestamp(&fbb) - let fbUnit: FTimeUnit - switch unit { - case .second: - fbUnit = .second - case .millisecond: - fbUnit = .millisecond - case .microsecond: - fbUnit = .microsecond - case .nanosecond: - fbUnit = .nanosecond - } - FTimestamp.add(unit: fbUnit, &fbb) - if let timezone { - let timezoneOffset = fbb.create(string: timezone) - FTimestamp.add(timezone: timezoneOffset, &fbb) - } - return .success(FTimestamp.endTimestamp(&fbb, start: startOffset)) - case .strct(_): - let startOffset = FStruct.startStruct_(&fbb) - return .success(FStruct.endStruct_(&fbb, start: startOffset)) - default: - return .failure( - .unknownType("Unable to add flatbuf type for Arrow type: \(arrowType)") - ) - } -} - -func addPadForAlignment(_ data: inout Data, alignment: Int = 8) { - let padding = data.count % Int(alignment) - if padding > 0 { - data.append(Data([UInt8](repeating: 0, count: alignment - padding))) - } -} - -func addPadForAlignment(_ writer: inout DataWriter, alignment: Int = 8) { - let padding = writer.count % Int(alignment) - if padding > 0 { - writer.append(Data([UInt8](repeating: 0, count: alignment - padding))) - } -} - -func getPadForAlignment(_ count: Int, alignment: Int = 8) -> Int { - let padding = count % Int(alignment) - if padding > 0 { - return count + (alignment - padding) - } - return count -} diff --git a/Sources/Arrow/ArrowWriterX.swift b/Sources/Arrow/ArrowWriterX.swift deleted file mode 100644 index 527dfcf..0000000 --- a/Sources/Arrow/ArrowWriterX.swift +++ /dev/null @@ -1,532 +0,0 @@ -// Copyright 2025 The Apache Software Foundation -// Copyright 2025 Columnar-Swift contributors -// -// 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. - -import FlatBuffers -import Foundation - -public protocol DataWriter { - var count: Int { get } - func append(_ data: Data) -} - -public class ArrowWriter { - public class InMemDataWriter: DataWriter { - public private(set) var data: Data - public var count: Int { data.count } - - public init(_ data: Data) { - self.data = data - } - - convenience init() { - self.init(Data()) - } - - public func append(_ data: Data) { - self.data.append(data) - } - } - - public class FileDataWriter: DataWriter { - private var handle: FileHandle - private var currentSize: Int = 0 - public var count: Int { currentSize } - public init(_ handle: FileHandle) { - self.handle = handle - } - - public func append(_ data: Data) { - self.handle.write(data) - self.currentSize += data.count - } - } - - public class Info { - let type: FMessageHeader - public let schema: ArrowSchema - public let batches: [RecordBatchX] - - init( - _ type: FMessageHeader, - schema: ArrowSchema, - batches: [RecordBatchX] - ) { - self.type = type - self.schema = schema - self.batches = batches - } - - convenience init(_ type: FMessageHeader, schema: ArrowSchema) { - self.init(type, schema: schema, batches: [RecordBatchX]()) - } - } - - public init() {} - - private func writeField( - _ fbb: inout FlatBufferBuilder, - field: ArrowField - ) -> Result { - var fieldsOffset: Offset? - if case .strct(let fields) = field.type { - var offsets: [Offset] = [] - for field in fields { - switch writeField(&fbb, field: field) { - case .success(let offset): - offsets.append(offset) - case .failure(let error): - return .failure(error) - } - } - fieldsOffset = fbb.createVector(ofOffsets: offsets) - } - - let nameOffset = fbb.create(string: field.name) - let fieldTypeOffsetResult = toFBType(&fbb, arrowType: field.type) - let startOffset = FField.startField(&fbb) - FField.add(name: nameOffset, &fbb) - FField.add(nullable: field.isNullable, &fbb) - if let childrenOffset = fieldsOffset { - FField.addVectorOf(children: childrenOffset, &fbb) - } - - switch toFBTypeEnum(field.type) { - case .success(let type): - FField.add(typeType: type, &fbb) - case .failure(let error): - return .failure(error) - } - - switch fieldTypeOffsetResult { - case .success(let offset): - FField.add(type: offset, &fbb) - return .success(FField.endField(&fbb, start: startOffset)) - case .failure(let error): - return .failure(error) - } - } - - private func writeSchema( - _ fbb: inout FlatBufferBuilder, - schema: ArrowSchema - ) -> Result { - var fieldOffsets: [Offset] = [] - for field in schema.fields { - switch writeField(&fbb, field: field) { - case .success(let offset): - fieldOffsets.append(offset) - case .failure(let error): - return .failure(error) - } - } - let fieldsOffset: Offset = fbb.createVector(ofOffsets: fieldOffsets) - let schemaOffset = - FSchema.createSchema( - &fbb, - endianness: .little, - fieldsVectorOffset: fieldsOffset - ) - return .success(schemaOffset) - } - - private func writeRecordBatches( - _ writer: inout DataWriter, - batches: [RecordBatchX] - ) -> Result<[FBlock], ArrowError> { - var rbBlocks: [FBlock] = .init() - for batch in batches { - let startIndex = writer.count - switch writeRecordBatch(batch: batch) { - case .success(let rbResult): - withUnsafeBytes(of: continuationMarker.littleEndian) { - writer.append(Data($0)) - } - withUnsafeBytes(of: UInt32(rbResult.count).littleEndian) { - writer.append(Data($0)) - } - writer.append(rbResult) - addPadForAlignment(&writer) - let metadataLength = writer.count - startIndex - let bodyStart = writer.count - switch writeRecordBatchData( - &writer, - fields: batch.schema.fields, - columns: batch.columns - ) { - case .success: - let bodyLength = writer.count - bodyStart - let expectedSize = startIndex + metadataLength + bodyLength - guard expectedSize == writer.count else { - return .failure( - .invalid( - "Invalid Block. Expected \(expectedSize), got \(writer.count)" - )) - } - rbBlocks.append( - FBlock( - offset: Int64(startIndex), - metaDataLength: Int32(metadataLength), - bodyLength: Int64(bodyLength) - ) - ) - case .failure(let error): - return .failure(error) - } - case .failure(let error): - return .failure(error) - } - } - - return .success(rbBlocks) - } - - private func writeFieldNodes( - _ fields: [ArrowField], - columns: [AnyArrowArray], - offsets: inout [Offset], - fbb: inout FlatBufferBuilder - ) { - for index in (0.. Result { - let schema = batch.schema - var fbb = FlatBufferBuilder() - - // write out field nodes - var fieldNodeOffsets: [Offset] = [] - fbb.startVector( - schema.fields.count, - elementSize: MemoryLayout.size - ) - writeFieldNodes( - schema.fields, - columns: batch.columns, - offsets: &fieldNodeOffsets, - fbb: &fbb - ) - let nodeOffset = fbb.endVector(len: fieldNodeOffsets.count) - // write out buffers - var buffers: [FBuffer] = .init() - var bufferOffset: Int = 0 - writeBufferInfo( - schema.fields, columns: batch.columns, - bufferOffset: &bufferOffset, buffers: &buffers, - fbb: &fbb - ) - FRecordBatch.startVectorOfBuffers(batch.schema.fields.count, in: &fbb) - for buffer in buffers.reversed() { - fbb.create(struct: buffer) - } - let batchBuffersOffset = fbb.endVector(len: buffers.count) - let startRb = FRecordBatch.startRecordBatch(&fbb) - FRecordBatch.addVectorOf(nodes: nodeOffset, &fbb) - FRecordBatch.addVectorOf(buffers: batchBuffersOffset, &fbb) - FRecordBatch.add(length: Int64(batch.length), &fbb) - let recordBatchOffset = FRecordBatch.endRecordBatch( - &fbb, - start: startRb - ) - let bodySize = Int64(bufferOffset) - let startMessage = FMessage.startMessage(&fbb) - FMessage.add(version: .max, &fbb) - FMessage.add(bodyLength: Int64(bodySize), &fbb) - FMessage.add(headerType: .recordbatch, &fbb) - FMessage.add(header: recordBatchOffset, &fbb) - let messageOffset = FMessage.endMessage(&fbb, start: startMessage) - fbb.finish(offset: messageOffset) - return .success(fbb.data) - } - - private func writeRecordBatchData( - _ writer: inout DataWriter, fields: [ArrowField], - columns: [AnyArrowArray] - ) -> Result { - for index in 0.. Result { - var fbb: FlatBufferBuilder = .init() - switch writeSchema(&fbb, schema: schema) { - case .success(let schemaOffset): - fbb.startVector( - rbBlocks.count, elementSize: MemoryLayout.size) - for blkInfo in rbBlocks.reversed() { - fbb.create(struct: blkInfo) - } - let rbBlkEnd = fbb.endVector(len: rbBlocks.count) - let footerStartOffset = FFooter.startFooter(&fbb) - FFooter.add(schema: schemaOffset, &fbb) - FFooter.addVectorOf(recordBatches: rbBlkEnd, &fbb) - let footerOffset = FFooter.endFooter(&fbb, start: footerStartOffset) - fbb.finish(offset: footerOffset) - return .success(fbb.data) - case .failure(let error): - return .failure(error) - } - } - - private func writeFile( - _ writer: inout DataWriter, - info: ArrowWriter.Info - ) -> Result { - var fbb: FlatBufferBuilder = .init() - switch writeSchema(&fbb, schema: info.schema) { - case .success(let schemaOffset): - fbb.finish(offset: schemaOffset) - writer.append(fbb.data) - addPadForAlignment(&writer) - case .failure(let error): - return .failure(error) - } - switch writeRecordBatches(&writer, batches: info.batches) { - case .success(let rbBlocks): - switch writeFooter(schema: info.schema, rbBlocks: rbBlocks) { - case .success(let footerData): - // fbb.finish(offset: Offset(offset: fbb.buffer.size)) - let footerOffset = writer.count - writer.append(footerData) - addPadForAlignment(&writer) - withUnsafeBytes(of: Int32(0).littleEndian) { - writer.append(Data($0)) - } - let footerDiff = (UInt32(writer.count) - UInt32(footerOffset)) - withUnsafeBytes(of: footerDiff.littleEndian) { - writer.append(Data($0)) - } - case .failure(let error): - return .failure(error) - } - case .failure(let error): - return .failure(error) - } - - return .success(true) - } - - public func writeStreaming( - _ info: ArrowWriter.Info - ) -> Result { - let writer: any DataWriter = InMemDataWriter() - switch toMessage(info.schema) { - case .success(let schemaData): - withUnsafeBytes(of: continuationMarker.littleEndian) { - writer.append(Data($0)) - } - withUnsafeBytes(of: UInt32(schemaData.count).littleEndian) { - writer.append(Data($0)) - } - writer.append(schemaData) - case .failure(let error): - return .failure(error) - } - - for batch in info.batches { - switch toMessage(batch) { - case .success(let batchData): - withUnsafeBytes(of: continuationMarker.littleEndian) { - writer.append(Data($0)) - } - withUnsafeBytes(of: UInt32(batchData[0].count).littleEndian) { - writer.append(Data($0)) - } - writer.append(batchData[0]) - writer.append(batchData[1]) - case .failure(let error): - return .failure(error) - } - } - withUnsafeBytes(of: continuationMarker.littleEndian) { - writer.append(Data($0)) - } - withUnsafeBytes(of: UInt32(0).littleEndian) { - writer.append(Data($0)) - } - if let memWriter = writer as? InMemDataWriter { - return .success(memWriter.data) - } else { - return .failure(.invalid("Unable to cast writer")) - } - } - - public func writeFile(_ info: ArrowWriter.Info) -> Result { - var writer: any DataWriter = InMemDataWriter() - switch writeFile(&writer, info: info) { - case .success: - if let memWriter = writer as? InMemDataWriter { - return .success(memWriter.data) - } else { - return .failure(.invalid("Unable to cast writer")) - } - case .failure(let error): - return .failure(error) - } - } - - public func toFile( - _ fileName: URL, - info: ArrowWriter.Info - ) -> Result { - do { - try Data().write(to: fileName) - } catch { - return .failure(.ioError("\(error)")) - } - guard let fileHandle = FileHandle(forUpdatingAtPath: fileName.path) else { - return .failure(.ioError("Unable to open \(fileName.path) for writing")) - } - defer { fileHandle.closeFile() } - - var markerData = fileMarker - addPadForAlignment(&markerData) - - var writer: any DataWriter = FileDataWriter(fileHandle) - writer.append(markerData) - switch writeFile(&writer, info: info) { - case .success: - writer.append(fileMarker) - case .failure(let error): - return .failure(error) - } - return .success(true) - } - - public func toMessage(_ batch: RecordBatchX) -> Result<[Data], ArrowError> { - var writer: any DataWriter = InMemDataWriter() - switch writeRecordBatch(batch: batch) { - case .success(let message): - writer.append(message) - addPadForAlignment(&writer) - var dataWriter: any DataWriter = InMemDataWriter() - switch writeRecordBatchData( - &dataWriter, fields: batch.schema.fields, columns: batch.columns) - { - case .success: - guard let inMemWriter = writer as? InMemDataWriter, - let inMemDataWriter = dataWriter as? InMemDataWriter - else { - return .failure(.invalid("Unable to cast writer")) - } - return .success([ - inMemWriter.data, - inMemDataWriter.data, - ]) - case .failure(let error): - return .failure(error) - } - case .failure(let error): - return .failure(error) - } - } - - public func toMessage(_ schema: ArrowSchema) -> Result { - var schemaSize: Int32 = 0 - var fbb = FlatBufferBuilder() - switch writeSchema(&fbb, schema: schema) { - case .success(let schemaOffset): - schemaSize = Int32(schemaOffset.o) - case .failure(let error): - return .failure(error) - } - - let startMessage = FMessage.startMessage(&fbb) - FMessage.add(bodyLength: Int64(0), &fbb) - FMessage.add(headerType: .schema, &fbb) - FMessage.add(header: Offset(offset: UOffset(schemaSize)), &fbb) - FMessage.add(version: .max, &fbb) - let messageOffset = FMessage.endMessage(&fbb, start: startMessage) - fbb.finish(offset: messageOffset) - return .success(fbb.data) - } -} diff --git a/Sources/Arrow/BitUtility.swift b/Sources/Arrow/BitUtility.swift deleted file mode 100644 index 55f19f7..0000000 --- a/Sources/Arrow/BitUtility.swift +++ /dev/null @@ -1,42 +0,0 @@ -// Copyright 2025 The Apache Software Foundation -// -// 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. - -import Foundation - -class BitUtility { - static func isSet(_ bit: UInt, buffer: ArrowBuffer) -> Bool { - let byteIndex = UInt(bit / 8) - let theByte = buffer.rawPointer.load( - fromByteOffset: Int(byteIndex), as: UInt8.self) - return theByte & UInt8(1 << (bit % 8)) > 0 - } - - static func setBit(_ bit: UInt, buffer: ArrowBuffer) { - let byteIndex = UInt(bit / 8) - var theByte = buffer.rawPointer.load( - fromByteOffset: Int(byteIndex), as: UInt8.self) - theByte |= UInt8(1 << (bit % 8)) - buffer.rawPointer.storeBytes( - of: theByte, toByteOffset: Int(byteIndex), as: UInt8.self) - } - - static func clearBit(_ bit: UInt, buffer: ArrowBuffer) { - let byteIndex = UInt(bit / 8) - var theByte = buffer.rawPointer.load( - fromByteOffset: Int(byteIndex), as: UInt8.self) - theByte &= ~(UInt8(1 << (bit % 8))) - buffer.rawPointer.storeBytes( - of: theByte, toByteOffset: Int(byteIndex), as: UInt8.self) - } -} diff --git a/Sources/Arrow/Buffer/ArrowBufferBackedOffsets.swift b/Sources/Arrow/Buffer/ArrowBufferBackedOffsets.swift deleted file mode 100644 index 75d4104..0000000 --- a/Sources/Arrow/Buffer/ArrowBufferBackedOffsets.swift +++ /dev/null @@ -1,40 +0,0 @@ -// Copyright 2025 The Columnar Swift Contributors -// -// 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. - -/// A buffer providing offsets, backed by an `ArrowBuffer` temporarily. -struct ArrowBufferBackedOffsets: OffsetsBuffer { - - let arrowBuffer: ArrowBuffer - - init(_ arrowBuffer: ArrowBuffer) { - self.arrowBuffer = arrowBuffer - } - - func offsets(at index: Int) -> (start: Int32, end: Int32) { - - let offsets = arrowBuffer - let offsetIndex = MemoryLayout.stride * Int(index) - var startIndex: Int32 = 0 - if index > 0 { - startIndex = offsets.rawPointer.advanced(by: offsetIndex) - .load(as: Int32.self) - } - let endIndex = offsets.rawPointer.advanced( - by: offsetIndex + MemoryLayout.stride - ) - .load(as: Int32.self) - - return (start: startIndex, end: endIndex) - } -} diff --git a/Sources/Arrow/Buffer/ArrowBufferProtocol.swift b/Sources/Arrow/Buffer/ArrowBufferProtocol.swift index e4d00de..6f42b4a 100644 --- a/Sources/Arrow/Buffer/ArrowBufferProtocol.swift +++ b/Sources/Arrow/Buffer/ArrowBufferProtocol.swift @@ -15,7 +15,7 @@ import Foundation /// An Arrow buffer. -public protocol ArrowBufferProtocol { +public protocol ArrowBufferProtocol: Sendable { var length: Int { get } func withUnsafeBytes( _ body: (UnsafeRawBufferPointer) throws -> R diff --git a/Sources/Arrow/Buffer/BinaryView.swift b/Sources/Arrow/Buffer/BinaryView.swift new file mode 100644 index 0000000..cdb6709 --- /dev/null +++ b/Sources/Arrow/Buffer/BinaryView.swift @@ -0,0 +1,113 @@ +// Copyright 2025 The Columnar Swift Contributors +// +// 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. + +/// A 16-byte view that either stores data inline or references a buffer. +@frozen public struct BinaryView { + private var words: InlineArray<4, Int32> + + private static let inlineThreshold = 12 + + public init() { + self.words = InlineArray(repeating: 0) + } + + public var length: Int32 { words[0] } + + private mutating func setLength(_ value: Int32) { + words[0] = value + } + + public var isInline: Bool { + length <= Self.inlineThreshold + } + + /// Access inline data for values ≤12 bytes. + public func withInlineData( + _ body: (Span) throws -> R + ) rethrows -> R { + precondition(isInline, "View is not inline.") + return try words.span.withUnsafeBytes { buffer in + // Skip first 4 bytes (length), read next `length` bytes + let dataBuffer = UnsafeRawBufferPointer( + start: buffer.baseAddress?.advanced(by: 4), + count: Int(length) + ) + let dataSpan = Span( + _unsafeElements: dataBuffer.bindMemory(to: UInt8.self) + ) + return try body(dataSpan) + } + } + + // MARK: - Referenced data access + + public var prefix: UInt32 { + precondition(!isInline, "View is inline.") + return UInt32(bitPattern: words.span[1]) + } + + public var bufferIndex: Int32 { + precondition(!isInline, "View is inline.") + return words.span[2] + } + + public var offset: Int32 { + precondition(!isInline, "View is inline.") + return words.span[3] + } + + // MARK: - Creation + + /// Create an inline view (for length ≤ 12). + public static func inline(_ data: Span) -> BinaryView { + precondition( + data.count <= inlineThreshold, + "Data too large for inline storage." + ) + var view = BinaryView() + view.setLength(Int32(data.count)) + var mutableSpan = view.words.mutableSpan + mutableSpan.withUnsafeMutableBytes { buffer in + // Copy data starting at byte 4 + data.withUnsafeBytes { sourceBytes in + let dest = UnsafeMutableRawBufferPointer( + start: buffer.baseAddress?.advanced(by: 4), + count: data.count + ) + dest.copyMemory(from: sourceBytes) + } + } + return view + } + + /// Create a referenced view (for length > 12) - trivial! + public static func referenced( + length: Int32, + prefix: UInt32, + bufferIndex: Int32, + offset: Int32 + ) -> BinaryView { + precondition( + length > inlineThreshold, + "Data small enough for inline storage." + ) + var view = BinaryView() + view.words[0] = length + view.words[1] = Int32(bitPattern: prefix) + view.words[2] = bufferIndex + view.words[3] = offset + + return view + } +} diff --git a/Sources/Arrow/Buffer/BorrowedBuffer.swift b/Sources/Arrow/Buffer/BorrowedBuffer.swift deleted file mode 100644 index b0cabd0..0000000 --- a/Sources/Arrow/Buffer/BorrowedBuffer.swift +++ /dev/null @@ -1,49 +0,0 @@ -// Copyright 2025 The Columnar Swift Contributors -// -// 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. - -import Foundation - -class BorrowedBuffer { - - let data: Data - let range: Range - - init(borrowing data: Data, range: Range) { - self.data = data - self.range = range - } -} - -class BorrowedOffsets: BorrowedBuffer, OffsetsBuffer { - - var count: Int - - init(count: Int, data: Data, range: Range) { - // Offsets are fenceposts. - precondition(count == range.count / MemoryLayout.stride - 1) - self.count = count - super.init(borrowing: data, range: range) - } - - func offsets(at index: Int) -> (start: Int32, end: Int32) { - precondition(index < count, "Index out of range") - return data.bytes.withUnsafeBytes { rawBuffer in - let sub = rawBuffer[range] - let span = Span(_unsafeBytes: sub) - let start = index > 0 ? span[index] : 0 - let end = span[index + 1] - return (start, end) - } - } -} diff --git a/Sources/Arrow/Buffer/FixedBufferBuilder.swift b/Sources/Arrow/Buffer/FixedBufferBuilder.swift index c346889..7219eec 100644 --- a/Sources/Arrow/Buffer/FixedBufferBuilder.swift +++ b/Sources/Arrow/Buffer/FixedBufferBuilder.swift @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -final class FixedWidthBufferBuilder { +final class FixedWidthBufferBuilder { var valueCount: Int var valueCapacity: Int private var buffer: UnsafeMutablePointer diff --git a/Sources/Arrow/Buffer/FixedWidthBuffer.swift b/Sources/Arrow/Buffer/FixedWidthBuffer.swift index a0023f1..e792dd8 100644 --- a/Sources/Arrow/Buffer/FixedWidthBuffer.swift +++ b/Sources/Arrow/Buffer/FixedWidthBuffer.swift @@ -13,22 +13,23 @@ // limitations under the License. public protocol FixedWidthBufferProtocol: ArrowBufferProtocol { - associatedtype ElementType: Numeric + associatedtype ElementType var length: Int { get } subscript(index: Int) -> ElementType { get } } /// A buffer used in Arrow arrays that hold fixed-width types. -final class FixedWidthBuffer: FixedWidthBufferProtocol -where T: Numeric { +public final class FixedWidthBuffer: @unchecked Sendable, + FixedWidthBufferProtocol +{ public typealias ElementType = T - public var length: Int - var capacity: Int + public let length: Int + let capacity: Int let valueCount: Int let ownsMemory: Bool - var buffer: UnsafePointer + let buffer: UnsafePointer - init( + public init( length: Int, capacity: Int, valueCount: Int, @@ -61,3 +62,26 @@ where T: Numeric { } } } + +extension FixedWidthBuffer { + + /// Build a fixed-width buffer from a fixed-width type array. + /// - Parameter values: The array to opy memory from. + /// - Returns: A buffer with the values copied into.. + public static func from(_ values: [T]) -> FixedWidthBuffer { + let count = values.count + let capacity = count * MemoryLayout.stride + let buffer = UnsafeMutablePointer.allocate(capacity: count) + // Copy values + for (index, value) in values.enumerated() { + buffer[index] = value + } + return FixedWidthBuffer( + length: capacity, + capacity: capacity, + valueCount: count, + ownsMemory: true, + buffer: UnsafePointer(buffer) + ) + } +} diff --git a/Sources/Arrow/Buffer/NullBuffer.swift b/Sources/Arrow/Buffer/NullBuffer.swift index 76ca0a8..b224941 100644 --- a/Sources/Arrow/Buffer/NullBuffer.swift +++ b/Sources/Arrow/Buffer/NullBuffer.swift @@ -53,13 +53,15 @@ public struct AllNullBuffer: NullBuffer, ArrowBufferEmpty { } /// A bit-packed buffer used to represent nulls and booleans in Arrow arrays. -final class BitPackedNullBuffer: NullBuffer, ArrowBufferUInt8 { +final class BitPackedNullBuffer: NullBuffer, ArrowBufferUInt8, + @unchecked Sendable +{ let length: Int let capacity: Int let valueCount: Int let ownsMemory: Bool let buffer: UnsafePointer - var nullCount: Int + let nullCount: Int init( length: Int, diff --git a/Sources/Arrow/Buffer/NullBufferBuilder.swift b/Sources/Arrow/Buffer/NullBufferBuilder.swift index 2414bf7..e58b499 100644 --- a/Sources/Arrow/Buffer/NullBufferBuilder.swift +++ b/Sources/Arrow/Buffer/NullBufferBuilder.swift @@ -13,7 +13,7 @@ // limitations under the License. /// A builder for bit-packed buffers used to represent nulls and booleans in Arrow arrays. -final class NullBufferBuilder { +public final class NullBufferBuilder { var length: Int var capacity: Int var bitCount: Int = 0 @@ -24,7 +24,7 @@ final class NullBufferBuilder { private var currentByte: UInt8 = 0 private var bitOffset: Int8 = 0 - init( + public init( length: Int = 0, minCapacity: Int = 64 ) { @@ -37,7 +37,7 @@ final class NullBufferBuilder { /// Appends a validity bit to the buffer. @inline(__always) - func appendValid(_ isValid: Bool) { + public func appendValid(_ isValid: Bool) { valueCount += 1 if isValid { currentByte |= 1 << bitOffset @@ -85,12 +85,11 @@ final class NullBufferBuilder { /// which always reports validity to be true, or if all nulls the inverse of this. Otherwise a buffer with /// capacity shrunk to a multiple of 64 bytes will be returned. /// - func finish() -> NullBuffer { + public func finish() -> NullBuffer { if bitOffset != 0 { flushByte() } precondition(ownsMemory, "Buffer already finished.") - if nullCount == 0 { return AllValidNullBuffer(valueCount: valueCount) } else if nullCount == valueCount { diff --git a/Sources/Arrow/Buffer/OffsetProtocol.swift b/Sources/Arrow/Buffer/OffsetProtocol.swift deleted file mode 100644 index 35cbe6d..0000000 --- a/Sources/Arrow/Buffer/OffsetProtocol.swift +++ /dev/null @@ -1,34 +0,0 @@ -// Copyright 2025 The Columnar Swift Contributors -// -// 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. - -/// A type which provides offset ranges in Arrow arrays. -protocol OffsetsBuffer { - - /// Get the start and end offsets for the element at index - /// - Parameter index: Zero-based index of the element - /// - Returns: Tuple of (start, end) offsets - func offsets(at index: Int) -> (start: Int32, end: Int32) - - /// Get just the length for the element at index - /// - Parameter index: Zero-based index of the element - /// - Returns: Length in bytes/elements - func length(at index: Int) -> Int32 -} - -extension OffsetsBuffer { - func length(at index: Int) -> Int32 { - let (start, end) = offsets(at: index) - return end - start - } -} diff --git a/Sources/Arrow/Buffer/VariableLength.swift b/Sources/Arrow/Buffer/VariableLength.swift index 3f6809f..815069d 100644 --- a/Sources/Arrow/Buffer/VariableLength.swift +++ b/Sources/Arrow/Buffer/VariableLength.swift @@ -50,16 +50,15 @@ public protocol VariableLengthBufferProtocol: ArrowBufferProtocol { at startIndex: Int, arrayLength: Int ) -> ElementType - } /// A buffer containing values with variable length, used in variable length type Arrow arrays. public final class VariableLengthTypeBuffer: - VariableLengthBufferProtocol, ArrowBufferUInt8 + VariableLengthBufferProtocol, ArrowBufferUInt8, @unchecked Sendable { public typealias ElementType = T - public var length: Int - var capacity: Int + public let length: Int + let capacity: Int let ownsMemory: Bool var buffer: UnsafePointer diff --git a/Sources/Arrow/ChunkedArray.swift b/Sources/Arrow/ChunkedArray.swift deleted file mode 100644 index 90bd7e1..0000000 --- a/Sources/Arrow/ChunkedArray.swift +++ /dev/null @@ -1,82 +0,0 @@ -// Copyright 2025 The Apache Software Foundation -// -// 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. - -import Foundation - -public class ChunkedArrayHolder { - public let type: ArrowType - public let length: UInt - public let nullCount: UInt - public let holder: Any - - public init(_ chunked: ChunkedArray) { - self.holder = chunked - self.length = chunked.length - self.type = chunked.type - self.nullCount = chunked.nullCount - } -} - -public class ChunkedArray { - public let arrays: [any ArrowArray] - public let type: ArrowType - public let nullCount: UInt - public let length: UInt - public var arrayCount: UInt { UInt(self.arrays.count) } - - public init(_ arrays: [any ArrowArray]) throws(ArrowError) { - if arrays.count == 0 { - throw ArrowError.arrayHasNoElements - } - - self.type = arrays[0].type - var len: UInt = 0 - var nullCount: UInt = 0 - for array in arrays { - len += array.length - nullCount += array.nullCount - } - - self.arrays = arrays - self.length = len - self.nullCount = nullCount - } - - public subscript(_ index: UInt) -> T? { - if arrays.count == 0 { - return nil - } - var localIndex = index - var arrayIndex = 0 - var len: UInt = arrays[arrayIndex].length - while localIndex > (len - 1) { - arrayIndex += 1 - if arrayIndex > arrays.count { - return nil - } - - localIndex -= len - len = arrays[arrayIndex].length - } - - return arrays[arrayIndex][localIndex] - } - - public func asString(_ index: UInt) -> String { - guard let value = self[index] else { - return "" - } - return "\(value)" - } -} diff --git a/Sources/Arrow/FlatBuffersHelpers.swift b/Sources/Arrow/FlatBuffersHelpers.swift deleted file mode 100644 index dff242d..0000000 --- a/Sources/Arrow/FlatBuffersHelpers.swift +++ /dev/null @@ -1,153 +0,0 @@ -// Copyright 2025 The Apache Software Foundation -// Copyright 2025 The Columnar-Swift Contributors -// -// 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. - -import Foundation - -func isFixedPrimitive(_ type: FType) -> Bool { - switch type { - case .int, .bool, .floatingpoint, .date, .time, .timestamp: - return true - default: - return false - } -} - -func findArrowType(_ field: FField) throws(ArrowError) -> ArrowType { - let type = field.typeType - switch type { - case .int: - guard let intType = field.type(type: FInt.self) else { - throw .invalid("Could not get integer type from \(field)") - } - let bitWidth = intType.bitWidth - if bitWidth == 8 { - if intType.isSigned { - return .int8 - } else { - return .uint8 - } - } - if bitWidth == 16 { - return intType.isSigned ? .int16 : .uint16 - } - if bitWidth == 32 { - return intType.isSigned ? .int32 : .uint32 - } - if bitWidth == 64 { - return intType.isSigned ? .int64 : .uint64 - } - throw .invalid("Unhandled integer bit width: \(bitWidth)") - case .bool: - return .boolean - case .floatingpoint: - guard let floatType = field.type(type: FFloatingPoint.self) else { - throw .invalid("Could not get floating point type from field") - } - switch floatType.precision { - case .half: - return .float16 - case .single: - return .float32 - case .double: - return .float64 - } - case .utf8: - return .utf8 - case .binary: - return .binary - case .date: - guard let dateType = field.type(type: FDate.self) else { - throw .invalid("Could not get date type from field") - } - if dateType.unit == .day { - return .date32 - } - return .date64 - case .time: - guard let timeType = field.type(type: FTime.self) else { - throw .invalid("Could not get time type from field") - } - if timeType.unit == .second || timeType.unit == .millisecond { - return .time32( - timeType.unit == .second ? .second : .millisecond - ) - } - return .time64( - timeType.unit == .microsecond ? .microsecond : .nanosecond - ) - case .timestamp: - guard let timestampType = field.type(type: FTimestamp.self) else { - throw .invalid("Could not get timestamp type from field") - } - let arrowUnit: TimeUnit - switch timestampType.unit { - case .second: - arrowUnit = .second - case .millisecond: - arrowUnit = .millisecond - case .microsecond: - arrowUnit = .microsecond - case .nanosecond: - arrowUnit = .nanosecond - } - let timezone = timestampType.timezone - return .timestamp(arrowUnit, timezone) - case .struct_: - guard field.type(type: FStruct.self) != nil else { - throw .invalid("Could not get struct type from field") - } - var fields: [ArrowField] = [] - for index in 0..= recordBatch.buffersCount { - throw ArrowError.outOfBounds(index: Int64(index)) - } -} diff --git a/Sources/Arrow/ProtoUtil.swift b/Sources/Arrow/ProtoUtil.swift index 47965da..8674c00 100644 --- a/Sources/Arrow/ProtoUtil.swift +++ b/Sources/Arrow/ProtoUtil.swift @@ -22,7 +22,7 @@ func fromProto( switch type { case .int: guard let intType = field.type(type: FInt.self) else { - throw .invalid("Invalid FlatBuffer: \(field)") + throw .init(.invalid("Invalid FlatBuffer: \(field)")) } let bitWidth = intType.bitWidth if bitWidth == 8 { @@ -38,7 +38,7 @@ func fromProto( arrowType = .boolean case .floatingpoint: guard let floatType = field.type(type: FFloatingPoint.self) else { - throw .invalid("Invalid FlatBuffer: \(field)") + throw .init(.invalid("Invalid FlatBuffer: \(field)")) } switch floatType.precision { case .half: @@ -54,7 +54,7 @@ func fromProto( arrowType = .binary case .date: guard let dateType = field.type(type: FDate.self) else { - throw .invalid("Invalid FlatBuffer: \(field)") + throw .init(.invalid("Invalid FlatBuffer: \(field)")) } if dateType.unit == .day { arrowType = .date32 @@ -63,7 +63,7 @@ func fromProto( } case .time: guard let timeType = field.type(type: FTime.self) else { - throw .invalid("Invalid FlatBuffer: \(field)") + throw .init(.invalid("Invalid FlatBuffer: \(field)")) } if timeType.unit == .second || timeType.unit == .millisecond { let arrowUnit: TimeUnit = @@ -76,7 +76,7 @@ func fromProto( } case .timestamp: guard let timestampType = field.type(type: FTimestamp.self) else { - throw .invalid("Invalid FlatBuffer: \(field)") + throw .init(.invalid("Invalid FlatBuffer: \(field)")) } let arrowUnit: TimeUnit switch timestampType.unit { @@ -94,7 +94,8 @@ func fromProto( var children: [ArrowField] = [] for index in 0.. - ) async throws -> ArrowReaderX.ArrowReaderResult { - let reader = ArrowReaderX() - let arrowResult = ArrowReaderX.makeArrowReaderResult() - for try await data in responseStream { - switch reader.fromMessage( - data.dataHeader, - dataBody: data.dataBody, - result: arrowResult, - useUnalignedBuffers: allowReadingUnalignedBuffers) - { - case .success: - continue - case .failure(let error): - throw error - } - } - - return arrowResult - } + // private func readMessages( + // _ responseStream: GRPCAsyncResponseStream + // ) async throws -> ArrowReaderX.ArrowReaderResult { + // let reader = ArrowReaderX() + // let arrowResult = ArrowReaderX.makeArrowReaderResult() + // for try await data in responseStream { + // switch reader.fromMessage( + // data.dataHeader, + // dataBody: data.dataBody, + // result: arrowResult, + // useUnalignedBuffers: allowReadingUnalignedBuffers) + // { + // case .success: + // continue + // case .failure(let error): + // throw error + // } + // } + // + // return arrowResult + // } private func writeBatches( _ requestStream: GRPCAsyncRequestStreamWriter, descriptor: FlightDescriptor, - recordBatches: [RecordBatchX] + recordBatches: [RecordBatch] ) async throws { - let writer = ArrowWriter() - switch writer.toMessage(recordBatches[0].schema) { - case .success(let schemaData): - try await requestStream.send( - FlightData( - schemaData, - dataBody: Data(), - flightDescriptor: descriptor - ).toProtocol()) - for recordBatch in recordBatches { - switch writer.toMessage(recordBatch) { - case .success(let data): - try await requestStream.send( - FlightData( - data[0], - dataBody: data[1], - flightDescriptor: descriptor - ).toProtocol()) - case .failure(let error): - throw error - } - } - requestStream.finish() - case .failure(let error): - throw error - } + // let writer = ArrowWriter() + // switch writer.toMessage(recordBatches[0].schema) { + // case .success(let schemaData): + // try await requestStream.send( + // FlightData( + // schemaData, + // dataBody: Data(), + // flightDescriptor: descriptor + // ).toProtocol()) + // for recordBatch in recordBatches { + // switch writer.toMessage(recordBatch) { + // case .success(let data): + // try await requestStream.send( + // FlightData( + // data[0], + // dataBody: data[1], + // flightDescriptor: descriptor + // ).toProtocol()) + // case .failure(let error): + // throw error + // } + // } + // requestStream.finish() + // case .failure(let error): + // throw error + // } + fatalError() } public func listActions(_ closure: (FlightActionType) -> Void) async throws { @@ -119,13 +120,13 @@ public class FlightClient { return FlightSchemaResult(try await schemaResultResponse.response) } - public func doGet( - _ ticket: FlightTicket, - readerResultClosure: (ArrowReaderX.ArrowReaderResult) throws -> Void - ) async throws { - let getResult = client.makeDoGetCall(ticket.toProtocol()) - try readerResultClosure(try await readMessages(getResult.responseStream)) - } + // public func doGet( + // _ ticket: FlightTicket, + // readerResultClosure: (ArrowReaderX.ArrowReaderResult) throws -> Void + // ) async throws { + // let getResult = client.makeDoGetCall(ticket.toProtocol()) + // try readerResultClosure(try await readMessages(getResult.responseStream)) + // } public func doGet( _ ticket: FlightTicket, @@ -139,7 +140,7 @@ public class FlightClient { public func doPut( _ descriptor: FlightDescriptor, - recordBatches: [RecordBatchX], + recordBatches: [RecordBatch], closure: (FlightPutResult) throws -> Void ) async throws { if recordBatches.isEmpty { @@ -178,21 +179,21 @@ public class FlightClient { } } - public func doExchange( - _ descriptor: FlightDescriptor, - recordBatches: [RecordBatchX], - closure: (ArrowReaderX.ArrowReaderResult) throws -> Void - ) async throws { - if recordBatches.isEmpty { - throw ArrowFlightError.emptyCollection - } - - let exchangeCall = client.makeDoExchangeCall() - try await writeBatches( - exchangeCall.requestStream, descriptor: descriptor, - recordBatches: recordBatches) - try closure(try await readMessages(exchangeCall.responseStream)) - } + // public func doExchange( + // _ descriptor: FlightDescriptor, + // recordBatches: [RecordBatchX], + // closure: (ArrowReaderX.ArrowReaderResult) throws -> Void + // ) async throws { + // if recordBatches.isEmpty { + // throw ArrowFlightError.emptyCollection + // } + // + // let exchangeCall = client.makeDoExchangeCall() + // try await writeBatches( + // exchangeCall.requestStream, descriptor: descriptor, + // recordBatches: recordBatches) + // try closure(try await readMessages(exchangeCall.responseStream)) + // } public func doExchange( flightData: FlightData, closure: (FlightData) throws -> Void diff --git a/Sources/ArrowFlight/FlightServer.swift b/Sources/ArrowFlight/FlightServer.swift index 5eb9ab0..3b48ab0 100644 --- a/Sources/ArrowFlight/FlightServer.swift +++ b/Sources/ArrowFlight/FlightServer.swift @@ -28,19 +28,20 @@ public enum ArrowFlightError: Error { } public func schemaToMessage(_ schema: ArrowSchema) throws -> Data { - let arrowWriter = ArrowWriter() - switch arrowWriter.toMessage(schema) { - case .success(let result): - var outputResult = Data() - withUnsafeBytes(of: Int32(0).littleEndian) { outputResult.append(Data($0)) } - withUnsafeBytes(of: Int32(result.count).littleEndian) { - outputResult.append(Data($0)) - } - outputResult.append(result) - return outputResult - case .failure(let error): - throw error - } + fatalError() + // let arrowWriter = ArrowWriter() + // switch arrowWriter.toMessage(schema) { + // case .success(let result): + // var outputResult = Data() + // withUnsafeBytes(of: Int32(0).littleEndian) { outputResult.append(Data($0)) } + // withUnsafeBytes(of: Int32(result.count).littleEndian) { + // outputResult.append(Data($0)) + // } + // outputResult.append(result) + // return outputResult + // case .failure(let error): + // throw error + // } } public func schemaFromMessage(_ schemaData: Data) throws(ArrowFlightError) @@ -51,20 +52,21 @@ public func schemaFromMessage(_ schemaData: Data) throws(ArrowFlightError) } let startIndex = schemaData.count - Int(messageLength) - let schema = schemaData[startIndex...] + let _ = schemaData[startIndex...] - let reader = ArrowReaderX() - let result = ArrowReaderX.makeArrowReaderResult() - switch reader.fromMessage(schema, dataBody: Data(), result: result) { - case .success: - if let resultSchema = result.schema { - return resultSchema - } else { - throw .unknown("Unable to parse Arrow schema from data") - } - case .failure: - throw .unknown("Unable to parse Arrow schema from data") - } + // let reader = ArrowReaderX() + // let result = ArrowReaderX.makeArrowReaderResult() + // switch reader.fromMessage(schema, dataBody: Data(), result: result) { + // case .success: + // if let resultSchema = result.schema { + // return resultSchema + // } else { + // throw .unknown("Unable to parse Arrow schema from data") + // } + // case .failure: + // throw .unknown("Unable to parse Arrow schema from data") + // } + fatalError() } public protocol ArrowFlightServer: Sendable { diff --git a/Sources/ArrowFlight/RecordBatchStreamReader.swift b/Sources/ArrowFlight/RecordBatchStreamReader.swift index 5d4f068..5c124a8 100644 --- a/Sources/ArrowFlight/RecordBatchStreamReader.swift +++ b/Sources/ArrowFlight/RecordBatchStreamReader.swift @@ -14,6 +14,7 @@ // limitations under the License. import Arrow +import ArrowIPC import Foundation import GRPC @@ -21,10 +22,10 @@ public struct RecordBatchStreamReader: AsyncSequence, AsyncIteratorProtocol, Sendable { public typealias AsyncIterator = RecordBatchStreamReader - public typealias Element = (Arrow.RecordBatchX?, FlightDescriptor?) - let reader = ArrowReaderX() + public typealias Element = (RecordBatch?, FlightDescriptor?) + // let reader = ArrowReader(url: URL(fileURLWithPath: "/dev/null")) // FIXME: this is hack to make this sendable - nonisolated(unsafe) var batches: [RecordBatchX] = [] + nonisolated(unsafe) var batches: [RecordBatch] = [] nonisolated(unsafe) var streamIterator: any AsyncIteratorProtocol var descriptor: FlightDescriptor? var batchIndex = 0 @@ -40,7 +41,7 @@ public struct RecordBatchStreamReader: AsyncSequence, AsyncIteratorProtocol, } public mutating func next() async throws -> ( - Arrow.RecordBatchX?, FlightDescriptor? + Arrow.RecordBatch?, FlightDescriptor? )? { guard !Task.isCancelled else { return nil @@ -52,7 +53,7 @@ public struct RecordBatchStreamReader: AsyncSequence, AsyncIteratorProtocol, return (batch, descriptor) } - let result = ArrowReaderX.makeArrowReaderResult() + var result: [RecordBatch] = [] while true { let streamData = try await self.streamIterator.next() if streamData == nil { @@ -66,21 +67,23 @@ public struct RecordBatchStreamReader: AsyncSequence, AsyncIteratorProtocol, let dataBody = flightData.dataBody let dataHeader = flightData.dataHeader descriptor = FlightDescriptor(flightData.flightDescriptor) - switch reader.fromMessage( - dataHeader, - dataBody: dataBody, - result: result, - useUnalignedBuffers: useUnalignedBuffers) - { - case .success(()): - if result.batches.count > 0 { - batches = result.batches - batchIndex = 1 - return (batches[0], descriptor) - } - case .failure(let error): - throw error - } + + // TODO: streaming + // switch reader.fromMessage( + // dataHeader, + // dataBody: dataBody, + // result: result, + // useUnalignedBuffers: useUnalignedBuffers) + // { + // case .success(()): + // if result.count > 0 { + // batches = result + // batchIndex = 1 + // return (batches[0], descriptor) + // } + // case .failure(let error): + // throw error + // } } } diff --git a/Sources/ArrowFlight/RecordBatchStreamWriter.swift b/Sources/ArrowFlight/RecordBatchStreamWriter.swift index 423e722..f848499 100644 --- a/Sources/ArrowFlight/RecordBatchStreamWriter.swift +++ b/Sources/ArrowFlight/RecordBatchStreamWriter.swift @@ -57,28 +57,29 @@ public struct RecordBatchStreamWriter: Sendable { self.stream = stream } - public func write(_ rb: RecordBatchX) async throws { - // FIXME: this was moved here to make this sendable. - let writer = ArrowWriter() - switch writer.toMessage(rb.schema) { - case .success(let schemaData): - let schemaFlightData = ProtoFlightData.with { - $0.dataHeader = schemaData - } - - try await self.stream.send(schemaFlightData) - switch writer.toMessage(rb) { - case .success(let recordMessages): - let rbMessage = ProtoFlightData.with { - $0.dataHeader = recordMessages[0] - $0.dataBody = recordMessages[1] - } - try await self.stream.send(rbMessage) - case .failure(let error): - throw error - } - case .failure(let error): - throw error - } + public func write(_ rb: RecordBatch) async throws { + // // FIXME: this was moved here to make this sendable. + // let writer = ArrowWriter() + // switch writer.toMessage(rb.schema) { + // case .success(let schemaData): + // let schemaFlightData = ProtoFlightData.with { + // $0.dataHeader = schemaData + // } + // + // try await self.stream.send(schemaFlightData) + // switch writer.toMessage(rb) { + // case .success(let recordMessages): + // let rbMessage = ProtoFlightData.with { + // $0.dataHeader = recordMessages[0] + // $0.dataBody = recordMessages[1] + // } + // try await self.stream.send(rbMessage) + // case .failure(let error): + // throw error + // } + // case .failure(let error): + // throw error + // } + fatalError() } } diff --git a/Sources/ArrowIPC/Array+IPC.swift b/Sources/ArrowIPC/Array+IPC.swift deleted file mode 100644 index 83bfb71..0000000 --- a/Sources/ArrowIPC/Array+IPC.swift +++ /dev/null @@ -1,79 +0,0 @@ -// Copyright 2025 The Columnar Swift Contributors -// -// 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. - -import Arrow -import Foundation - -///// A `Data` backed Arrow utf8 array. -//typealias ArrowArrayUtf8 = ArrowArrayVariable< -// FixedWidthBufferIPC, -// VariableLengthBufferIPC -//> -// -//extension ArrowArrayUtf8 { -// -// /// Build a `Data` backed Arrow utf8 array. -// /// - Parameters: -// /// - length: The array length. -// /// - nullBuffer: The null buffer. -// /// - offsetsBuffer: A view over file-backed data. -// /// - valueBuffer: A view over file-backed data. -// /// - Returns: A file-backed Arrow utf8 array. -// static func utf8( -// length: Int, -// nullBuffer: NullBuffer, -// offsetsBuffer: FileDataBuffer, -// valueBuffer: FileDataBuffer -// ) -> Self { -// let offsetsBufferTyped = FixedWidthBufferIPC(buffer: offsetsBuffer) -// let valueBufferTyped = VariableLengthBufferIPC(buffer: valueBuffer) -// return Self( -// length: length, -// nullBuffer: nullBuffer, -// offsetsBuffer: offsetsBufferTyped, -// valueBuffer: valueBufferTyped -// ) -// } -//} -// -//typealias ArrowArrayBinary = ArrowArrayVariable< -// FixedWidthBufferIPC, -// VariableLengthBufferIPC -//> -// -//extension ArrowArrayBinary { -// -// /// Build a `Data` backed Arrow binary array. -// /// - Parameters: -// /// - length: The array length. -// /// - nullBuffer: The null buffer. -// /// - offsetsBuffer: A view over file-backed data. -// /// - valueBuffer: A view over file-backed data. -// /// - Returns: A file-backed Arrow utf8 array. -// static func binary( -// length: Int, -// nullBuffer: NullBuffer, -// offsetsBuffer: FileDataBuffer, -// valueBuffer: FileDataBuffer -// ) -> Self { -// let offsetsBufferTyped = FixedWidthBufferIPC(buffer: offsetsBuffer) -// let valueBufferTyped = VariableLengthBufferIPC(buffer: valueBuffer) -// return Self( -// length: length, -// nullBuffer: nullBuffer, -// offsetsBuffer: offsetsBufferTyped, -// valueBuffer: valueBufferTyped -// ) -// } -//} diff --git a/Sources/ArrowIPC/ArrowBufferIPC.swift b/Sources/ArrowIPC/ArrowBufferIPC.swift new file mode 100644 index 0000000..f450a73 --- /dev/null +++ b/Sources/ArrowIPC/ArrowBufferIPC.swift @@ -0,0 +1,100 @@ +// Copyright 2025 The Columnar Swift Contributors +// +// 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. + +import Arrow + +/// An Arrow buffer backed by file data. +internal protocol ArrowBufferIPC: ArrowBufferProtocol { + var buffer: FileDataBuffer { get } +} + +extension ArrowBufferIPC { + public func withUnsafeBytes( + _ body: (UnsafeRawBufferPointer) throws -> R + ) rethrows -> R { + try buffer.data.withUnsafeBytes { dataPtr in + let rangedPtr = UnsafeRawBufferPointer( + rebasing: dataPtr[buffer.range] + ) + return try body(rangedPtr) + } + } +} + +/// A `Data` backed buffer for null bitmaps and boolean arrays. +struct NullBufferIPC: NullBuffer, ArrowBufferIPC { + + let buffer: FileDataBuffer + var valueCount: Int + let nullCount: Int + + var length: Int { (valueCount + 7) / 8 } + + func isSet(_ bit: Int) -> Bool { + precondition(bit < valueCount, "Bit index \(bit) out of range") + let byteIndex = bit / 8 + // precondition(length > byteIndex, "Bit index \(bit) out of range") + let offsetIndex = buffer.range.lowerBound + byteIndex + let byte = self.buffer.data[offsetIndex] + return byte & (1 << (bit % 8)) > 0 + } +} + +/// A `Data` backed buffer for fixed-width types. +struct FixedWidthBufferIPC: FixedWidthBufferProtocol, ArrowBufferIPC +where Element: BitwiseCopyable { + typealias ElementType = Element + let buffer: FileDataBuffer + var length: Int { buffer.range.count } + + init(buffer: FileDataBuffer) { + self.buffer = buffer + } + + subscript(index: Int) -> Element { + buffer.data.withUnsafeBytes { rawBuffer in + let sub = rawBuffer[buffer.range] + let span = Span(_unsafeBytes: sub) + return span[index] + } + } +} + +/// A `Data` backed buffer for variable-length types. +struct VariableLengthBufferIPC< + Element: VariableLength, OffsetType: FixedWidthInteger +>: + VariableLengthBufferProtocol, ArrowBufferIPC +{ + typealias ElementType = Element + let buffer: FileDataBuffer + var length: Int { buffer.range.count } + init(buffer: FileDataBuffer) { + self.buffer = buffer + } + + func loadVariable( + at startIndex: Int, + arrayLength: Int + ) -> Element { + precondition(startIndex + arrayLength <= self.length) + return buffer.data.withUnsafeBytes { rawBuffer in + let offsetStart = buffer.range.lowerBound + startIndex + let offsetEnd = offsetStart + arrayLength + let slice = rawBuffer[offsetStart.. FTimeUnit { + switch self { + case .second: return .second + case .millisecond: return .millisecond + case .microsecond: return .microsecond + case .nanosecond: return .nanosecond + } + } +} + +extension ArrowField { + + /// Parses an `ArrowField` from the FlatBuffers representation. + /// - Parameter field: + /// - Returns: The `ArrowField`. + static func parse(from field: FField) throws(ArrowError) -> Self { + let fieldType: ArrowType = try .parse(from: field) + guard let fieldName = field.name else { + throw .init(.invalid("Field name not found")) + } + let fieldMetadata = (0.. Self { + /// - Returns: The `ArrowType`, including all nested fields which are parsed recursively. + /// - Throws: An `ArrowError` if parsing fails. + static func parse(from field: FField) throws(ArrowError) -> Self { let type = field.typeType switch type { case .int: guard let intType = field.type(type: FInt.self) else { - throw .invalid("Could not get integer type from \(field)") + throw .init(.invalid("Could not get integer type from \(field)")) } let bitWidth = intType.bitWidth if bitWidth == 8 { @@ -45,12 +82,12 @@ extension ArrowType { if bitWidth == 64 { return intType.isSigned ? .int64 : .uint64 } - throw .invalid("Unhandled integer bit width: \(bitWidth)") + throw .init(.invalid("Unhandled integer bit width: \(bitWidth)")) case .bool: return .boolean case .floatingpoint: guard let floatType = field.type(type: FFloatingPoint.self) else { - throw .invalid("Could not get floating point type from field") + throw .init(.invalid("Could not get floating point type from field")) } switch floatType.precision { case .half: @@ -62,16 +99,21 @@ extension ArrowType { } case .utf8: return .utf8 + case .utf8view: + return .utf8View case .binary: return .binary + case .binaryview: + return .binaryView case .fixedsizebinary: guard let fType = field.type(type: FFixedSizeBinary.self) else { - throw .invalid("Could not get byteWidth from fixed binary field.") + throw .init( + .invalid("Could not get byteWidth from fixed binary field.")) } return .fixedSizeBinary(fType.byteWidth) case .date: guard let dateType = field.type(type: FDate.self) else { - throw .invalid("Could not get date type from field") + throw .init(.invalid("Could not get date type from field")) } if dateType.unit == .day { return .date32 @@ -79,7 +121,7 @@ extension ArrowType { return .date64 case .time: guard let timeType = field.type(type: FTime.self) else { - throw .invalid("Could not get time type from field") + throw .init(.invalid("Could not get time type from field")) } if timeType.unit == .second || timeType.unit == .millisecond { return .time32( @@ -89,9 +131,23 @@ extension ArrowType { return .time64( timeType.unit == .microsecond ? .microsecond : .nanosecond ) + case .duration: + guard let durationType = field.type(type: FDuration.self) else { + throw .init(.invalid("Could not get duration type from field")) + } + switch durationType.unit { + case .second: + return .duration(.second) + case .millisecond: + return .duration(.millisecond) + case .microsecond: + return .duration(.microsecond) + case .nanosecond: + return .duration(.nanosecond) + } case .timestamp: guard let timestampType = field.type(type: FTimestamp.self) else { - throw .invalid("Could not get timestamp type from field") + throw .init(.invalid("Could not get timestamp type from field")) } let arrowUnit: TimeUnit switch timestampType.unit { @@ -108,12 +164,13 @@ extension ArrowType { return .timestamp(arrowUnit, timezone) case .struct_: guard field.type(type: FStruct.self) != nil else { - throw .invalid("Could not get struct type from field") + throw .init(.invalid("Could not get struct type from field")) } var fields: [ArrowField] = [] for index in 0.. FType { switch self { case .int8, .int16, .int32, .int64, .uint8, .uint16, .uint32, .uint64: return .int case .float16, .float32, .float64: return .floatingpoint - case .utf8: - return .utf8 case .binary: return .binary + case .binaryView: + return .binaryview + case .utf8: + return .utf8 + case .utf8View: + return .utf8view case .boolean: return .bool case .date32, .date64: @@ -160,10 +241,20 @@ extension ArrowType { return .time case .timestamp: return .timestamp + case .duration: + return .duration case .strct: return .struct_ + case .list: + return .list + case .map: + return .map + case .fixedSizeBinary: + return .fixedsizebinary + case .fixedSizeList: + return .fixedsizelist default: - throw .invalid("Unhandled field type: \(self)") + throw .init(.invalid("Unhandled field type: \(self)")) } } } diff --git a/Sources/ArrowIPC/ArrowReader.swift b/Sources/ArrowIPC/ArrowReader.swift index cb21629..1cb1092 100644 --- a/Sources/ArrowIPC/ArrowReader.swift +++ b/Sources/ArrowIPC/ArrowReader.swift @@ -25,87 +25,11 @@ let continuationMarker = UInt32(0xFFFF_FFFF) struct FileDataBuffer { let data: Data let range: Range -} - -/// An Arrow buffer backed by file data. -internal protocol ArrowBufferIPC: ArrowBufferProtocol { - var buffer: FileDataBuffer { get } -} - -extension ArrowBufferIPC { - public func withUnsafeBytes( - _ body: (UnsafeRawBufferPointer) throws -> R - ) rethrows -> R { - try buffer.data.withUnsafeBytes { dataPtr in - let rangedPtr = UnsafeRawBufferPointer( - rebasing: dataPtr[buffer.range] - ) - return try body(rangedPtr) - } - } -} - -/// A `Data` backed buffer for null bitmaps and boolean arrays. -struct NullBufferIPC: NullBuffer, ArrowBufferIPC { - - let buffer: FileDataBuffer - var valueCount: Int - let nullCount: Int - - var length: Int { - buffer.range.count - } - func isSet(_ bit: Int) -> Bool { - precondition(bit < valueCount, "Bit index \(bit) out of range") - let byteIndex = bit / 8 - // precondition(length > byteIndex, "Bit index \(bit) out of range") - let offsetIndex = buffer.range.lowerBound + byteIndex - let byte = self.buffer.data[offsetIndex] - return byte & (1 << (bit % 8)) > 0 - } -} - -/// A `Data` backed buffer for fixed-width types. -struct FixedWidthBufferIPC: FixedWidthBufferProtocol, ArrowBufferIPC -where - Element: Numeric, Element: BitwiseCopyable -{ - typealias ElementType = Element - let buffer: FileDataBuffer - var length: Int { buffer.range.count } - - subscript(index: Int) -> Element { - buffer.data.withUnsafeBytes { rawBuffer in - let sub = rawBuffer[buffer.range] - let span = Span(_unsafeBytes: sub) - return span[index] - } - } -} - -/// A `Data` backed buffer for variable-length types. -struct VariableLengthBufferIPC< - Element: VariableLength, OffsetType: FixedWidthInteger ->: - VariableLengthBufferProtocol, ArrowBufferIPC -{ - typealias ElementType = Element - let buffer: FileDataBuffer - var length: Int { buffer.range.count } - - func loadVariable( - at startIndex: Int, - arrayLength: Int - ) -> Element { - precondition(startIndex + arrayLength <= self.length) - return buffer.data.withUnsafeBytes { rawBuffer in - let offsetStart = buffer.range.lowerBound + startIndex - let offsetEnd = offsetStart + arrayLength - let slice = rawBuffer[offsetStart..) { + self.data = data + self.range = range + precondition(range.lowerBound <= range.upperBound) } } @@ -122,26 +46,35 @@ public struct ArrowReader { /// - Parameter url: the file to read from. /// - Throws: a ParsingError if the file could not be read. public init(url: URL) throws { - self.data = try Data(contentsOf: url, options: .mappedIfSafe) + try validateFileMarker() + } + /// Create an `ArrowReader` from Arrow IPC data. + /// + /// - Parameter data: Arrow IPC format data (file or stream format). + /// - Throws: a ParsingError if the data is not valid Arrow IPC format. + public init(data: Data) throws { + self.data = data + try validateFileMarker() + } + + private func validateFileMarker() throws { try data.withParserSpan { input in let marker = try [UInt8](parsing: &input, byteCount: 6) guard marker == fileMarker else { - throw ArrowError.invalid("Invalid Arrow file") + throw ArrowError(.invalid("Invalid Arrow file")) } } } - func read() throws -> (ArrowSchema, [RecordBatch]) { + public func read() throws -> (ArrowSchema, [RecordBatch]) { let footerData = try data.withParserSpan { input in let count = input.count try input.seek(toAbsoluteOffset: count - 10) - let footerLength = try Int(parsingLittleEndian: &input, byteCount: 4) try input.seek(toAbsoluteOffset: count - 10 - footerLength) - return try [UInt8](parsing: &input, byteCount: footerLength) } @@ -152,7 +85,7 @@ public struct ArrowReader { let footer: FFooter = getRoot(byteBuffer: &footerBuffer) guard let schema = footer.schema else { - throw ArrowError.invalid("Missing schema in footer") + throw ArrowError(.invalid("Missing schema in footer")) } let arrowSchema = try loadSchema(schema: schema) var recordBatches: [RecordBatch] = [] @@ -160,14 +93,14 @@ public struct ArrowReader { // MARK: Record batch parsing for index in 0.. AnyArrowArrayProtocol { + bufferIndex: inout Int32, + variadicBufferIndex: inout Int32, + ) throws(ArrowError) -> AnyArrowArrayProtocol { guard nodeIndex < rbMessage.nodesCount, let node = rbMessage.nodes(at: nodeIndex) else { - throw ArrowError.invalid("Missing node at index \(nodeIndex)") + throw ArrowError(.invalid("Missing node at index \(nodeIndex)")) } nodeIndex += 1 let buffer0 = try nextBuffer( @@ -312,19 +248,55 @@ public struct ArrowReader { length: length, elementType: UInt64.self, nullBuffer: nullBuffer, buffer: buffer1) default: - throw ArrowError.invalid("TODO: Unimplemented arrow type: \(arrowType)") + throw .init(.notImplemented) + } + } else if arrowType.isTemporal { + let buffer1 = try nextBuffer( + message: rbMessage, index: &bufferIndex, offset: offset, data: data) + switch arrowType { + case .date32: + return makeFixedArray( + length: length, elementType: Int32.self, + nullBuffer: nullBuffer, buffer: buffer1) + case .date64: + return makeFixedArray( + length: length, elementType: Int64.self, + nullBuffer: nullBuffer, buffer: buffer1) + case .time32(_): + return makeFixedArray( + length: length, elementType: UInt32.self, + nullBuffer: nullBuffer, buffer: buffer1) + case .time64(_): + return makeFixedArray( + length: length, elementType: UInt64.self, + nullBuffer: nullBuffer, buffer: buffer1) + case .timestamp(_, _): + return makeFixedArray( + length: length, elementType: Int64.self, + nullBuffer: nullBuffer, buffer: buffer1) + case .duration(_): + return makeFixedArray( + length: length, elementType: Int64.self, + nullBuffer: nullBuffer, buffer: buffer1) + case .interval(_): + return makeFixedArray( + length: length, elementType: Int32.self, + nullBuffer: nullBuffer, buffer: buffer1) + default: + throw ArrowError(.notImplemented) } } else if arrowType.isVariable { let buffer1 = try nextBuffer( message: rbMessage, index: &bufferIndex, offset: offset, data: data) let buffer2 = try nextBuffer( message: rbMessage, index: &bufferIndex, offset: offset, data: data) - - let offsetsBufferTyped = FixedWidthBufferIPC(buffer: buffer1) - + let offsetsBufferTyped = FixedWidthBufferIPC( + buffer: buffer1 + ) if arrowType == .utf8 { let valueBufferTyped = VariableLengthBufferIPC( - buffer: buffer2) + buffer: buffer2 + ) return ArrowArrayVariable( length: length, nullBuffer: nullBuffer, @@ -341,44 +313,113 @@ public struct ArrowReader { valueBuffer: valueBufferTyped ) } else { - throw ArrowError.notImplemented + throw .init(.notImplemented) + } + } else if arrowType.isBinaryView { + let viewsBuffer = try nextBuffer( + message: rbMessage, + index: &bufferIndex, + offset: offset, + data: data + ) + let viewsBufferTyped = FixedWidthBufferIPC( + buffer: viewsBuffer) + + let variadicCount = rbMessage.variadicBufferCounts( + at: variadicBufferIndex) + variadicBufferIndex += 1 + + switch arrowType { + case .binaryView: + var dataBuffers: [VariableLengthBufferIPC] = [] + for _ in 0..( + buffer: dataBuffer) + dataBuffers.append(dataBufferTyped) + } + return ArrowArrayBinaryView( + offset: 0, + length: length, + nullBuffer: nullBuffer, + viewsBuffer: viewsBufferTyped, + dataBuffers: dataBuffers + ) + case .utf8View: + var dataBuffers: [VariableLengthBufferIPC] = [] + for _ in 0..( + buffer: dataBuffer + ) + dataBuffers.append(dataBufferTyped) + } + return ArrowArrayBinaryView( + offset: 0, + length: length, + nullBuffer: nullBuffer, + viewsBuffer: viewsBufferTyped, + dataBuffers: dataBuffers + ) + default: + throw .init(.notImplemented) } } else if arrowType.isNested { switch arrowType { - case .list(let childField): + case .list(let childField), .map(let childField, _): + // A map is simply a list of struct items. let buffer1 = try nextBuffer( message: rbMessage, index: &bufferIndex, offset: offset, data: data) - var offsetsBuffer = FixedWidthBufferIPC(buffer: buffer1) + var offsetsBuffer = FixedWidthBufferIPC( + buffer: buffer1 + ) let array: AnyArrowArrayProtocol = try loadField( rbMessage: rbMessage, field: childField, offset: offset, nodeIndex: &nodeIndex, - bufferIndex: &bufferIndex + bufferIndex: &bufferIndex, + variadicBufferIndex: &variadicBufferIndex ) if offsetsBuffer.length == 0 { // Empty offsets buffer is valid when child array is empty // There could be any number of empty lists referencing into an empty list guard array.length == 0 else { - throw ArrowError.invalid( - "Empty offsets buffer but non-empty child array") + throw .init( + .invalid("Empty offsets buffer but non-empty child array") + ) } let emptyBuffer = emptyOffsetBuffer(offsetCount: length + 1) - offsetsBuffer = FixedWidthBufferIPC(buffer: emptyBuffer) + offsetsBuffer = FixedWidthBufferIPC( + buffer: emptyBuffer + ) } else { let requiredBytes = (length + 1) * MemoryLayout.stride guard offsetsBuffer.length >= requiredBytes else { - throw ArrowError.invalid( - "Offsets buffer too small: need \(requiredBytes) bytes for \(length) lists" + throw ArrowError( + .invalid( + "Offsets buffer of length: \(offsetsBuffer.length) too small: need \(requiredBytes) bytes for \(length) lists" + ) ) } // Verify last offset matches child array length let lastOffset = offsetsBuffer[length] guard lastOffset == Int32(array.length) else { - throw ArrowError.invalid( - "Expected last offset to match child array length.") + throw ArrowError( + .invalid( + "Expected last offset to match child array length.")) } } return makeListArray( @@ -393,7 +434,8 @@ public struct ArrowReader { field: field, offset: offset, nodeIndex: &nodeIndex, - bufferIndex: &bufferIndex + bufferIndex: &bufferIndex, + variadicBufferIndex: &variadicBufferIndex ) return ArrowFixedSizeListArray( length: length, @@ -409,7 +451,8 @@ public struct ArrowReader { field: field, offset: offset, nodeIndex: &nodeIndex, - bufferIndex: &bufferIndex + bufferIndex: &bufferIndex, + variadicBufferIndex: &variadicBufferIndex ) arrays.append((field.name, array)) } @@ -419,7 +462,7 @@ public struct ArrowReader { fields: arrays ) default: - throw ArrowError.notImplemented + throw ArrowError(.notImplemented) } } else { // MARK: Unclassifiable types. @@ -435,16 +478,22 @@ public struct ArrowReader { valueBuffer: valueBufferTyped ) } - throw ArrowError.notImplemented + throw ArrowError(.notImplemented) } } func nextBuffer( - message: FRecordBatch, index: inout Int32, offset: Int64, data: Data - ) throws -> FileDataBuffer { + message: FRecordBatch, + index: inout Int32, + offset: Int64, + data: Data + ) throws(ArrowError) -> FileDataBuffer { guard index < message.buffersCount, let buffer = message.buffers(at: index) else { - throw ArrowError.invalid("Invalid buffer index.") + throw .init( + .invalid( + "Buffer index \(index) requested for message with \(message.buffersCount) buffers." + )) } index += 1 let startOffset = offset + buffer.offset @@ -460,7 +509,9 @@ public struct ArrowReader { nullBuffer: NullBuffer, buffer: FileDataBuffer ) -> ArrowArrayNumeric { - let fixedBuffer = FixedWidthBufferIPC(buffer: buffer) + let fixedBuffer = FixedWidthBufferIPC( + buffer: buffer + ) return ArrowArrayNumeric( length: length, nullBuffer: nullBuffer, @@ -468,16 +519,12 @@ public struct ArrowReader { ) } - func makeListArray( + func makeListArray( length: Int, nullBuffer: NullBuffer, - offsetsBuffer: OffsetsBuffer, + offsetsBuffer: any FixedWidthBufferProtocol, values: AnyArrowArrayProtocol - ) -> ArrowListArray - where - OffsetsBuffer: FixedWidthBufferProtocol, - OffsetsBuffer.ElementType: FixedWidthInteger & SignedInteger - { + ) -> ArrowListArray { ArrowListArray( length: length, nullBuffer: nullBuffer, @@ -497,7 +544,7 @@ public struct ArrowReader { var fields: [ArrowField] = [] for index in 0.. Self { - let fieldType: ArrowType = try .type(for: field) - guard let fieldName = field.name else { - throw .invalid("Field name not found") - } - let fieldMetadata = (0.. [Offset] { + var keyValueOffsets: [Offset] = [] + for (key, value) in metadata { + let keyOffset = fbb.create(string: key) + let valueOffset = fbb.create(string: value) + let kvOffset = FKeyValue.createKeyValue( + &fbb, + keyOffset: keyOffset, + valueOffset: valueOffset + ) + keyValueOffsets.append(kvOffset) + } + return keyValueOffsets + } + func write( field: ArrowField, to fbb: inout FlatBufferBuilder, ) throws(ArrowError) -> Offset { + // Create child fields first. var fieldsOffset: Offset? if case .strct(let fields) = field.type { var offsets: [Offset] = [] @@ -30,9 +54,24 @@ extension ArrowWriter { offsets.append(offset) } fieldsOffset = fbb.createVector(ofOffsets: offsets) + } else if case .list(let childField) = field.type { + let offset = try write(field: childField, to: &fbb) + fieldsOffset = fbb.createVector(ofOffsets: [offset]) + } else if case .fixedSizeList(let childField, _) = field.type { + let offset = try write(field: childField, to: &fbb) + fieldsOffset = fbb.createVector(ofOffsets: [offset]) + } else if case .map(let childField, _) = field.type { + let offset = try write(field: childField, to: &fbb) + fieldsOffset = fbb.createVector(ofOffsets: [offset]) } + // Create all strings and nested objects before startField. let nameOffset = fbb.create(string: field.name) let fieldTypeOffset = try append(arrowType: field.type, to: &fbb) + // Create metadata vector before startField. + let metadata = field.metadata + let keyValueOffsets = metadataOffsets(metadata: metadata, fbb: &fbb) + let customMetadataOffset = fbb.createVector(ofOffsets: keyValueOffsets) + // Start the Field table. let startOffset = FField.startField(&fbb) FField.add(name: nameOffset, &fbb) FField.add(nullable: field.isNullable, &fbb) @@ -42,9 +81,16 @@ extension ArrowWriter { let typeType = try field.type.fType() FField.add(typeType: typeType, &fbb) FField.add(type: fieldTypeOffset, &fbb) + FField.addVectorOf(customMetadata: customMetadataOffset, &fbb) return FField.endField(&fbb, start: startOffset) } + /// Append the arrow type to the FlatBuffers builder. + /// - Parameters: + /// - arrowType: The `ArrowType`. + /// - fbb: The FlatBuffers builder. + /// - Returns: The offset to the newly appended arrow type. + /// - Throws: An `ArrowError` if the type is not serializable. func append( arrowType: ArrowType, to fbb: inout FlatBufferBuilder, @@ -66,8 +112,17 @@ extension ArrowWriter { return FFloatingPoint.createFloatingPoint(&fbb, precision: .double) case .utf8: return FUtf8.endUtf8(&fbb, start: FUtf8.startUtf8(&fbb)) + case .utf8View: + return FUtf8View.endUtf8View(&fbb, start: FUtf8View.startUtf8View(&fbb)) case .binary: return FBinary.endBinary(&fbb, start: FBinary.startBinary(&fbb)) + case .binaryView: + return FBinaryView.endBinaryView( + &fbb, start: FBinaryView.startBinaryView(&fbb)) + case .fixedSizeBinary(let byteWidth): + let startOffset = FFixedSizeBinary.startFixedSizeBinary(&fbb) + FFixedSizeBinary.add(byteWidth: byteWidth, &fbb) + return FFixedSizeBinary.endFixedSizeBinary(&fbb, start: startOffset) case .boolean: return FBool.endBool(&fbb, start: FBool.startBool(&fbb)) case .date32: @@ -81,12 +136,16 @@ extension ArrowWriter { case .time32(let unit): let startOffset = FTime.startTime(&fbb) FTime.add(unit: unit == .second ? .second : .millisecond, &fbb) + FTime.add(bitWidth: 32, &fbb) return FTime.endTime(&fbb, start: startOffset) case .time64(let unit): let startOffset = FTime.startTime(&fbb) FTime.add(unit: unit == .microsecond ? .microsecond : .nanosecond, &fbb) + FTime.add(bitWidth: 64, &fbb) return FTime.endTime(&fbb, start: startOffset) case .timestamp(let unit, let timezone): + // Timezone string must be created before starting the timestamp table. + let timezoneOffset: Offset? = timezone.map { fbb.create(string: $0) } let startOffset = FTimestamp.startTimestamp(&fbb) let fbUnit: FTimeUnit switch unit { @@ -100,17 +159,31 @@ extension ArrowWriter { fbUnit = .nanosecond } FTimestamp.add(unit: fbUnit, &fbb) - if let timezone { - let timezoneOffset = fbb.create(string: timezone) + if let timezoneOffset { FTimestamp.add(timezone: timezoneOffset, &fbb) } return FTimestamp.endTimestamp(&fbb, start: startOffset) - case .strct(_): + case .duration(let timeUnit): + let startOffset = FDuration.startDuration(&fbb) + FDuration.add(unit: timeUnit.toFlatBufferUnit(), &fbb) + return FDuration.endDuration(&fbb, start: startOffset) + case .strct: let startOffset = FStruct.startStruct_(&fbb) return FStruct.endStruct_(&fbb, start: startOffset) + case .list: + let startOffset = FList.startList(&fbb) + return FList.endList(&fbb, start: startOffset) + case .fixedSizeList(_, let listSize): + let startOffset = FFixedSizeList.startFixedSizeList(&fbb) + FFixedSizeList.add(listSize: listSize, &fbb) + return FFixedSizeList.endFixedSizeList(&fbb, start: startOffset) + case .map: + let startOffset = FMap.startMap(&fbb) + return FMap.endMap(&fbb, start: startOffset) default: - throw .unknownType( - "Unable to add flatbuf type for Arrow type: \(arrowType)") + throw .init( + .unknownType( + "Unable to add FlatBuffers type for Arrow type: \(arrowType).")) } } } diff --git a/Sources/ArrowIPC/ArrowWriter.swift b/Sources/ArrowIPC/ArrowWriter.swift index 1027fc3..fbc59a1 100644 --- a/Sources/ArrowIPC/ArrowWriter.swift +++ b/Sources/ArrowIPC/ArrowWriter.swift @@ -26,7 +26,7 @@ public struct ArrowWriter { write(bytes: fileMarker) } - mutating func finish() throws { + public mutating func finish() throws { data.append(contentsOf: fileMarker) try data.write(to: url) } @@ -67,7 +67,7 @@ public struct ArrowWriter { precondition(data.count % 8 == 0, "File must be aligned to 8 bytes.") } - mutating func write( + public mutating func write( schema: ArrowSchema, recordBatches: [RecordBatch] ) throws { @@ -113,10 +113,19 @@ public struct ArrowWriter { fieldOffsets.append(offset) } let fieldsOffset: Offset = fbb.createVector(ofOffsets: fieldOffsets) + let metadataOffset = { + if let metadata = schema.metadata, !metadata.isEmpty { + let metadataOffsets = metadataOffsets(metadata: metadata, fbb: &fbb) + return fbb.createVector(ofOffsets: metadataOffsets) + } else { + return Offset() + } + }() let schemaOffset = FSchema.createSchema( &fbb, endianness: .little, - fieldsVectorOffset: fieldsOffset + fieldsVectorOffset: fieldsOffset, + customMetadataVectorOffset: metadataOffset ) return schemaOffset } @@ -127,14 +136,7 @@ public struct ArrowWriter { ) throws(ArrowError) -> Data { var fbb: FlatBufferBuilder = .init() let schemaOffset = try write(schema: schema, to: &fbb) - fbb.startVector( - blocks.count, - elementSize: MemoryLayout.size - ) - for block in blocks.reversed() { - fbb.create(struct: block) - } - let blocksOffset = fbb.endVector(len: blocks.count) + let blocksOffset = fbb.createVector(ofStructs: blocks) let footerStartOffset = FFooter.startFooter(&fbb) FFooter.add(schema: schemaOffset, &fbb) FFooter.addVectorOf(recordBatches: blocksOffset, &fbb) @@ -150,11 +152,8 @@ public struct ArrowWriter { ) throws -> [FBlock] { var blocks: [FBlock] = .init() for recordBatch in recordBatches { - let startIndex = data.count - let message = try write(batch: recordBatch) - // TODO: Better API for marker/count var buffer = Data() withUnsafeBytes(of: continuationMarker.littleEndian) { val in buffer.append(contentsOf: val) @@ -177,17 +176,17 @@ public struct ArrowWriter { let bodyLength = data.count - bodyStart let expectedSize = startIndex + metadataLength + bodyLength guard expectedSize == data.count else { - throw ArrowError.invalid( - "Invalid Block. Expected \(expectedSize), got \(data.count)" - ) + throw ArrowError( + .invalid( + "Invalid Block. Expected \(expectedSize), got \(data.count)" + )) } - blocks.append( - FBlock( - offset: Int64(startIndex), - metaDataLength: Int32(metadataLength), - bodyLength: Int64(bodyLength) - ) + let block = FBlock( + offset: Int64(startIndex), + metaDataLength: Int32(metadataLength), + bodyLength: Int64(bodyLength) ) + blocks.append(block) } return blocks } @@ -195,8 +194,9 @@ public struct ArrowWriter { private mutating func writeRecordBatchData( fields: [ArrowField], arrays: [AnyArrowArrayProtocol] - ) throws { + ) throws(ArrowError) { for index in 0.. else { + throw .init(.invalid("List type array expected.")) } + try writeRecordBatchData( + fields: [childType], + arrays: [listArray.values] + ) + case .fixedSizeList(let childType, _): + guard let listArray = array as? ArrowFixedSizeListArray else { + throw .init(.invalid("Fixed size list type array expected.")) + } + try writeRecordBatchData( + fields: [childType], + arrays: [listArray.values] + ) + default: + break } } } + /// Write the record batch message. + /// - Parameter batch: The `RecordBatch` to write metadata for. + /// - Returns: The FlatBuffers message serialized to `Data`. + /// - Throws: An `ArrowError`if arrays are unreadable. private func write( batch: RecordBatch - ) throws -> Data { + ) throws(ArrowError) -> Data { let schema = batch.schema var fbb = FlatBufferBuilder() // MARK: Field nodes. - var fieldNodeOffsets: [Offset] = [] - fbb.startVector( - schema.fields.count, - elementSize: MemoryLayout.size - ) + var fieldNodes: [FFieldNode] = [] writeFieldNodes( fields: schema.fields, - columns: batch.arrays, - offsets: &fieldNodeOffsets, - fbb: &fbb + arrays: batch.arrays, + nodes: &fieldNodes, ) - let nodeOffset = fbb.endVector(len: fieldNodeOffsets.count) + let nodeOffset = fbb.createVector(ofStructs: fieldNodes) // MARK: Buffers. - var buffers: [FBuffer] = .init() - var bufferOffset: Int = 0 - writeBufferInfo( - schema.fields, columns: batch.arrays, - bufferOffset: &bufferOffset, buffers: &buffers, - fbb: &fbb + var buffers: [FBuffer] = [] + var variadicBufferCounts: [Int64] = [] + var bufferOffset: Int64 = 0 + try writeBufferInfo( + schema.fields, + arrays: batch.arrays, + bufferOffset: &bufferOffset, + buffers: &buffers, + variadicBufferCounts: &variadicBufferCounts ) - FRecordBatch.startVectorOfBuffers(batch.schema.fields.count, in: &fbb) - for buffer in buffers.reversed() { - fbb.create(struct: buffer) - } - let batchBuffersOffset = fbb.endVector(len: buffers.count) + let batchBuffersOffset = fbb.createVector(ofStructs: buffers) + let variadicCountsOffset = fbb.createVector(variadicBufferCounts) + + // MARK: Start record batch. let startRb = FRecordBatch.startRecordBatch(&fbb) FRecordBatch.addVectorOf(nodes: nodeOffset, &fbb) FRecordBatch.addVectorOf(buffers: batchBuffersOffset, &fbb) FRecordBatch.add(length: Int64(batch.length), &fbb) + FRecordBatch.addVectorOf(variadicBufferCounts: variadicCountsOffset, &fbb) + let recordBatchOffset = FRecordBatch.endRecordBatch( &fbb, start: startRb @@ -274,62 +294,116 @@ public struct ArrowWriter { return fbb.data } + /// Write the field nodes. + /// - Parameters: + /// - fields: The Arrow fields. + /// - arrays: The Arrow arrays. + /// - nodes: The field node array being written to. private func writeFieldNodes( fields: [ArrowField], - columns: [AnyArrowArrayProtocol], - offsets: inout [Offset], - fbb: inout FlatBufferBuilder + arrays: [AnyArrowArrayProtocol], + nodes: inout [FFieldNode] ) { - for index in (0.. { + writeFieldNodes( + fields: [childField], + arrays: [column.values], + nodes: &nodes, ) } + case .fixedSizeList(let childField, _): + if let column = column as? ArrowFixedSizeListArray { + writeFieldNodes( + fields: [childField], + arrays: [column.values], + nodes: &nodes, + ) + } + default: + break } } } private func writeBufferInfo( _ fields: [ArrowField], - columns: [AnyArrowArrayProtocol], - bufferOffset: inout Int, + arrays: [AnyArrowArrayProtocol], + bufferOffset: inout Int64, buffers: inout [FBuffer], - fbb: inout FlatBufferBuilder - ) { + variadicBufferCounts: inout [Int64] + ) throws(ArrowError) { for index in 0.. else { + throw .init(.invalid("Expected ArrowListArray")) + } + try writeBufferInfo( + [childField], + arrays: [column.values], + bufferOffset: &bufferOffset, + buffers: &buffers, + variadicBufferCounts: &variadicBufferCounts + ) + case .fixedSizeList(let childField, _): + guard let column = array as? ArrowFixedSizeListArray else { + throw .init(.invalid("Expected ArrowFixedSizeListArray")) + } + try writeBufferInfo( + [childField], + arrays: [column.values], + bufferOffset: &bufferOffset, + buffers: &buffers, + variadicBufferCounts: &variadicBufferCounts + ) + default: + break } } } diff --git a/Sources/ArrowIPC/Generated/FlatBuffersTypes.swift b/Sources/ArrowIPC/Generated/FlatBuffersTypes.swift index 1e0f927..7656c23 100644 --- a/Sources/ArrowIPC/Generated/FlatBuffersTypes.swift +++ b/Sources/ArrowIPC/Generated/FlatBuffersTypes.swift @@ -26,13 +26,19 @@ typealias FInt = org_apache_arrow_flatbuf_Int typealias FBool = org_apache_arrow_flatbuf_Bool typealias FDate = org_apache_arrow_flatbuf_Date typealias FTime = org_apache_arrow_flatbuf_Time +typealias FDuration = org_apache_arrow_flatbuf_Duration typealias FTimestamp = org_apache_arrow_flatbuf_Timestamp typealias FTimeUnit = org_apache_arrow_flatbuf_TimeUnit typealias FStruct = org_apache_arrow_flatbuf_Struct_ +typealias FList = org_apache_arrow_flatbuf_List typealias FUtf8 = org_apache_arrow_flatbuf_Utf8 +typealias FUtf8View = org_apache_arrow_flatbuf_Utf8View typealias FBinary = org_apache_arrow_flatbuf_Binary +typealias FBinaryView = org_apache_arrow_flatbuf_BinaryView typealias FFixedSizeBinary = org_apache_arrow_flatbuf_FixedSizeBinary typealias FFixedSizeList = org_apache_arrow_flatbuf_FixedSizeList +typealias FMap = org_apache_arrow_flatbuf_Map typealias FMessageHeader = org_apache_arrow_flatbuf_MessageHeader +typealias FKeyValue = org_apache_arrow_flatbuf_KeyValue diff --git a/Tests/ArrowFlightTests/FlightTest.swift b/Tests/ArrowFlightTests/FlightTest.swift index 1054e15..1b4ed11 100644 --- a/Tests/ArrowFlightTests/FlightTest.swift +++ b/Tests/ArrowFlightTests/FlightTest.swift @@ -34,375 +34,375 @@ func makeSchema() -> ArrowSchema { .finish() } -func makeRecordBatch() throws -> RecordBatchX { - let doubleBuilder: NumberArrayBuilder = - try ArrowArrayBuilders.loadNumberArrayBuilder() - doubleBuilder.append(11.11) - doubleBuilder.append(22.22) - doubleBuilder.append(33.33) - doubleBuilder.append(44.44) - let stringBuilder = try ArrowArrayBuilders.loadStringArrayBuilder() - stringBuilder.append("test10") - stringBuilder.append("test22") - stringBuilder.append("test33") - stringBuilder.append("test44") - let date32Builder = try ArrowArrayBuilders.loadDate32ArrayBuilder() - let date2 = Date(timeIntervalSinceReferenceDate: 86400 * 1) - let date1 = Date(timeIntervalSinceReferenceDate: 86400 * 5000 + 352) - date32Builder.append(date1) - date32Builder.append(date2) - date32Builder.append(date1) - date32Builder.append(date2) - let doubleArray = try doubleBuilder.finish() - let stringArray = try stringBuilder.finish() - let date32Array = try date32Builder.finish() - let result = RecordBatchX.Builder() - .addColumn("col1", arrowArray: doubleArray) - .addColumn("col2", arrowArray: stringArray) - .addColumn("col3", arrowArray: date32Array) - .finish() - switch result { - case .success(let recordBatch): - return recordBatch - case .failure(let error): - throw error - } -} - -nonisolated(unsafe) var flights: [String: FlightInfo] = [:] -final class MyFlightServer: ArrowFlightServer { - func doExchange( - _ reader: ArrowFlight.RecordBatchStreamReader, - writer: ArrowFlight.RecordBatchStreamWriter - ) async throws { - do { - for try await rbData in reader { - let rb = rbData.0! - XCTAssertEqual(rb.schema.fields.count, 3) - XCTAssertEqual(rb.length, 4) - } - - let rb = try makeRecordBatch() - try await writer.write(rb) - } catch { - print("Unknown error: \(error)") - } - } - - func doPut( - _ reader: ArrowFlight.RecordBatchStreamReader, - writer: ArrowFlight.PutResultDataStreamWriter - ) async throws { - for try await rbData in reader { - let rb = rbData.0! - let key = String(decoding: rbData.1!.cmd, as: UTF8.self) - flights[key] = try FlightInfo( - schemaToMessage(rb.schema), endpoints: [], descriptor: rbData.1) - XCTAssertEqual(rb.schema.fields.count, 3) - XCTAssertEqual(rb.length, 4) - try await writer.write(FlightPutResult()) - } - } - - func doGet( - _ ticket: ArrowFlight.FlightTicket, - writer: ArrowFlight.RecordBatchStreamWriter - ) async throws { - try await writer.write(try makeRecordBatch()) - } - - func getSchema(_ request: ArrowFlight.FlightDescriptor) async throws - -> ArrowFlight.FlightSchemaResult - { - XCTAssertEqual(String(bytes: request.cmd, encoding: .utf8)!, "schema info") - XCTAssertEqual(request.type, .cmd) - return try ArrowFlight.FlightSchemaResult(schemaToMessage(makeSchema())) - } - - func getFlightInfo(_ request: ArrowFlight.FlightDescriptor) async throws - -> ArrowFlight.FlightInfo - { - let key = String(decoding: request.cmd, as: UTF8.self) - if flights[key] != nil { - return ArrowFlight.FlightInfo(flights[key]!.toProtocol()) - } - - throw ArrowFlightError.ioError("Flight not found") - } - - func listFlights( - _ criteria: ArrowFlight.FlightCriteria, - writer: ArrowFlight.FlightInfoStreamWriter - ) async throws { - XCTAssertEqual( - String(bytes: criteria.expression, encoding: .utf8), - "flight criteria expression") - for flightData in flights { - try await writer.write(flightData.value) - } - } - - func listActions(_ writer: ArrowFlight.ActionTypeStreamWriter) async throws { - try await writer.write( - FlightActionType("clear", description: "Clear the stored flights.")) - try await writer.write( - FlightActionType("shutdown", description: "Shut down this server.")) - } - - func doAction(_ action: FlightAction, writer: ResultStreamWriter) async throws - { - XCTAssertEqual(action.type, "healthcheck") - XCTAssertEqual( - String(bytes: action.body, encoding: .utf8)!, "healthcheck body") - try await writer.write( - FlightResult("test_action result".data(using: .utf8)!)) - } -} - -struct FlightServerImpl { - var port = 1234 - nonisolated(unsafe) static var server: Server? - nonisolated(unsafe) static var group: MultiThreadedEventLoopGroup? - static func run() async throws { - do { - // Create an event loop group for the server to run on. - let group = MultiThreadedEventLoopGroup(numberOfThreads: System.coreCount) - // Create a provider using the features we read. - let provider = ArrowFlight.makeFlightServer(MyFlightServer()) - - // Start the server and print its address once it has started. - FlightServerImpl.server = try await Server.insecure(group: group) - .withServiceProviders([provider]) - .bind(host: "localhost", port: 8088) - .get() - - print("server started on port \(server!.channel.localAddress!.port!)") - // Wait on the server's `onClose` future to stop the program from exiting. - } catch { - print("Unknown server error: \(error)") - } - } - - static func syncShutdown() throws { - try group?.syncShutdownGracefully() - } -} - -public class FlightClientTester { - var client: FlightClient? - var group: MultiThreadedEventLoopGroup? - var channel: GRPCChannel? - - init() async throws { - // Load the features. - let group = PlatformSupport.makeEventLoopGroup(loopCount: 1) - let channel = try GRPCChannelPool.with( - target: .host("localhost", port: 8088), - transportSecurity: .plaintext, - eventLoopGroup: group - ) - - client = FlightClient(channel: channel) - } - - func syncShutdown() throws { - try group?.syncShutdownGracefully() - try channel?.close().wait() - } - - func listActionTest() async throws { - var actionTypes: [FlightActionType] = [] - try await client?.listActions({ action in - actionTypes.append(action) - }) - - XCTAssertEqual(actionTypes.count, 2) - - XCTAssertEqual(actionTypes[0].type, "clear") - XCTAssertEqual(actionTypes[0].description, "Clear the stored flights.") - XCTAssertEqual(actionTypes[1].type, "shutdown") - XCTAssertEqual(actionTypes[1].description, "Shut down this server.") - } - - func listFlightsTest() async throws { - let flightCriteria = FlightCriteria( - "flight criteria expression".data(using: .utf8)!) - var numCalls = 0 - try await client?.listFlights( - flightCriteria, - closure: { data in - if let schema = data.schema { - XCTAssertGreaterThanOrEqual(schema.fields.count, 0) - numCalls += 1 - } - }) - - XCTAssertEqual(numCalls, 2) - } - - func doActionTest(_ type: String, actionBody: Data) async throws { - let action = FlightAction(type, body: actionBody) - var actionResults: [FlightResult] = [] - try await client?.doAction( - action, - closure: { result in - actionResults.append(result) - }) - - XCTAssertEqual(actionResults.count, 1) - XCTAssertEqual( - String(bytes: actionResults[0].body, encoding: .utf8), - "test_action result") - } - - func getSchemaTest(_ cmd: Data) async throws { - let descriptor = FlightDescriptor(cmd: cmd) - let schemaResult = try await client?.getSchema(descriptor) - let schema = schemaResult!.schema! - XCTAssertEqual(schema.fields.count, 3) - } - - func doGetTest(_ flightData: Data) async throws { - let ticket = FlightTicket(flightData) - var numCall = 0 - try await client?.doGet( - ticket, - readerResultClosure: { rb in - numCall += 1 - XCTAssertEqual(rb.schema!.fields.count, 3) - XCTAssertEqual(rb.batches[0].length, 4) - switch ArrowTable.from(recordBatches: rb.batches) { - case .success(let table): - for column in table.columns { - switch column.type { - case .float64: - let doubleArray = try column.data() as? ChunkedArray - XCTAssertNotNil(doubleArray) - XCTAssertEqual(doubleArray?[0], 11.11) - XCTAssertEqual(doubleArray?.asString(0), "11.11") - default: - continue - } - } - case .failure(let error): - throw error - } - }) - - XCTAssertEqual(numCall, 1) - } - - func doGetTestFlightData(_ flightData: Data) async throws { - let ticket = FlightTicket(flightData) - var numCall = 0 - let reader = ArrowReaderX() - let arrowResult = ArrowReaderX.makeArrowReaderResult() - try await client?.doGet( - ticket, - flightDataClosure: { flightData in - switch reader.fromMessage( - flightData.dataHeader, dataBody: flightData.dataBody, - result: arrowResult) - { - case .success: - numCall += 1 - case .failure(let error): - throw error - } - }) - - XCTAssertEqual(numCall, 2) - } - - func doPutTest(_ cmd: String) async throws { - let descriptor = FlightDescriptor(cmd: cmd.data(using: .utf8)!) - let rb = try makeRecordBatch() - var numCall = 0 - try await client?.doPut( - descriptor, recordBatches: [rb], - closure: { _ in - numCall += 1 - }) - - XCTAssertEqual(numCall, 1) - } - - func doExchangeTest() async throws { - let descriptor = FlightDescriptor(cmd: "flight_ticket".data(using: .utf8)!) - let rb = try makeRecordBatch() - var numCall = 0 - try await client?.doExchange( - descriptor, recordBatches: [rb], - closure: { result in - numCall += 1 - XCTAssertEqual(result.schema?.fields.count, 3) - XCTAssertEqual(result.batches[0].length, 4) - }) - - XCTAssertEqual(numCall, 1) - } -} - -actor FlightServerData { - public var serverup = false - func setServerUp(_ serverUp: Bool) { - self.serverup = serverUp - } - - func isServerUp() -> Bool { - serverup - } -} - -final class FlightTest: XCTestCase { - - func testFlightServer() async throws { - let serverData = FlightServerData() - let basicTask = Task { - try await FlightServerImpl.run() - defer { - print("server shutting down") - do { - try FlightServerImpl.syncShutdown() - } catch { - } - } - - await serverData.setServerUp(true) - try await FlightServerImpl.server?.onClose.get() - return "done" - } - - let secondTask = Task { - defer { - _ = FlightServerImpl.server?.close() - } - - while await !serverData.isServerUp() { - try await Task.sleep(nanoseconds: 1_000_000) - } - - let clientImpl = try await FlightClientTester() - defer { - do { - try clientImpl.syncShutdown() - } catch { - } - } - try await clientImpl.listActionTest() - try await clientImpl.doPutTest("flight_ticket") - try await clientImpl.doPutTest("flight_another") - try await clientImpl.listFlightsTest() - try await clientImpl.doActionTest( - "healthcheck", actionBody: Data("healthcheck body".utf8)) - try await clientImpl.getSchemaTest(Data("schema info".utf8)) - try await clientImpl.doGetTest(Data("'flight_ticket'".utf8)) - try await clientImpl.doGetTestFlightData(Data("'flight_another'".utf8)) - try await clientImpl.doExchangeTest() - return "done" - } +//func makeRecordBatch() throws -> RecordBatchX { +// let doubleBuilder: NumberArrayBuilder = +// try ArrowArrayBuilders.loadNumberArrayBuilder() +// doubleBuilder.append(11.11) +// doubleBuilder.append(22.22) +// doubleBuilder.append(33.33) +// doubleBuilder.append(44.44) +// let stringBuilder = try ArrowArrayBuilders.loadStringArrayBuilder() +// stringBuilder.append("test10") +// stringBuilder.append("test22") +// stringBuilder.append("test33") +// stringBuilder.append("test44") +// let date32Builder = try ArrowArrayBuilders.loadDate32ArrayBuilder() +// let date2 = Date(timeIntervalSinceReferenceDate: 86400 * 1) +// let date1 = Date(timeIntervalSinceReferenceDate: 86400 * 5000 + 352) +// date32Builder.append(date1) +// date32Builder.append(date2) +// date32Builder.append(date1) +// date32Builder.append(date2) +// let doubleArray = try doubleBuilder.finish() +// let stringArray = try stringBuilder.finish() +// let date32Array = try date32Builder.finish() +// let result = RecordBatchX.Builder() +// .addColumn("col1", arrowArray: doubleArray) +// .addColumn("col2", arrowArray: stringArray) +// .addColumn("col3", arrowArray: date32Array) +// .finish() +// switch result { +// case .success(let recordBatch): +// return recordBatch +// case .failure(let error): +// throw error +// } +//} +// +//nonisolated(unsafe) var flights: [String: FlightInfo] = [:] +//final class MyFlightServer: ArrowFlightServer { +// func doExchange( +// _ reader: ArrowFlight.RecordBatchStreamReader, +// writer: ArrowFlight.RecordBatchStreamWriter +// ) async throws { +// do { +// for try await rbData in reader { +// let rb = rbData.0! +// XCTAssertEqual(rb.schema.fields.count, 3) +// XCTAssertEqual(rb.length, 4) +// } +// +// let rb = try makeRecordBatch() +// try await writer.write(rb) +// } catch { +// print("Unknown error: \(error)") +// } +// } +// +// func doPut( +// _ reader: ArrowFlight.RecordBatchStreamReader, +// writer: ArrowFlight.PutResultDataStreamWriter +// ) async throws { +// for try await rbData in reader { +// let rb = rbData.0! +// let key = String(decoding: rbData.1!.cmd, as: UTF8.self) +// flights[key] = try FlightInfo( +// schemaToMessage(rb.schema), endpoints: [], descriptor: rbData.1) +// XCTAssertEqual(rb.schema.fields.count, 3) +// XCTAssertEqual(rb.length, 4) +// try await writer.write(FlightPutResult()) +// } +// } +// +// func doGet( +// _ ticket: ArrowFlight.FlightTicket, +// writer: ArrowFlight.RecordBatchStreamWriter +// ) async throws { +// try await writer.write(try makeRecordBatch()) +// } +// +// func getSchema(_ request: ArrowFlight.FlightDescriptor) async throws +// -> ArrowFlight.FlightSchemaResult +// { +// XCTAssertEqual(String(bytes: request.cmd, encoding: .utf8)!, "schema info") +// XCTAssertEqual(request.type, .cmd) +// return try ArrowFlight.FlightSchemaResult(schemaToMessage(makeSchema())) +// } +// +// func getFlightInfo(_ request: ArrowFlight.FlightDescriptor) async throws +// -> ArrowFlight.FlightInfo +// { +// let key = String(decoding: request.cmd, as: UTF8.self) +// if flights[key] != nil { +// return ArrowFlight.FlightInfo(flights[key]!.toProtocol()) +// } +// +// throw ArrowFlightError.ioError("Flight not found") +// } +// +// func listFlights( +// _ criteria: ArrowFlight.FlightCriteria, +// writer: ArrowFlight.FlightInfoStreamWriter +// ) async throws { +// XCTAssertEqual( +// String(bytes: criteria.expression, encoding: .utf8), +// "flight criteria expression") +// for flightData in flights { +// try await writer.write(flightData.value) +// } +// } +// +// func listActions(_ writer: ArrowFlight.ActionTypeStreamWriter) async throws { +// try await writer.write( +// FlightActionType("clear", description: "Clear the stored flights.")) +// try await writer.write( +// FlightActionType("shutdown", description: "Shut down this server.")) +// } +// +// func doAction(_ action: FlightAction, writer: ResultStreamWriter) async throws +// { +// XCTAssertEqual(action.type, "healthcheck") +// XCTAssertEqual( +// String(bytes: action.body, encoding: .utf8)!, "healthcheck body") +// try await writer.write( +// FlightResult("test_action result".data(using: .utf8)!)) +// } +//} +// +//struct FlightServerImpl { +// var port = 1234 +// nonisolated(unsafe) static var server: Server? +// nonisolated(unsafe) static var group: MultiThreadedEventLoopGroup? +// static func run() async throws { +// do { +// // Create an event loop group for the server to run on. +// let group = MultiThreadedEventLoopGroup(numberOfThreads: System.coreCount) +// // Create a provider using the features we read. +// let provider = ArrowFlight.makeFlightServer(MyFlightServer()) +// +// // Start the server and print its address once it has started. +// FlightServerImpl.server = try await Server.insecure(group: group) +// .withServiceProviders([provider]) +// .bind(host: "localhost", port: 8088) +// .get() +// +// print("server started on port \(server!.channel.localAddress!.port!)") +// // Wait on the server's `onClose` future to stop the program from exiting. +// } catch { +// print("Unknown server error: \(error)") +// } +// } +// +// static func syncShutdown() throws { +// try group?.syncShutdownGracefully() +// } +//} +// +//public class FlightClientTester { +// var client: FlightClient? +// var group: MultiThreadedEventLoopGroup? +// var channel: GRPCChannel? +// +// init() async throws { +// // Load the features. +// let group = PlatformSupport.makeEventLoopGroup(loopCount: 1) +// let channel = try GRPCChannelPool.with( +// target: .host("localhost", port: 8088), +// transportSecurity: .plaintext, +// eventLoopGroup: group +// ) +// +// client = FlightClient(channel: channel) +// } +// +// func syncShutdown() throws { +// try group?.syncShutdownGracefully() +// try channel?.close().wait() +// } +// +// func listActionTest() async throws { +// var actionTypes: [FlightActionType] = [] +// try await client?.listActions({ action in +// actionTypes.append(action) +// }) +// +// XCTAssertEqual(actionTypes.count, 2) +// +// XCTAssertEqual(actionTypes[0].type, "clear") +// XCTAssertEqual(actionTypes[0].description, "Clear the stored flights.") +// XCTAssertEqual(actionTypes[1].type, "shutdown") +// XCTAssertEqual(actionTypes[1].description, "Shut down this server.") +// } +// +// func listFlightsTest() async throws { +// let flightCriteria = FlightCriteria( +// "flight criteria expression".data(using: .utf8)!) +// var numCalls = 0 +// try await client?.listFlights( +// flightCriteria, +// closure: { data in +// if let schema = data.schema { +// XCTAssertGreaterThanOrEqual(schema.fields.count, 0) +// numCalls += 1 +// } +// }) +// +// XCTAssertEqual(numCalls, 2) +// } +// +// func doActionTest(_ type: String, actionBody: Data) async throws { +// let action = FlightAction(type, body: actionBody) +// var actionResults: [FlightResult] = [] +// try await client?.doAction( +// action, +// closure: { result in +// actionResults.append(result) +// }) +// +// XCTAssertEqual(actionResults.count, 1) +// XCTAssertEqual( +// String(bytes: actionResults[0].body, encoding: .utf8), +// "test_action result") +// } +// +// func getSchemaTest(_ cmd: Data) async throws { +// let descriptor = FlightDescriptor(cmd: cmd) +// let schemaResult = try await client?.getSchema(descriptor) +// let schema = schemaResult!.schema! +// XCTAssertEqual(schema.fields.count, 3) +// } +// +// func doGetTest(_ flightData: Data) async throws { +// let ticket = FlightTicket(flightData) +// var numCall = 0 +// try await client?.doGet( +// ticket, +// readerResultClosure: { rb in +// numCall += 1 +// XCTAssertEqual(rb.schema!.fields.count, 3) +// XCTAssertEqual(rb.batches[0].length, 4) +// switch ArrowTable.from(recordBatches: rb.batches) { +// case .success(let table): +// for column in table.columns { +// switch column.type { +// case .float64: +// let doubleArray = try column.data() as? ChunkedArray +// XCTAssertNotNil(doubleArray) +// XCTAssertEqual(doubleArray?[0], 11.11) +// XCTAssertEqual(doubleArray?.asString(0), "11.11") +// default: +// continue +// } +// } +// case .failure(let error): +// throw error +// } +// }) +// +// XCTAssertEqual(numCall, 1) +// } +// +// func doGetTestFlightData(_ flightData: Data) async throws { +// let ticket = FlightTicket(flightData) +// var numCall = 0 +// let reader = ArrowReaderX() +// let arrowResult = ArrowReaderX.makeArrowReaderResult() +// try await client?.doGet( +// ticket, +// flightDataClosure: { flightData in +// switch reader.fromMessage( +// flightData.dataHeader, dataBody: flightData.dataBody, +// result: arrowResult) +// { +// case .success: +// numCall += 1 +// case .failure(let error): +// throw error +// } +// }) +// +// XCTAssertEqual(numCall, 2) +// } +// +// func doPutTest(_ cmd: String) async throws { +// let descriptor = FlightDescriptor(cmd: cmd.data(using: .utf8)!) +// let rb = try makeRecordBatch() +// var numCall = 0 +// try await client?.doPut( +// descriptor, recordBatches: [rb], +// closure: { _ in +// numCall += 1 +// }) +// +// XCTAssertEqual(numCall, 1) +// } +// +// func doExchangeTest() async throws { +// let descriptor = FlightDescriptor(cmd: "flight_ticket".data(using: .utf8)!) +// let rb = try makeRecordBatch() +// var numCall = 0 +// try await client?.doExchange( +// descriptor, recordBatches: [rb], +// closure: { result in +// numCall += 1 +// XCTAssertEqual(result.schema?.fields.count, 3) +// XCTAssertEqual(result.batches[0].length, 4) +// }) +// +// XCTAssertEqual(numCall, 1) +// } +//} +// +//actor FlightServerData { +// public var serverup = false +// func setServerUp(_ serverUp: Bool) { +// self.serverup = serverUp +// } +// +// func isServerUp() -> Bool { +// serverup +// } +//} - _ = try await [basicTask.value, secondTask.value] - print("done running") - } -} +//final class FlightTest: XCTestCase { +// +// func testFlightServer() async throws { +// let serverData = FlightServerData() +// let basicTask = Task { +// try await FlightServerImpl.run() +// defer { +// print("server shutting down") +// do { +// try FlightServerImpl.syncShutdown() +// } catch { +// } +// } +// +// await serverData.setServerUp(true) +// try await FlightServerImpl.server?.onClose.get() +// return "done" +// } +// +// let secondTask = Task { +// defer { +// _ = FlightServerImpl.server?.close() +// } +// +// while await !serverData.isServerUp() { +// try await Task.sleep(nanoseconds: 1_000_000) +// } +// +// let clientImpl = try await FlightClientTester() +// defer { +// do { +// try clientImpl.syncShutdown() +// } catch { +// } +// } +// try await clientImpl.listActionTest() +// try await clientImpl.doPutTest("flight_ticket") +// try await clientImpl.doPutTest("flight_another") +// try await clientImpl.listFlightsTest() +// try await clientImpl.doActionTest( +// "healthcheck", actionBody: Data("healthcheck body".utf8)) +// try await clientImpl.getSchemaTest(Data("schema info".utf8)) +// try await clientImpl.doGetTest(Data("'flight_ticket'".utf8)) +// try await clientImpl.doGetTestFlightData(Data("'flight_another'".utf8)) +// try await clientImpl.doExchangeTest() +// return "done" +// } +// +// _ = try await [basicTask.value, secondTask.value] +// print("done running") +// } +//} diff --git a/Tests/ArrowIPCTests/Gold/ArrowGold.swift b/Tests/ArrowIPCTests/ArrowGold.swift similarity index 83% rename from Tests/ArrowIPCTests/Gold/ArrowGold.swift rename to Tests/ArrowIPCTests/ArrowGold.swift index 555bd67..7fa9ff4 100644 --- a/Tests/ArrowIPCTests/Gold/ArrowGold.swift +++ b/Tests/ArrowIPCTests/ArrowGold.swift @@ -14,7 +14,7 @@ import Foundation -/// The JSON file structure used to validate gold-standard Arrow test files. +/// The JSON file structure used to validate gold-standard Arrow test files. struct ArrowGold: Codable, Equatable { let schema: Schema let batches: [Batch] @@ -139,6 +139,8 @@ struct ArrowGold: Codable, Equatable { let validity: [Int]? let offset: [Int]? let data: [DataValue]? + let views: [View?]? + let variadicDataBuffers: [String]? let children: [Column]? enum CodingKeys: String, CodingKey { @@ -147,6 +149,8 @@ struct ArrowGold: Codable, Equatable { case validity = "VALIDITY" case offset = "OFFSET" case data = "DATA" + case views = "VIEWS" + case variadicDataBuffers = "VARIADIC_DATA_BUFFERS" case children } } @@ -195,6 +199,42 @@ enum DataValue: Codable, Equatable { } } +/// Represents an inline value in a binary view or utf8 view. +struct View: Codable, Equatable { + + let size: Int32 + let inlined: String? + let prefixHex: String? + let bufferIndex: Int32? + let offset: Int32? + + // Inlined case (≤12 bytes) + init(size: Int32, inlined: String) { + self.size = size + self.inlined = inlined + self.prefixHex = nil + self.bufferIndex = nil + self.offset = nil + } + + // Reference case (>12 bytes) + init(size: Int32, prefixHex: String, bufferIndex: Int32, offset: Int32) { + self.size = size + self.inlined = nil + self.prefixHex = prefixHex + self.bufferIndex = bufferIndex + self.offset = offset + } + + enum CodingKeys: String, CodingKey { + case size = "SIZE" + case inlined = "INLINED" + case prefixHex = "PREFIX_HEX" + case bufferIndex = "BUFFER_INDEX" + case offset = "OFFSET" + } +} + extension ArrowGold.Column { /// Filter for the valid values. @@ -206,12 +246,17 @@ extension ArrowGold.Column { let filteredData = data?.enumerated().map { index, value in validity[index] == 1 ? value : .null } + let filteredViews = views?.enumerated().map { index, value in + validity[index] == 1 ? value : nil + } return Self( name: name, count: count, validity: validity, offset: offset, data: filteredData, + views: filteredViews, + variadicDataBuffers: variadicDataBuffers, children: children?.map { $0.withoutJunkData() } ) } diff --git a/Tests/ArrowIPCTests/Gold/ArrowJSONEncoder.swift b/Tests/ArrowIPCTests/ArrowGoldEncoder.swift similarity index 58% rename from Tests/ArrowIPCTests/Gold/ArrowJSONEncoder.swift rename to Tests/ArrowIPCTests/ArrowGoldEncoder.swift index ed923e5..194d1fb 100644 --- a/Tests/ArrowIPCTests/Gold/ArrowJSONEncoder.swift +++ b/Tests/ArrowIPCTests/ArrowGoldEncoder.swift @@ -29,7 +29,7 @@ func encodeColumn( field: ArrowField ) throws(ArrowError) -> ArrowGold.Column { guard let array = array as? (any ArrowArrayProtocol) else { - throw .invalid("Expected ArrowArray, got \(type(of: array))") + throw .init(.invalid("Expected ArrowArray, got \(type(of: array))")) } // Validity is always present in the gold files. let validity: [Int] = (0.. 0 { switch field.type { - case .list(let listField): + case .list(let listField), .map(let listField, _): guard let listArray = array as? ListArrayProtocol else { - throw ArrowError.invalid("Expected list array") + throw .init(.invalid("Expected list array.")) } let childColumn = try encodeColumn( array: listArray.values, field: listField) @@ -67,7 +75,7 @@ func encodeColumn( data = nil case .fixedSizeList(let listField, _): guard let listArray = array as? ListArrayProtocol else { - throw ArrowError.invalid("Expected list array") + throw .init(.invalid("Expected fixed-size list array.")) } let childColumn = try encodeColumn( array: listArray.values, field: listField) @@ -75,7 +83,7 @@ func encodeColumn( data = nil case .strct(let arrowFields): guard let structArray = array as? ArrowStructArray else { - throw ArrowError.invalid("Expected list array") + throw .init(.invalid("Expected struct array.")) } children = [] for (arrowField, (_, array)) in zip(arrowFields, structArray.fields) { @@ -113,14 +121,41 @@ func encodeColumn( data = try extractFloatData(from: array, expectedType: Float32.self) case .float64: data = try extractFloatData(from: array, expectedType: Float64.self) + case .time32(_): + data = try extractIntData(from: array, expectedType: UInt32.self) + case .date32: + data = try extractIntData(from: array, expectedType: Int32.self) + case .date64: + data = try extractIntData(from: array, expectedType: Int64.self) + case .time64(_): + data = try extractIntData(from: array, expectedType: UInt64.self) + case .timestamp(_, _): + data = try extractIntData(from: array, expectedType: Int64.self) + case .duration(_): + data = try extractIntData(from: array, expectedType: Int64.self) + case .interval(.yearMonth): + data = try extractIntData(from: array, expectedType: Int32.self) + case .interval(.dayTime): + data = try extractIntData(from: array, expectedType: Int64.self) + case .interval(.monthDayNano): + // This is tricky - 128 bits (4 + 4 + 8 bytes) + // Might need special handling or extract as raw bytes + data = try extractIntData(from: array, expectedType: Int64.self) case .binary: try extractBinaryData(from: array, into: &data) case .fixedSizeBinary(_): try extractBinaryData(from: array, into: &data) case .utf8: try extractUtf8Data(from: array, into: &data) + case .binaryView, .utf8View: + try extractBinaryViewData( + from: array, + into: &views, + variadicDataBuffers: &variadicDataBuffers + ) default: - throw .invalid("Encoder did not handle a field type: \(field.type)") + throw .init( + .invalid("Encoder did not handle a field type: \(field.type)")) } } return .init( @@ -129,6 +164,8 @@ func encodeColumn( validity: validity, offset: offsets, data: data, + views: views, + variadicDataBuffers: variadicDataBuffers, children: children ) } @@ -138,7 +175,7 @@ func extractIntData( expectedType: T.Type ) throws(ArrowError) -> [DataValue] { guard let typedArray = array as? ArrowArrayNumeric else { - throw .invalid("Expected \(T.self) array, got \(type(of: array))") + throw .init(.invalid("Expected \(T.self) array, got \(type(of: array))")) } do { return try (0..( } } } catch { - throw .invalid("Failed to extract Int data: \(error)") + throw .init(.invalid("Failed to extract Int data: \(error)")) } } @@ -161,7 +198,7 @@ func extractFloatData( expectedType: T.Type ) throws(ArrowError) -> [DataValue] { guard let typedArray = array as? ArrowArrayNumeric else { - throw ArrowError.invalid("Expected \(T.self) array, got \(type(of: array))") + throw .init(.invalid("Expected \(T.self) array, got \(type(of: array))")) } let encoder = JSONEncoder() let decoder = JSONDecoder() @@ -184,11 +221,11 @@ func extractFloatData( let jsonNumber = try decoder.decode(Float.self, from: data) return .string(String(jsonNumber)) } else { - throw ArrowError.invalid("Expected float type") + throw ArrowError(.invalid("Expected float type")) } } } catch { - throw .ioError("Failed to round-trip float to/from JSON") + throw .init(.ioError("Failed to round-trip float to/from JSON")) } } @@ -196,7 +233,7 @@ func extractBoolData( from array: AnyArrowArrayProtocol ) throws(ArrowError) -> [DataValue] { guard let typedArray = array as? ArrowArrayBoolean else { - throw .invalid("Expected boolean array, got \(type(of: array))") + throw .init(.invalid("Expected boolean array, got \(type(of: array))")) } return (0.. Data? + let isStringView: Bool + if let stringArray = array as? StringArrayProtocol { + buffers = stringArray.buffers + length = stringArray.length + isStringView = true + getValue = { i in + guard let str = stringArray[i] else { return nil } + return Data(str.utf8) + } + } else if let binaryArray = array as? BinaryArrayProtocol { + buffers = binaryArray.buffers + length = binaryArray.length + isStringView = false + getValue = { i in binaryArray[i] } + } else { + throw .init(.invalid("Expected StringView or BinaryView array")) + } + + // Get the data buffers (skip null and views buffers) + let dataBuffers = Array(buffers.dropFirst(2)) + if !dataBuffers.isEmpty { + variadicDataBuffers = [] + } + + // Serialize buffers and track cumulative offsets + var bufferOffsets: [Int] = [0] + var cumulativeOffset = 0 + + for buffer in dataBuffers { + let hexString = buffer.withUnsafeBytes { ptr in + ptr.map { String(format: "%02X", $0) }.joined() + } + variadicDataBuffers?.append(hexString) + cumulativeOffset += buffer.length + bufferOffsets.append(cumulativeOffset) + } + // Helper to map global offset to (bufferIndex, localOffset) + func findBuffer(for globalOffset: Int) -> ( + bufferIndex: Int32, localOffset: Int32 + ) { + for i in 0..= bufferOffsets[i] && globalOffset < bufferOffsets[i + 1] + { + return (Int32(i), Int32(globalOffset - bufferOffsets[i])) + } + } + fatalError("Offset \(globalOffset) out of range") + } + // Track position in logical concatenated buffer + var logicalOffset = 0 + dataValues = (0.. View? in + guard let data = getValue(i) else { + return nil + } + let bytes = Array(data) + let size = Int32(bytes.count) + if size <= 12 { + // Inline - for strings use UTF-8 string, for binary use hex + let inlinedValue: String + if isStringView, let str = String(data: data, encoding: .utf8) { + inlinedValue = str + } else { + inlinedValue = bytes.map { String(format: "%02X", $0) }.joined() + } + return View(size: size, inlined: inlinedValue) + } else { + // Map to buffer index and local offset + let (bufferIndex, localOffset) = findBuffer(for: logicalOffset) + logicalOffset += bytes.count + + let prefix = bytes.prefix(4).map { String(format: "%02X", $0) }.joined() + return View( + size: size, + prefixHex: prefix, + bufferIndex: bufferIndex, + offset: localOffset + ) + } + } +} diff --git a/Tests/ArrowIPCTests/ArrowTestingGold.swift b/Tests/ArrowIPCTests/ArrowTestingGold.swift new file mode 100644 index 0000000..495f6b4 --- /dev/null +++ b/Tests/ArrowIPCTests/ArrowTestingGold.swift @@ -0,0 +1,316 @@ +// Copyright 2025 The Columnar Swift Contributors +// +// 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. + +import Foundation +import Testing + +@testable import Arrow +@testable import ArrowIPC + +/// Tests round trip from JSON -> Array -> JSON. +/// +/// See https://arrow.apache.org/docs/format/Integration.html#strategy +/// +/// The producer typically reads a JSON file, converts it to in-memory Arrow data, and exposes this data +/// using the format under test. The consumer reads the data in the said format and converts it back to +/// Arrow in-memory data; it also reads the same JSON file as the producer, and validates that both +/// datasets are identical. +struct ArrowTestingGold { + + static let testCases: [String] = [ + "generated_binary", + "generated_binary_view", + "generated_binary_no_batches", + "generated_binary_zerolength", + "generated_custom_metadata", + "generated_nested", + "generated_recursive_nested", + "generated_map", + "generated_datetime", + "generated_duration", + "generated_primitive", + "generated_primitive_no_batches", + "generated_primitive_zerolength", + ] + + static let allTestCases: [String] = [ + "generated_binary", + "generated_binary_no_batches", + "generated_binary_view", + "generated_binary_zerolength", + "generated_custom_metadata", + "generated_datetime", + "generated_decimal", + "generated_decimal256", + "generated_decimal32", + "generated_decimal64", + "generated_dictionary", + "generated_dictionary_unsigned", + "generated_duplicate_fieldnames", + "generated_duration", + "generated_extension", + "generated_interval", + "generated_interval_mdn", + "generated_large_binary", + "generated_list_view", + "generated_map", + "generated_map_non_canonical", + "generated_nested", + "generated_nested_dictionary", + "generated_nested_large_offsets", + "generated_null", + "generated_null_trivial", + "generated_primitive", + "generated_primitive_no_batches", + "generated_primitive_zerolength", + "generated_recursive_nested", + "generated_run_end_encoded", + "generated_union", + ] + + @Test(arguments: testCases) + func read(name: String) throws { + let resourceURL = try loadTestResource( + name: name, + withExtension: "json.lz4", + subdirectory: "integration/cpp-21.0.0" + ) + let lz4Data = try Data(contentsOf: resourceURL) + let lz4 = try LZ4(parsing: lz4Data) + let testCase = try JSONDecoder().decode(ArrowGold.self, from: lz4.data) + let testFile = try loadTestResource( + name: name, + withExtension: "arrow_file", + subdirectory: "integration/cpp-21.0.0" + ) + let arrowReader = try ArrowReader(url: testFile) + let (arrowSchema, recordBatches) = try arrowReader.read() + #expect(testCase.batches.count == recordBatches.count) + // Strip placeholder values. + let expectedBatches = testCase.batches.map { batch in + ArrowGold.Batch( + count: batch.count, + columns: batch.columns.map { $0.withoutJunkData() } + ) + } + let expectedSchema = testCase.schema + let expectedDictionaries = testCase.dictionaries + let expectedGold = ArrowGold( + schema: expectedSchema, + batches: expectedBatches, + dictionaries: expectedDictionaries + ) + // These comparisons are redundant but help pinpoint where issues arise. + let actualSchema = encode(schema: arrowSchema) + #expect(actualSchema == expectedSchema) + let actualBatches = try encode(batches: recordBatches, schema: arrowSchema) + #expect(actualBatches.count == expectedBatches.count) + if actualBatches != expectedBatches { + try diffEncodable(actualBatches, expectedBatches, maxLinesToPrint: 20) + try printCodable(actualBatches) + } + #expect(actualBatches == expectedBatches) + let actualGold = ArrowGold( + schema: actualSchema, + batches: actualBatches, + dictionaries: nil + ) + // The gold-standard comparison. + #expect(actualGold == expectedGold) + } + + @Test(arguments: testCases) + func write(name: String) async throws { + let resourceURL = try loadTestResource( + name: name, + withExtension: "json.lz4", + subdirectory: "integration/cpp-21.0.0" + ) + let lz4Data = try Data(contentsOf: resourceURL) + let lz4 = try LZ4(parsing: lz4Data) + let testCase = try JSONDecoder().decode(ArrowGold.self, from: lz4.data) + let testFile = try loadTestResource( + name: name, + withExtension: "arrow_file", + subdirectory: "integration/cpp-21.0.0" + ) + let arrowReader = try ArrowReader(url: testFile) + let (arrowSchema, recordBatchesExpected) = try arrowReader.read() + + let tempDir = FileManager.default.temporaryDirectory + let tempFile = tempDir.appendingPathComponent(UUID().uuidString + ".arrow") + + var arrowWriter = ArrowWriter(url: tempFile) + try arrowWriter.write( + schema: arrowSchema, + recordBatches: recordBatchesExpected + ) + try arrowWriter.finish() + + // FileManager.default.createFile(atPath: "/tmp/\(name).arrow", contents: arrowWriter.data) + + // pyArrow round trip only supported on macOS. + #if os(macOS) + let ipcData = try await pyArrowRoundTrip(ipcData: arrowWriter.data) + #else + let ipcData = arrowWriter.data + #endif + + let testReader = try ArrowReader(data: ipcData) + let (arrowSchemaRead, recordBatchesRead) = try testReader.read() + + for recordBatch in recordBatchesRead { + let lengths = recordBatch.arrays.map(\.length) + guard let first = lengths.first else { + Issue.record("Empty batch") + return + } + guard lengths.allSatisfy({ $0 == first }) else { + Issue.record("Mixed-length batch.") + return + } + } + + let actualSchema = encode(schema: arrowSchemaRead) + let expectedSchema = testCase.schema + let expectedBatches = testCase.batches.map { batch in + ArrowGold.Batch( + count: batch.count, + columns: batch.columns.map { $0.withoutJunkData() } + ) + } + let expectedDictionaries = testCase.dictionaries + let expectedGold = ArrowGold( + schema: expectedSchema, + batches: expectedBatches, + dictionaries: expectedDictionaries + ) + if actualSchema != expectedSchema { + try diffEncodable(actualSchema, expectedSchema) + try printCodable(actualSchema) + try printCodable(expectedSchema) + // return + } + #expect(actualSchema == expectedSchema) + #expect(recordBatchesRead.count == expectedBatches.count) + let actualBatches = try encode( + batches: recordBatchesRead, schema: arrowSchema) + + if actualBatches != expectedBatches { + for (a, e) in zip(actualBatches, expectedBatches) where a != e { + for (aField, eField) in zip(a.columns, e.columns) { + if aField == eField { + print("MATCH: \(aField)") + } else { + try printCodable(aField) + try printCodable(eField) + try diffEncodable(aField, eField) + return + } + } + } + } + + #expect(actualBatches == expectedBatches) + let actualGold = ArrowGold( + schema: actualSchema, + batches: actualBatches, + dictionaries: nil + ) + // The gold-standard comparison. + #expect(actualGold == expectedGold) + } +} + +private func encode(schema: ArrowSchema) -> ArrowGold.Schema { + let fields = schema.fields.map { arrowField in + arrowField.toGoldField() + } + let encodedMetadata: [String: String]? = + switch schema.metadata { + case .none: nil + case .some(let metadata): metadata.isEmpty ? nil : metadata + } + return .init(fields: fields, metadata: encodedMetadata) +} + +private func encode( + batches: [RecordBatch], + schema: ArrowSchema +) throws -> [ArrowGold.Batch] { + try batches.map { recordBatch in + var columns: [ArrowGold.Column] = [] + for (field, array) in zip(schema.fields, recordBatch.arrays) { + let encoded = try encodeColumn(array: array, field: field) + columns.append(encoded) + } + return .init(count: recordBatch.length, columns: columns) + } +} + +/// A utility to diff encodable objects, useful in tests encoding to JSON. +/// - Parameters: +/// - actual: The actual +/// - expected: The expected `Encodable` object. +/// - label: An optional label to differentiate multiple diffs. +/// - maxLinesToPrint: An optional limit to the number of lines printed. +/// - Throws: An error if encoding fails or string data is unrepresentable in utf8. +func diffEncodable( + _ actual: T, + _ expected: T, + label: String = "", + maxLinesToPrint: Int? = nil +) throws { + let encoder = JSONEncoder() + encoder.outputFormatting = [.prettyPrinted, .sortedKeys] + let actualJSON = try encoder.encode(actual) + let expectedJSON = try encoder.encode(expected) + guard + let actualString = String(data: actualJSON, encoding: .utf8), + let expectedString = String(data: expectedJSON, encoding: .utf8) + else { + throw ArrowError(.runtimeError("Invalid UTF-8 data.")) + } + let actualLines = actualString.split(separator: "\n") + let expectedLines = expectedString.split(separator: "\n") + let maxLines = max(actualLines.count, expectedLines.count) + var hasDifferences = false + var printedLines = 0 + for i in 0..= limit { + let remaining = (i.. [ArrowGold.Field]? { switch self { - case .list(let field), .largeList(let field), .fixedSizeList(let field, _): + case .list(let field), .largeList(let field), .fixedSizeList(let field, _), + .map(let field, _): return [field.toGoldField()] case .strct(let fields): return fields.map { $0.toGoldField() } diff --git a/Tests/ArrowIPCTests/ArrowWriterTests.swift b/Tests/ArrowIPCTests/ArrowWriterTests.swift deleted file mode 100644 index 52014d4..0000000 --- a/Tests/ArrowIPCTests/ArrowWriterTests.swift +++ /dev/null @@ -1,75 +0,0 @@ -// Copyright 2025 The Columnar Swift Contributors -// -// 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. - -import Foundation -import Testing - -@testable import Arrow -@testable import ArrowIPC - -struct ArrowWriterTests { - - @Test func writeBasics() throws { - - let outputUrl = FileManager.default.temporaryDirectory - .appending(path: "bool-test.arrow") - let writer = ArrowWriter(url: outputUrl) - #expect(writer.data.count == 8) - - } - - @Test func writeBoolean() throws { - - let schema: ArrowSchema = ArrowSchema.Builder() - .addField("one", type: .boolean, isNullable: true) - .addField("two", type: .utf8, isNullable: true) - .finish() - - let builder = ArrayBuilderBoolean() - builder.append(true) - builder.append(false) - builder.appendNull() - builder.append(false) - builder.append(true) - let one = builder.finish() - - let builder2 = ArrayBuilderString() - builder2.append("zero") - builder2.append("one") - builder2.append("two") - builder2.append("three") - builder2.append("four") - let two = builder2.finish() - - let recordBatch = RecordBatch(schema: schema, columns: [one, two]) - - checkBoolRecordBatch(recordBatch: recordBatch) - - let outputUrl = FileManager.default.temporaryDirectory - .appending(path: "bool-test.arrow") - var writer = ArrowWriter(url: outputUrl) - try writer.write(schema: schema, recordBatches: [recordBatch]) - try writer.finish() - - let arrowReader = try ArrowReader(url: outputUrl) - let (arrowSchema, recordBatches) = try arrowReader.read() - - for recordBatch in recordBatches { - checkBoolRecordBatch(recordBatch: recordBatch) - } - // try FileManager.default.copyItem(at: outputUrl, to: URL(fileURLWithPath: "/tmp/bool-test-swift.arrow")) - - } - -} diff --git a/Tests/ArrowIPCTests/ArrowReaderTests.swift b/Tests/ArrowIPCTests/BasicTests.swift similarity index 65% rename from Tests/ArrowIPCTests/ArrowReaderTests.swift rename to Tests/ArrowIPCTests/BasicTests.swift index be869d4..1462f8f 100644 --- a/Tests/ArrowIPCTests/ArrowReaderTests.swift +++ b/Tests/ArrowIPCTests/BasicTests.swift @@ -12,12 +12,13 @@ // See the License for the specific language governing permissions and // limitations under the License. -import Arrow +import Foundation import Testing +@testable import Arrow @testable import ArrowIPC -struct ArrowReaderTests { +struct BasicTests { @Test func boolFile() throws { let url = try loadTestResource(name: "testdata_bool") @@ -73,7 +74,7 @@ struct ArrowReaderTests { @Test func structFile() throws { let url = try loadTestResource(name: "testdata_struct") let arrowReader = try ArrowReader(url: url) - let (arrowSchema, recordBatches) = try arrowReader.read() + let (_, recordBatches) = try arrowReader.read() for recordBatch in recordBatches { let structArray = try #require( recordBatch.arrays[0] as? ArrowStructArray) @@ -93,4 +94,55 @@ struct ArrowReaderTests { #expect(boolArray.length == 3) } } + + @Test func writeBasics() throws { + + let outputUrl = FileManager.default.temporaryDirectory + .appending(path: "bool-test.arrow") + let writer = ArrowWriter(url: outputUrl) + #expect(writer.data.count == 8) + + } + + @Test func writeBoolean() throws { + let schema: ArrowSchema = ArrowSchema.Builder() + .addField("one", type: .boolean, isNullable: true) + .addField("two", type: .utf8, isNullable: true) + .finish() + + let builder = ArrayBuilderBoolean() + builder.append(true) + builder.append(false) + builder.appendNull() + builder.append(false) + builder.append(true) + let one = builder.finish() + + let builder2 = ArrayBuilderString() + builder2.append("zero") + builder2.append("one") + builder2.append("two") + builder2.append("three") + builder2.append("four") + let two = builder2.finish() + + let recordBatch = RecordBatch(schema: schema, columns: [one, two]) + + checkBoolRecordBatch(recordBatch: recordBatch) + + let outputUrl = FileManager.default.temporaryDirectory + .appending(path: "bool-test.arrow") + var writer = ArrowWriter(url: outputUrl) + try writer.write(schema: schema, recordBatches: [recordBatch]) + try writer.finish() + + let arrowReader = try ArrowReader(url: outputUrl) + let (_, recordBatches) = try arrowReader.read() + + for recordBatch in recordBatches { + checkBoolRecordBatch(recordBatch: recordBatch) + } + // try FileManager.default.copyItem(at: outputUrl, to: URL(fileURLWithPath: "/tmp/bool-test-swift.arrow")) + + } } diff --git a/Tests/ArrowIPCTests/Gold/ArrowTestingGold.swift b/Tests/ArrowIPCTests/Gold/ArrowTestingGold.swift deleted file mode 100644 index ab15cbf..0000000 --- a/Tests/ArrowIPCTests/Gold/ArrowTestingGold.swift +++ /dev/null @@ -1,473 +0,0 @@ -// Copyright 2025 The Columnar Swift Contributors -// -// 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. - -import Foundation -import Testing - -@testable import Arrow -@testable import ArrowIPC - -struct ArrowTestingIPC { - - static let allTests = [ - "generated_binary", - "generated_binary_no_batches", - "generated_binary_view", - "generated_binary_zerolength", - "generated_custom_metadata", - "generated_datetime", - "generated_decimal", - "generated_decimal256", - "generated_decimal32", - "generated_decimal64", - "generated_dictionary", - "generated_dictionary_unsigned", - "generated_duplicate_fieldnames", - "generated_duration", - "generated_extension", - "generated_interval", - "generated_interval_mdn", - "generated_large_binary", - "generated_list_view", - "generated_map", - "generated_map_non_canonical", - "generated_nested", - "generated_nested_dictionary", - "generated_nested_large_offsets", - "generated_null", - "generated_null_trivial", - "generated_primitive", - "generated_primitive_no_batches", - "generated_primitive_zerolength", - "generated_recursive_nested", - "generated_run_end_encoded", - "generated_union", - ] - - static let testCases: [String] = [ - "generated_primitive", - "generated_primitive_no_batches", - "generated_primitive_zerolength", - "generated_binary", - "generated_binary_zerolength", - "generated_binary_no_batches", - "generated_custom_metadata", - "generated_nested", - ] - - // @Test(.serialized, arguments: testCases) - @Test(arguments: testCases) - func gold(name: String) throws { - - // print(name) - // print(Self.testCases) - - // let todos = Set(Self.allTests).subtracting(Set(Self.testCases)) - // for todo in todos.sorted() { - // print(todo) - // } - - let resourceURL = try loadTestResource( - name: name, - withExtension: "json.lz4", - subdirectory: "integration/cpp-21.0.0" - ) - let lz4Data = try Data(contentsOf: resourceURL) - let lz4 = try LZ4(parsing: lz4Data) - let testCase = try JSONDecoder().decode(ArrowGold.self, from: lz4.data) - let testFile = try loadTestResource( - name: name, - withExtension: "arrow_file", - subdirectory: "integration/cpp-21.0.0" - ) - let arrowReader = try ArrowReader(url: testFile) - let (arrowSchema, recordBatches) = try arrowReader.read() - - #expect(testCase.batches.count == recordBatches.count) - - let expectedMetadata = testCase.schema.metadata ?? [:] - #expect(expectedMetadata == arrowSchema.metadata) - - for (testBatch, recordBatch) in zip(testCase.batches, recordBatches) { - for ( - (arrowField, arrowArray), - (expectedField, expectedColumn) - ) in zip( - zip(arrowSchema.fields, recordBatch.arrays), - zip(testCase.schema.fields, testBatch.columns) - ) { - - #expect(arrowField.name == expectedField.name) - #expect(arrowField.isNullable == expectedField.nullable) - #expect(arrowField.type.matches(expectedField: expectedField)) - #expect(arrowArray.length == expectedColumn.count) - #expect(arrowField.name == expectedColumn.name) - // #expect(arrowField.metadata == expectedMetadata) - - switch arrowField.type { - case .fixedSizeBinary(let byteWidth): - guard let expectedByteWidth = expectedField.type.byteWidth else { - throw ArrowError.invalid( - "Test case is missing byteWidth for fixedSizeBinary field." - ) - } - #expect(expectedByteWidth == byteWidth) - guard let actual = arrowArray as? any BinaryArrayProtocol else { - Issue.record( - "Expected ArrowArrayOfData but got \(type(of: arrowArray))" - ) - continue - } - try testFixedWidthBinary(actual: actual, expected: expectedColumn) - case .boolean: - try testBoolean(actual: arrowArray, expected: expectedColumn) - case .int8: - try testFixedWidth( - actual: arrowArray, expected: expectedColumn, as: Int8.self) - case .uint8: - try testFixedWidth( - actual: arrowArray, expected: expectedColumn, as: UInt8.self) - case .int16: - try testFixedWidth( - actual: arrowArray, expected: expectedColumn, as: Int16.self) - case .uint16: - try testFixedWidth( - actual: arrowArray, expected: expectedColumn, as: UInt16.self) - case .int32: - try testFixedWidth( - actual: arrowArray, expected: expectedColumn, as: Int32.self) - case .uint32: - try testFixedWidth( - actual: arrowArray, expected: expectedColumn, as: UInt32.self) - case .int64: - try testFixedWidth( - actual: arrowArray, expected: expectedColumn, as: Int64.self) - case .uint64: - try testFixedWidth( - actual: arrowArray, expected: expectedColumn, as: UInt64.self) - case .float32: - try testFixedWidth( - actual: arrowArray, expected: expectedColumn, as: Float.self) - case .float64: - try testFixedWidth( - actual: arrowArray, expected: expectedColumn, as: Double.self) - case .binary: - try testVariableLength( - actual: arrowArray, expected: expectedColumn, type: arrowField.type) - case .utf8: - try testVariableLength( - actual: arrowArray, expected: expectedColumn, type: arrowField.type) - case .list(_): - try validateListArray(actual: arrowArray, expected: expectedColumn) - break - case .fixedSizeList(_, let listSize): - try validateFixedWidthListArray( - actual: arrowArray, - expected: expectedColumn, - listSize: listSize - ) - break - default: - // throw ArrowError.invalid( - print( - "TODO: Implement test for arrow field type: \(arrowField.type)") - } - } - } - } - - func testFixedWidthBinary( - actual: any BinaryArrayProtocol, - expected: ArrowGold.Column, - ) throws { - guard let validity = expected.validity, let dataValues = expected.data - else { - throw ArrowError.invalid("Test column is incomplete.") - } - for (i, isNull) in validity.enumerated() { - guard case .string(let hex) = dataValues[i] else { - throw ArrowError.invalid("Data values are not all strings.") - } - guard let data = Data(hex: hex) else { - Issue.record("Failed to decode data from hex: \(hex)") - return - } - if isNull == 0 { - #expect(actual[i] == nil) - } else { - #expect(actual[i] == data) - } - } - } - - func testBoolean( - actual: AnyArrowArrayProtocol, - expected: ArrowGold.Column - ) throws { - guard let expectedValidity = expected.validity, - let expectedValues = expected.data - else { - throw ArrowError.invalid("Test column is incomplete.") - } - guard let array = actual as? ArrowArrayBoolean, - array.length == expectedValidity.count - else { - Issue.record("Array type mismatch") - return - } - for (i, isNull) in expectedValidity.enumerated() { - guard case .bool(let expectedValue) = expectedValues[i] else { - throw ArrowError.invalid("Expected boolean value") - } - if isNull == 0 { - #expect(array[i] == nil) - } else { - #expect(array[i] == expectedValue) - } - } - } - - func testFixedWidth( - actual: AnyArrowArrayProtocol, - expected: ArrowGold.Column, - as type: T.Type - ) throws where T: BinaryInteger & LosslessStringConvertible { - guard let expectedValidity = expected.validity, - let expectedValues = expected.data - else { - throw ArrowError.invalid("Test column is incomplete.") - } - guard let array = actual as? any ArrowArrayProtocol, - array.length == expectedValidity.count - else { - Issue.record("Array type mismatch") - return - } - for (i, isNull) in expectedValidity.enumerated() { - let expected: T - if case .int(let intVal) = expectedValues[i] { - expected = try T(throwingOnOverflow: intVal) - } else if case .string(let strVal) = expectedValues[i], - let parsed = T(strVal) - { - expected = parsed - } else { - throw ArrowError.invalid("Expected integer value or numeric string") - } - - if isNull == 0 { - #expect(array[i] == nil) - } else { - #expect(array[i] as? T == expected) - } - } - } - - func testFixedWidth( - actual: AnyArrowArrayProtocol, - expected: ArrowGold.Column, - as type: T.Type - ) throws where T: BinaryFloatingPoint & LosslessStringConvertible { - guard let expectedValidity = expected.validity, - let expectedValues = expected.data - else { - throw ArrowError.invalid("Test column is incomplete.") - } - guard let array = actual as? any ArrowArrayProtocol, - array.length == expectedValidity.count - else { - Issue.record("Array type mismatch") - return - } - for (i, isValid) in expectedValidity.enumerated() { - guard case .string(let strVal) = expectedValues[i], - let expected = T(strVal) - else { - throw ArrowError.invalid("Expected float value or numeric string") - } - if isValid == 1 { - #expect(array[i] as? T == expected) - print("comparing \(array[i]) to \(expected)") - } else { - #expect(array[i] == nil) - } - } - } - - func validateFixedWidthListArray( - actual: AnyArrowArrayProtocol, - expected: ArrowGold.Column, - listSize: Int32 - ) throws { - - guard let expectedValidity = expected.validity - else { - throw ArrowError.invalid("Test column is incomplete.") - } - guard let listArray = actual as? ArrowFixedSizeListArray - else { - Issue.record("Unexpected array type: \(type(of: actual))") - return - } - - for (i, isNull) in expectedValidity.enumerated() { - if isNull == 0 { - #expect(listArray[i] == nil) - } else { - guard let actualChildSlice = listArray[i] else { - Issue.record("Expected non-null list at index \(i)") - continue - } - #expect(actualChildSlice.length == listSize) - } - } - } - - func validateListArray( - actual: AnyArrowArrayProtocol, - expected: ArrowGold.Column - ) throws { - guard let expectedValidity = expected.validity, - let expectedOffsets = expected.offset - else { - throw ArrowError.invalid("Test column is incomplete.") - } - - // Validate the offsets buffer - actual.buffers[1].withUnsafeBytes { ptr in - let offsets = ptr.bindMemory(to: Int32.self) - #expect(offsets.count == expectedOffsets.count) - for (i, expectedOffset) in expectedOffsets.enumerated() { - let actualOffset = offsets[i] - #expect(actualOffset == expectedOffset) - } - } - - // TODO: Need a simpler type signature at call site. - guard let listArray = actual as? ArrowListArray> - else { - Issue.record("Unexpected array type: \(type(of: actual))") - return - } - - guard let child = expected.children?.first else { - throw ArrowError.invalid("List array missing child column") - } - - // Validate each list entry - for (i, isNull) in expectedValidity.enumerated() { - if isNull == 0 { - #expect(listArray[i] == nil) - } else { - guard let actualChildSlice = listArray[i] else { - Issue.record("Expected non-null list at index \(i)") - continue - } - - // Get expected range from offsets - let childStartOffset = Int(expectedOffsets[i]) - let childEndOffset = Int(expectedOffsets[i + 1]) - let expectedLength = childEndOffset - childStartOffset - - #expect(actualChildSlice.length == expectedLength) - - // Validate each element in this list - for j in 0.. Array -> JSON. -/// -/// See https://arrow.apache.org/docs/format/Integration.html#strategy -/// -/// The producer typically reads a JSON file, converts it to in-memory Arrow data, and exposes this data -/// using the format under test. The consumer reads the data in the said format and converts it back to -/// Arrow in-memory data; it also reads the same JSON file as the producer, and validates that both -/// datasets are identical. -/// -struct ArrowTestingJSON { - - static let testCases: [String] = [ - "generated_primitive", - "generated_primitive_no_batches", - "generated_primitive_zerolength", - "generated_binary", - "generated_binary_zerolength", - "generated_custom_metadata", - "generated_nested", - "generated_recursive_nested", - ] - - @Test(arguments: testCases) - func json(name: String) throws { - let resourceURL = try loadTestResource( - name: name, - withExtension: "json.lz4", - subdirectory: "integration/cpp-21.0.0" - ) - let lz4Data = try Data(contentsOf: resourceURL) - let lz4 = try LZ4(parsing: lz4Data) - let testCase = try JSONDecoder().decode(ArrowGold.self, from: lz4.data) - let testFile = try loadTestResource( - name: name, - withExtension: "arrow_file", - subdirectory: "integration/cpp-21.0.0" - ) - let arrowReader = try ArrowReader(url: testFile) - let (arrowSchema, recordBatches) = try arrowReader.read() - - #expect(testCase.batches.count == recordBatches.count) - - // Strip placeholder values. - let expectedBatches = testCase.batches.map { batch in - ArrowGold.Batch( - count: batch.count, - columns: batch.columns.map { $0.withoutJunkData() } - ) - } - let expectedSchema = testCase.schema - let expectedDictionaries = testCase.dictionaries - let _ = ArrowGold( - schema: expectedSchema, - batches: expectedBatches, - dictionaries: expectedDictionaries - ) - let actualSchema = encodeSchema(schema: arrowSchema) - #expect(actualSchema == expectedSchema) - for (testBatch, recordBatch) in zip(expectedBatches, recordBatches) { - for ( - (arrowField, arrowArray), - (_, expected) - ) in zip( - zip(arrowSchema.fields, recordBatch.arrays), - zip(testCase.schema.fields, testBatch.columns) - ) { - let actual = try encodeColumn(array: arrowArray, field: arrowField) - - #expect(actual == expected) - - // This is just useful for pin-pointing differences. - if actual != expected { - print("==== \(expected.name) ====") - #expect(actual.validity == expected.validity) - #expect(actual.offset == expected.offset) - if actual.data != expected.data { - guard let actualData = actual.data, - let expectedData = expected.data, let validity = actual.validity - else { - throw ArrowError.invalid("Expected and actual data both nil") - } - for (i, isValid) in validity.enumerated() { - if isValid == 1 { - let aV = actualData[i] - let eV = expectedData[i] - #expect(aV == eV) - } - } - } - } - } - } - } - -} - -private func encodeSchema(schema: ArrowSchema) -> ArrowGold.Schema { - let fields = schema.fields.map { arrowField in - arrowField.toGoldField() - } - let encodedMetadata: [String: String]? = - switch schema.metadata { - case .none: nil - case .some(let metadata): metadata.isEmpty ? nil : metadata - } - - return .init(fields: fields, metadata: encodedMetadata) -} diff --git a/Tests/ArrowIPCTests/LZ4.swift b/Tests/ArrowIPCTests/LZ4.swift index 4711df5..8ce9fa5 100644 --- a/Tests/ArrowIPCTests/LZ4.swift +++ b/Tests/ArrowIPCTests/LZ4.swift @@ -118,7 +118,7 @@ public struct LZ4 { init(parsing input: inout ParserSpan) throws { let magicNumber = try UInt32(parsingLittleEndian: &input) guard magicNumber == 0x184D_2204 else { - throw ArrowError.invalid("Invalid magic number") + throw ArrowError(.invalid("Invalid magic number")) } self.flags = try Flags(parsingLittleEndian: &input) @@ -195,7 +195,7 @@ public struct LZ4 { self.length = try Int( parsingLZ4Sequence: &input, token: copyLengthToken, constant: 4) guard offset != 0 else { - throw ArrowError.invalid("2") + throw ArrowError(.invalid("2")) } } } @@ -219,7 +219,7 @@ extension LZ4: ExpressibleByParsing { if size.isCompressed { guard let max = header.flags.blockMaximum else { - throw ArrowError.invalid("Invalid block maximum in header flags") + throw ArrowError(.invalid("Invalid block maximum in header flags")) } // TODO: Switch to OutputSpan when we can implement a copying append var blockData = Data(capacity: max.bytes) @@ -232,9 +232,10 @@ extension LZ4: ExpressibleByParsing { } guard seq.offset <= blockData.count else { - throw ArrowError.invalid( - "Tried to offset too much: \(seq.offset) with count \(blockData.count)" - ) + throw ArrowError( + .invalid( + "Tried to offset too much: \(seq.offset) with count \(blockData.count)" + )) } if seq.offset == 0 { diff --git a/Tests/ArrowIPCTests/OvertureReaderTests.swift b/Tests/ArrowIPCTests/OvertureReaderTests.swift new file mode 100644 index 0000000..2a7d7a0 --- /dev/null +++ b/Tests/ArrowIPCTests/OvertureReaderTests.swift @@ -0,0 +1,33 @@ +// Copyright 2025 The Columnar Swift Contributors +// +// 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. + +import Arrow +import Testing + +@testable import ArrowIPC + +struct OvertureReaderTests { + + @Test func overtureFile() throws { + + let url = try loadTestResource(name: "overture_sample") + print(url) + let reader = try ArrowReader(url: url) + + let (_, recordBatches) = try reader.read() + for recordBatch in recordBatches { + print(recordBatch) + } + } +} diff --git a/Tests/ArrowIPCTests/Resources/overture_sample.arrow b/Tests/ArrowIPCTests/Resources/overture_sample.arrow new file mode 100644 index 0000000..4046f3a Binary files /dev/null and b/Tests/ArrowIPCTests/Resources/overture_sample.arrow differ diff --git a/Tests/ArrowIPCTests/Subprocess.swift b/Tests/ArrowIPCTests/Subprocess.swift new file mode 100644 index 0000000..2df011d --- /dev/null +++ b/Tests/ArrowIPCTests/Subprocess.swift @@ -0,0 +1,59 @@ +// Copyright 2025 The Columnar Swift Contributors +// +// 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. + +import Arrow +import Foundation +import Subprocess + +/// Deserialize Arrow IPC and re-serialize to IPC. +/// - Parameter ipcData: The IPC data. +/// - Throws: An error if either the subprocess call fails, or the called script fails. +/// - Returns: The +func pyArrowRoundTrip(ipcData: Data) async throws -> Data { + + // Validate with PyArrow + let result = try await run( + .path("/opt/anaconda3/bin/python3"), + arguments: [ + "-c", + """ + import pyarrow as pa + import sys + from io import BytesIO + + input_data = sys.stdin.buffer.read() + buffer = BytesIO(input_data) + + reader = pa.ipc.open_file(buffer) + + writer = pa.ipc.new_file(sys.stdout.buffer, reader.schema) + for i in range(reader.num_record_batches): + batch = reader.get_batch(i) + writer.write_batch(batch) + writer.close() + """, + ], + input: .data(ipcData), + output: .data(limit: 100_000_000), + error: .string(limit: 4096) + ) + + guard result.terminationStatus.isSuccess else { + let errorMessage = result.standardError ?? "Unknown error" + throw ArrowError( + .invalid("Unable to validate IPC data with PyArrow: \(errorMessage).")) + } + + return result.standardOutput +} diff --git a/Tests/ArrowIPCTests/TestSupport.swift b/Tests/ArrowIPCTests/TestSupport.swift index f19c28a..5bb0c9f 100644 --- a/Tests/ArrowIPCTests/TestSupport.swift +++ b/Tests/ArrowIPCTests/TestSupport.swift @@ -23,7 +23,7 @@ func printTestJSON(_ value: ArrowGold) throws { encoder.outputFormatting = .prettyPrinted let result = try encoder.encode(value) guard let formattedString = String(data: result, encoding: .utf8) else { - throw ArrowError.unknownError("Unable to encode JSON.") + throw ArrowError(.unknownError("Unable to encode JSON.")) } print(formattedString) } @@ -38,7 +38,8 @@ func loadTestResource( ) { return resource } else { - throw .runtimeError("Couldn't find \(name).\(ext) in the test resources.") + throw .init( + .runtimeError("Couldn't find \(name).\(ext) in the test resources.")) } } @@ -94,3 +95,31 @@ extension Data { self = data } } + +/// Pretty print an encodable value. +/// - Parameter value: The value to print. +/// - Throws: On failed utf8 encoding. +func printCodable(_ value: T) throws { + let encoder = JSONEncoder() + encoder.outputFormatting = [.prettyPrinted, .sortedKeys] + let data = try encoder.encode(value) + guard let formattted = String(data: data, encoding: .utf8) else { + throw ArrowError(.invalid("UTF-8 encode failed.")) + } + print(formattted) +} + +extension ArrowArrayVariable { + + /// Debug print offsets buffer. + func printOffsets() { + // Print offsets buffer values + buffers[1].withUnsafeBytes { bufferPtr in + let typedPtr = bufferPtr.bindMemory(to: OffsetType.self) + print("Offsets buffer (\(typedPtr.count) elements):") + for i in 0.. = - try ArrowArrayBuilders.loadNumberArrayBuilder() - for index in 0..<100 { - arrayBuilder.append(UInt8(index)) - } - - #expect(arrayBuilder.nullCount == 0) - arrayBuilder.append(nil) - #expect(arrayBuilder.length == 101) - #expect(arrayBuilder.capacity == 128) - #expect(arrayBuilder.nullCount == 1) - let array = try arrayBuilder.finish() - #expect(array.length == 101) - #expect(array[1]! == 1) - #expect(array[10]! == 10) - #expect(try array.isNull(at: 100) == true) - - // for buffer in array.arrowData.buffers { - // let dataAddress = UInt(bitPattern: buffer.rawPointer) - // #expect(dataAddress % 64 == 0, "Buffer should be 64-byte aligned") - // } - } - - @Test func doubleArray() throws { - - // MARK: Double array - let doubleBuilder: NumberArrayBuilder = - try ArrowArrayBuilders.loadNumberArrayBuilder() - doubleBuilder.append(14) - doubleBuilder.append(40.4) - #expect(doubleBuilder.nullCount == 0) - #expect(doubleBuilder.length == 2) - #expect(doubleBuilder.capacity == 256) - let doubleArray = try doubleBuilder.finish() - #expect(doubleArray.length == 2) - #expect(doubleArray[0]! == 14) - #expect(doubleArray[1]! == 40.4) - } - - @Test func stringArray() throws { - let stringBuilder = try ArrowArrayBuilders.loadStringArrayBuilder() - for index in 0..<100 { - if index % 10 == 9 { - stringBuilder.append(nil) - } else { - stringBuilder.append("test" + String(index)) - } - } - - #expect(stringBuilder.nullCount == 10) - #expect(stringBuilder.length == 100) - #expect(stringBuilder.capacity == 640) - let stringArray = try stringBuilder.finish() - #expect(stringArray.length == 100) - for index in 0..) - for index in 0..<100 { - uint8HBuilder.appendAny(UInt8(index)) - } - - let uint8Holder = try uint8HBuilder.toAnyArrowArray() - #expect(uint8Holder.nullCount == 0) - #expect(uint8Holder.length == 100) - - let stringHBuilder: AnyArrowArrayBuilder = - (try ArrowArrayBuilders.loadStringArrayBuilder()) - for index in 0..<100 { - if index % 10 == 9 { - stringHBuilder.appendAny(nil) - } else { - stringHBuilder.appendAny("test" + String(index)) - } - } - - let stringHolder = try stringHBuilder.toAnyArrowArray() - #expect(stringHolder.nullCount == 10) - #expect(stringHolder.length == 100) - } - - @Test func addVArgs() throws { - let arrayBuilder: NumberArrayBuilder = - try ArrowArrayBuilders.loadNumberArrayBuilder() - arrayBuilder.append(0, 1, 2, 3, 4, 5, 6, 7, 8, 9) - #expect(arrayBuilder.length == 10) - #expect(try arrayBuilder.finish()[2] == 2) - let doubleBuilder: NumberArrayBuilder = - try ArrowArrayBuilders.loadNumberArrayBuilder() - doubleBuilder.append(0, 1.1, 2.2, 3.3, 4.4, 5.5, 6.6, 7.7, 8.8) - #expect(doubleBuilder.length == 9) - #expect(try doubleBuilder.finish()[4] == 4.4) - let stringBuilder = try ArrowArrayBuilders.loadStringArrayBuilder() - stringBuilder.append("0", "1", "2", "3", "4", "5", "6") - #expect(stringBuilder.length == 7) - #expect(try stringBuilder.finish()[4] == "4") - let boolBuilder = try ArrowArrayBuilders.loadBoolArrayBuilder() - boolBuilder.append(true, false, true, false) - #expect(try boolBuilder.finish()[2] == true) - } - - @Test func addArray() throws { - let arrayBuilder: NumberArrayBuilder = - try ArrowArrayBuilders.loadNumberArrayBuilder() - arrayBuilder.append([0, 1, 2, 3, 4, 5, 6, 7, 8, 9]) - #expect(arrayBuilder.length == 10) - #expect(try arrayBuilder.finish()[2] == 2) - let doubleBuilder: NumberArrayBuilder = - try ArrowArrayBuilders.loadNumberArrayBuilder() - doubleBuilder.append([0, 1.1, 2.2, 3.3, 4.4, 5.5, 6.6, 7.7, 8.8]) - #expect(doubleBuilder.length == 9) - #expect(try doubleBuilder.finish()[4] == 4.4) - let stringBuilder = try ArrowArrayBuilders.loadStringArrayBuilder() - stringBuilder.append(["0", "1", "2", "3", "4", "5", "6"]) - #expect(stringBuilder.length == 7) - #expect(try stringBuilder.finish()[4] == "4") - let boolBuilder = try ArrowArrayBuilders.loadBoolArrayBuilder() - boolBuilder.append([true, false, true, false]) - #expect(try boolBuilder.finish()[2] == true) - } - - @Test func listArrayPrimitive() throws { - let field = ArrowField(listFieldWith: .int32, isNullable: false) - let listBuilder = try ListArrayBuilder(.list(field)) - - listBuilder.append([Int32(1), Int32(2), Int32(3)]) - listBuilder.append([Int32(4), Int32(5)]) - listBuilder.append(nil) - listBuilder.append([Int32(6), Int32(7), Int32(8), Int32(9)]) - - #expect(listBuilder.length == 4) - #expect(listBuilder.nullCount == 1) - - let listArray = try listBuilder.finish() - #expect(listArray.length == 4) - - let firstList = listArray[0] - #expect(firstList != nil, "First list should not be nil") - #expect(firstList!.count == 3, "First list should have 3 elements") - #expect(firstList![0] as? Int32 == 1) - #expect(firstList![1] as? Int32 == 2) - #expect(firstList![2] as? Int32 == 3) - - let secondList = listArray[1] - #expect(secondList!.count == 2) - #expect(secondList![0] as? Int32 == 4) - #expect(secondList![1] as? Int32 == 5) - - #expect(listArray[2] == nil) - - let fourthList = listArray[3] - #expect(fourthList!.count == 4) - #expect(fourthList![0] as? Int32 == 6) - #expect(fourthList![3] as? Int32 == 9) - } - - @Test func listArrayNested() throws { - let field = ArrowField(listFieldWith: .int32, isNullable: false) - let innerListType: ArrowType = .list(field) - let outerField = ArrowField(listFieldWith: innerListType, isNullable: false) - let outerListBuilder = try ListArrayBuilder(.list(outerField)) - - guard - let innerListBuilder = outerListBuilder.valueBuilder as? ListArrayBuilder - else { - Issue.record("Failed to cast valueBuilder to ListArrayBuilder") - return - } - - outerListBuilder.bufferBuilder.append(2) - innerListBuilder.append([Int32(1), Int32(2)]) - innerListBuilder.append([Int32(3), Int32(4), Int32(5)]) - - outerListBuilder.bufferBuilder.append(1) - innerListBuilder.append([Int32(6)]) - - outerListBuilder.bufferBuilder.append(nil) - - outerListBuilder.bufferBuilder.append([]) - - let nestedArray = try outerListBuilder.finish() - #expect(nestedArray.length == 4) - #expect(nestedArray.nullCount == 1) - - let firstOuterList = nestedArray[0]! - #expect(firstOuterList.count == 2) - - let firstInnerList = firstOuterList[0] as! [Any?] - #expect(firstInnerList.count == 2) - #expect(firstInnerList[0] as? Int32 == 1) - #expect(firstInnerList[1] as? Int32 == 2) - - let secondInnerList = firstOuterList[1] as! [Any?] - #expect(secondInnerList.count == 3) - #expect(secondInnerList[0] as? Int32 == 3) - #expect(secondInnerList[1] as? Int32 == 4) - #expect(secondInnerList[2] as? Int32 == 5) - - let secondOuterList = nestedArray[1]! - #expect(secondOuterList.count == 1) - - let thirdInnerList = secondOuterList[0] as! [Any?] - #expect(thirdInnerList.count == 1) - #expect(thirdInnerList[0] as? Int32 == 6) - - #expect(nestedArray[2] == nil) - - let emptyList = nestedArray[3]! - #expect(emptyList.count == 0) - } -} diff --git a/Tests/ArrowTests/Array/BasicArrayTests.swift b/Tests/ArrowTests/BasicArrayTests.swift similarity index 94% rename from Tests/ArrowTests/Array/BasicArrayTests.swift rename to Tests/ArrowTests/BasicArrayTests.swift index 76b9c81..66eecbc 100644 --- a/Tests/ArrowTests/Array/BasicArrayTests.swift +++ b/Tests/ArrowTests/BasicArrayTests.swift @@ -233,4 +233,23 @@ struct BasicArrayTests { #expect(nsArray[1] == 1_609_459_200_000_000_000) #expect(nsArray[2] == 1_609_545_600_000_000_000) } + + @Test func inlineStrings() throws { + let builder = ArrayBuilderBinaryView() + + builder.append("hello") + builder.append("world") + builder.append("short") + + let array: ArrowArrayBinaryView = builder.finish() + + #expect(array.length == 3) + #expect(array[0] == "hello") + #expect(array[1] == "world") + #expect(array[2] == "short") + + // All inline, so no data buffers + print(array.buffers.count) // prints 2 + #expect(array.buffers.count == 2) + } } diff --git a/Tests/ArrowTests/BinaryViewTests.swift b/Tests/ArrowTests/BinaryViewTests.swift new file mode 100644 index 0000000..0896aa0 --- /dev/null +++ b/Tests/ArrowTests/BinaryViewTests.swift @@ -0,0 +1,132 @@ +// BinaryViewTests.swift +// Arrow +// +// Created by Will Temperley on 04/01/2026. All rights reserved. +// Copyright 2026 Will Temperley. +// +// Copying or reproduction of this file via any medium requires prior express +// written permission from the copyright holder. +// ----------------------------------------------------------------------------- +/// +/// Implementation notes, links and internal documentation go here. +/// +// ----------------------------------------------------------------------------- + +import Testing + +@testable import Arrow + +@Suite("BinaryView Tests") +struct BinaryViewTests { + + @Test("Inline strings stay inline") + func inlineStrings() throws { + let builder = ArrayBuilderBinaryView(maxBufferSize: 2 * 1024 * 1024) + + // All should be inline (≤12 bytes) + builder.append("hello") // 5 bytes + builder.append("world") // 5 bytes + builder.append("short") // 5 bytes + + let array: ArrowArrayBinaryView = builder.finish() + + #expect(array.length == 3) + #expect(array[0] == "hello") + #expect(array[1] == "world") + #expect(array[2] == "short") + + // Should have no data buffers (all inline) + #expect(array.dataBuffers.count == 0) + #expect(array.buffers.count == 2) // null + views only + } + + @Test("Referenced strings use data buffers") + func referencedStrings() throws { + let builder = ArrayBuilderBinaryView(maxBufferSize: 2 * 1024 * 1024) + + // Should be referenced (>12 bytes) + builder.append("this is a longer string") + builder.append("another long string here") + + let array = builder.finish() + + #expect(array.length == 2) + #expect(array[0] == "this is a longer string") + #expect(array[1] == "another long string here") + + // Should have 1 data buffer + #expect(array.dataBuffers.count == 1) + #expect(array.buffers.count == 3) // null + views + 1 data buffer + } + + @Test("Mixed inline and referenced strings") + func mixedInlineAndReferenced() throws { + let builder = ArrayBuilderBinaryView(maxBufferSize: 2 * 1024 * 1024) + + builder.append("short") // inline (5 bytes) + builder.append("this is much longer") // referenced (19 bytes) + builder.append("tiny") // inline (4 bytes) + builder.appendNull() + builder.append("123456789012") // inline (exactly 12 bytes) + builder.append("1234567890123") // referenced (13 bytes) + + let array = builder.finish() + + #expect(array.length == 6) + #expect(array[0] == "short") + #expect(array[1] == "this is much longer") + #expect(array[2] == "tiny") + #expect(array[3] == nil) + #expect(array[4] == "123456789012") + #expect(array[5] == "1234567890123") + #expect(array.nullCount == 1) + #expect(array.dataBuffers.count == 1) + } + + @Test("Multiple data buffers with small buffer size") + func multipleDataBuffers() throws { + // Small buffer to force rotation + let builder = ArrayBuilderBinaryView(maxBufferSize: 100) + + // Each string is >12 bytes, so all referenced + builder.append("first long string value") + builder.append("second long string value") + builder.append("third long string value") + builder.append("fourth long string value") + builder.append("fifth long string value") + + let array = builder.finish() + + #expect(array.length == 5) + #expect(array[0] == "first long string value") + #expect(array[1] == "second long string value") + #expect(array[2] == "third long string value") + #expect(array[3] == "fourth long string value") + #expect(array[4] == "fifth long string value") + + // With 100 byte limit, should have multiple buffers + #expect(array.dataBuffers.count > 1) + } + + @Test("Zero-copy slicing shares buffers") + func zeroCopySlicing() throws { + let builder = ArrayBuilderBinaryView(maxBufferSize: 2 * 1024 * 1024) + + builder.append("a") + builder.append("b") + builder.append("c") + builder.append("d") + builder.append("e") + + let array = builder.finish() + let slice = array.slice(offset: 1, length: 3) + + #expect(slice.length == 3) + #expect(slice[0] == "b") + #expect(slice[1] == "c") + #expect(slice[2] == "d") + + // Slicing should share buffers + #expect(array.buffers.count == slice.buffers.count) + } +} diff --git a/Tests/ArrowTests/BufferTests.swift b/Tests/ArrowTests/BufferTests.swift index 3cd126d..60ac3a4 100644 --- a/Tests/ArrowTests/BufferTests.swift +++ b/Tests/ArrowTests/BufferTests.swift @@ -54,13 +54,10 @@ struct BufferTests { } @Test func fixedWidthBufferBuilder() { - let builder = FixedWidthBufferBuilder() - for i in 0..<10_000 { builder.append(Int64(i)) } - let buffer = builder.finish() #expect(buffer.length == 10_000 * MemoryLayout.stride) } @@ -76,28 +73,4 @@ struct BufferTests { #expect(array[i] == Int32(i)) } } - - @Test func offsetsBuffer() { - - let offsets: [UInt32] = [0, 4, 8, 12, 16, 20] - let data = offsets.withUnsafeBufferPointer { buffer in - Data( - buffer: UnsafeBufferPointer( - start: buffer.baseAddress, - count: buffer.count) - ) - } - #expect(data.count == 24) - let range = 0.. Arrow.ArrowSchema { - let schemaBuilder = ArrowSchema.Builder() - return - schemaBuilder - .addField("colBool", type: .boolean, isNullable: false) - .addField("colUInt8", type: .uint8, isNullable: true) - .addField("colUInt16", type: .uint16, isNullable: true) - .addField("colUInt32", type: .uint32, isNullable: true) - .addField("colUInt64", type: .uint64, isNullable: true) - .addField("colInt8", type: .int8, isNullable: false) - .addField("colInt16", type: .int16, isNullable: false) - .addField("colInt32", type: .int32, isNullable: false) - .addField("colInt64", type: .int64, isNullable: false) - .addField("colString", type: .utf8, isNullable: false) - .addField("colBinary", type: .binary, isNullable: false) - .addField("colDate32", type: .date32, isNullable: false) - .addField("colDate64", type: .date64, isNullable: false) - // .addField("colTime32", type: .time32, isNullable: false) - .addField("colTime32s", type: .time32(.second), isNullable: false) - .addField("colTime32m", type: .time32(.millisecond), isNullable: false) - // .addField("colTime64", type: .time64, isNullable: false) - .addField("colTime64u", type: .time64(.microsecond), isNullable: false) - .addField("colTime64n", type: .time64(.nanosecond), isNullable: false) - // .addField("colTimestamp", type: ArrowType(ArrowType.ArrowTimestamp), isNullable: false) - .addField( - "colTimestampts", type: .timestamp(.second, nil), isNullable: false - ) - .addField( - "colTimestamptm", type: .timestamp(.millisecond, nil), isNullable: false - ) - .addField( - "colTimestamptu", type: .timestamp(.microsecond, nil), isNullable: false - ) - .addField( - "colTimestamptn", type: .timestamp(.nanosecond, nil), isNullable: false - ) - .addField("colFloat", type: .float32, isNullable: false) - .addField("colDouble", type: .float64, isNullable: false) - .finish() - } - - func checkImportField( - _ cSchema: ArrowC.ArrowSchema, name: String, type: ArrowType - ) throws { - let importer = ArrowCImporter() - switch importer.importField(cSchema) { - case .success(let arrowField): - #expect(arrowField.type == type) - #expect(arrowField.name == name) - case .failure(let error): - throw error - } - } - - @Test - @MainActor - func testImportExportSchema() throws { - let schema = makeSchema() - let exporter = ArrowCExporter() - for arrowField in schema.fields { - var cSchema = ArrowC.ArrowSchema() - switch exporter.exportField(&cSchema, field: arrowField) { - case .success: - try checkImportField( - cSchema, name: arrowField.name, type: arrowField.type) - case .failure(let error): - throw error - } - } - } - - @Test - @MainActor - func testImportExportArray() throws { - // Issue.record("Fix the deallocation. Test it too if possible.") - let stringBuilder = try ArrowArrayBuilders.loadStringArrayBuilder() - for index in 0..<100 { - if index % 10 == 9 { - stringBuilder.append(nil) - } else { - stringBuilder.append("test" + String(index)) - } - } - - #expect(stringBuilder.nullCount == 10) - #expect(stringBuilder.length == 100) - // XCTAssertEqual(stringBuilder.capacity, 640) - let stringArray = try stringBuilder.finish() - let exporter = ArrowCExporter() - var cArray = ArrowC.ArrowArray() - exporter.exportArray(&cArray, arrowData: stringArray.arrowData) - let cArrayMutPtr = UnsafeMutablePointer.allocate( - capacity: 1) - cArrayMutPtr.pointee = cArray - defer { - cArrayMutPtr.deallocate() - } - - let importer = ArrowCImporter() - switch importer.importArray(UnsafePointer(cArrayMutPtr), arrowType: .utf8) { - case .success(let holder): - let builder = RecordBatchX.Builder() - switch builder - .addColumn("test", arrowArray: holder) - .finish() - { - case .success(let rb): - #expect(rb.columnCount == 1) - #expect(rb.length == 100) - let col1: any Arrow.ArrowArray = try rb.data(for: 0) - for index in 0..] = [] + var offset = 0 + var remaining = totalLength + for _ in 0..() + for i in 0.. 0 }) + // Fuzz random accesses + for _ in 0..<5000 { + // TODO: consider an API which doesn't crash on invalid indexes + // let index = Int.random(in: -10..] = [] + for chunkIdx in 0..<10 { + let builder = ArrayBuilderFixedWidth() + for i in 0.. 0 { + #expect(chunkedArray[boundaryIdx - 1] == flatArray[boundaryIdx - 1]) + } + // At boundary + #expect(chunkedArray[boundaryIdx] == flatArray[boundaryIdx]) + // After boundary + if boundaryIdx + 1 < flatArray.count { + #expect(chunkedArray[boundaryIdx + 1] == flatArray[boundaryIdx + 1]) + } + } + } + } +} diff --git a/Tests/ArrowTests/CodableTests.swift b/Tests/ArrowTests/CodableTests.swift index 7397c30..341ec5a 100644 --- a/Tests/ArrowTests/CodableTests.swift +++ b/Tests/ArrowTests/CodableTests.swift @@ -17,369 +17,369 @@ import Testing @testable import Arrow -struct CodableTests { - - public class TestClass: Codable { - public var propBool: Bool - public var propInt8: Int8 - public var propInt16: Int16 - public var propInt32: Int32 - public var propInt64: Int64 - public var propUInt8: UInt8 - public var propUInt16: UInt16 - public var propUInt32: UInt32 - public var propUInt64: UInt64 - public var propFloat: Float - public var propDouble: Double? - public var propString: String - public var propDate: Date - - public required init() { - self.propBool = false - self.propInt8 = 1 - self.propInt16 = 2 - self.propInt32 = 3 - self.propInt64 = 4 - self.propUInt8 = 5 - self.propUInt16 = 6 - self.propUInt32 = 7 - self.propUInt64 = 8 - self.propFloat = 9 - self.propDouble = 10 - self.propString = "11" - self.propDate = Date.now - } - } - - @Test func arrowKeyedDecoder() throws { - let date1 = Date(timeIntervalSinceReferenceDate: 86400 * 5000 + 352) - let boolBuilder = try ArrowArrayBuilders.loadBoolArrayBuilder() - let int8Builder: NumberArrayBuilder = - try ArrowArrayBuilders.loadNumberArrayBuilder() - let int16Builder: NumberArrayBuilder = - try ArrowArrayBuilders.loadNumberArrayBuilder() - let int32Builder: NumberArrayBuilder = - try ArrowArrayBuilders.loadNumberArrayBuilder() - let int64Builder: NumberArrayBuilder = - try ArrowArrayBuilders.loadNumberArrayBuilder() - let uint8Builder: NumberArrayBuilder = - try ArrowArrayBuilders.loadNumberArrayBuilder() - let uint16Builder: NumberArrayBuilder = - try ArrowArrayBuilders.loadNumberArrayBuilder() - let uint32Builder: NumberArrayBuilder = - try ArrowArrayBuilders.loadNumberArrayBuilder() - let uint64Builder: NumberArrayBuilder = - try ArrowArrayBuilders.loadNumberArrayBuilder() - let floatBuilder: NumberArrayBuilder = - try ArrowArrayBuilders.loadNumberArrayBuilder() - let doubleBuilder: NumberArrayBuilder = - try ArrowArrayBuilders.loadNumberArrayBuilder() - let stringBuilder = try ArrowArrayBuilders.loadStringArrayBuilder() - let dateBuilder = try ArrowArrayBuilders.loadDate64ArrayBuilder() - - boolBuilder.append(false, true, false) - int8Builder.append(10, 11, 12) - int16Builder.append(20, 21, 22) - int32Builder.append(30, 31, 32) - int64Builder.append(40, 41, 42) - uint8Builder.append(50, 51, 52) - uint16Builder.append(60, 61, 62) - uint32Builder.append(70, 71, 72) - uint64Builder.append(80, 81, 82) - floatBuilder.append(90.1, 91.1, 92.1) - doubleBuilder.append(101.1, nil, nil) - stringBuilder.append("test0", "test1", "test2") - dateBuilder.append(date1, date1, date1) - let result = RecordBatchX.Builder() - .addColumn("propBool", arrowArray: try boolBuilder.finish()) - .addColumn("propInt8", arrowArray: try int8Builder.finish()) - .addColumn("propInt16", arrowArray: try int16Builder.finish()) - .addColumn("propInt32", arrowArray: try int32Builder.finish()) - .addColumn("propInt64", arrowArray: try int64Builder.finish()) - .addColumn("propUInt8", arrowArray: try uint8Builder.finish()) - .addColumn("propUInt16", arrowArray: try uint16Builder.finish()) - .addColumn("propUInt32", arrowArray: try uint32Builder.finish()) - .addColumn("propUInt64", arrowArray: try uint64Builder.finish()) - .addColumn("propFloat", arrowArray: try floatBuilder.finish()) - .addColumn("propDouble", arrowArray: try doubleBuilder.finish()) - .addColumn("propString", arrowArray: try stringBuilder.finish()) - .addColumn("propDate", arrowArray: try dateBuilder.finish()) - .finish() - switch result { - case .success(let rb): - let decoder = ArrowDecoder(rb) - let testClasses = try decoder.decode(TestClass.self) - for index in 0.. = - try ArrowArrayBuilders.loadNumberArrayBuilder() - int8Builder.append(10, 11, 12) - let result = RecordBatchX.Builder() - .addColumn("propInt8", arrowArray: try int8Builder.finish()) - .finish() - switch result { - case .success(let rb): - let decoder = ArrowDecoder(rb) - let testData = try decoder.decode(Int8?.self) - for index in 0.. = - try ArrowArrayBuilders.loadNumberArrayBuilder() - int8WNilBuilder.append(10, nil, 12, nil) - let resultWNil = RecordBatchX.Builder() - .addColumn( - "propInt8", - arrowArray: try int8WNilBuilder.finish() - ) - .finish() - switch resultWNil { - case .success(let rb): - let decoder = ArrowDecoder(rb) - let testData = try decoder.decode(Int8?.self) - for index in 0.. = - try ArrowArrayBuilders.loadNumberArrayBuilder() - let stringBuilder = try ArrowArrayBuilders.loadStringArrayBuilder() - int8Builder.append(10, 11, 12, 13) - stringBuilder.append("test10", "test11", "test12", "test13") - switch RecordBatchX.Builder() - .addColumn("propInt8", arrowArray: try int8Builder.finish()) - .addColumn("propString", arrowArray: try stringBuilder.finish()) - .finish() - { - case .success(let rb): - let decoder = ArrowDecoder(rb) - let testData = try decoder.decode([Int8: String].self) - for data in testData { - #expect("test\(data.key)" == data.value) - } - case .failure(let err): - throw err - } - - switch RecordBatchX.Builder() - .addColumn("propString", arrowArray: try stringBuilder.finish()) - .addColumn("propInt8", arrowArray: try int8Builder.finish()) - .finish() - { - case .success(let rb): - let decoder = ArrowDecoder(rb) - let testData = try decoder.decode([String: Int8].self) - for data in testData { - #expect("test\(data.value)" == data.key) - } - case .failure(let err): - throw err - } - } - - @Test func arrowMapDecoderWithNull() throws { - let int8Builder: NumberArrayBuilder = - try ArrowArrayBuilders.loadNumberArrayBuilder() - let stringWNilBuilder = try ArrowArrayBuilders.loadStringArrayBuilder() - int8Builder.append(10, 11, 12, 13) - stringWNilBuilder.append(nil, "test11", nil, "test13") - let resultWNil = RecordBatchX.Builder() - .addColumn("propInt8", arrowArray: try int8Builder.finish()) - .addColumn("propString", arrowArray: try stringWNilBuilder.finish()) - .finish() - switch resultWNil { - case .success(let rb): - let decoder = ArrowDecoder(rb) - let testData = try decoder.decode([Int8: String?].self) - for data in testData { - let str = data.value - if data.key % 2 == 0 { - #expect(str == nil) - } else { - #expect(str == "test\(data.key)") - } - } - case .failure(let err): - throw err - } - } - - func getArrayValue(_ rb: RecordBatchX, colIndex: Int, rowIndex: UInt) -> T? - { - let anyArray = rb.columns[colIndex] - return anyArray.asAny(UInt(rowIndex)) as? T - } - - @Test func arrowKeyedEncoder() throws { - var infos: [TestClass] = [] - for index in 0..<10 { - let tClass = TestClass() - let offset = index * 12 - tClass.propBool = index % 2 == 0 - tClass.propInt8 = Int8(offset + 1) - tClass.propInt16 = Int16(offset + 2) - tClass.propInt32 = Int32(offset + 3) - tClass.propInt64 = Int64(offset + 4) - tClass.propUInt8 = UInt8(offset + 5) - tClass.propUInt16 = UInt16(offset + 6) - tClass.propUInt32 = UInt32(offset + 7) - tClass.propUInt64 = UInt64(offset + 8) - tClass.propFloat = Float(offset + 9) - tClass.propDouble = index % 2 == 0 ? Double(offset + 10) : nil - tClass.propString = "\(offset + 11)" - tClass.propDate = Date.now - infos.append(tClass) - } - - let rb = try ArrowEncoder.encode(infos)! - #expect(Int(rb.length) == infos.count) - #expect(rb.columns.count == 13) - #expect(rb.columns[0].type == .boolean) - #expect(rb.columns[1].type == .int8) - #expect(rb.columns[2].type == .int16) - #expect(rb.columns[3].type == .int32) - #expect(rb.columns[4].type == .int64) - #expect(rb.columns[5].type == .uint8) - #expect(rb.columns[6].type == .uint16) - #expect(rb.columns[7].type == .uint32) - #expect(rb.columns[8].type == .uint64) - #expect(rb.columns[9].type == .float32) - #expect(rb.columns[10].type == .float64) - #expect(rb.columns[11].type == .utf8) - #expect(rb.columns[12].type == .date64) - for index in 0..<10 { - let offset = index * 12 - #expect( - getArrayValue(rb, colIndex: 0, rowIndex: UInt(index)) - == (index % 2 == 0)) - #expect( - getArrayValue(rb, colIndex: 1, rowIndex: UInt(index)) - == Int8(offset + 1)) - #expect( - getArrayValue(rb, colIndex: 2, rowIndex: UInt(index)) - == Int16(offset + 2) - ) - #expect( - getArrayValue(rb, colIndex: 3, rowIndex: UInt(index)) - == Int32(offset + 3) - ) - #expect( - getArrayValue(rb, colIndex: 4, rowIndex: UInt(index)) - == Int64(offset + 4) - ) - #expect( - getArrayValue(rb, colIndex: 5, rowIndex: UInt(index)) - == UInt8(offset + 5) - ) - #expect( - getArrayValue(rb, colIndex: 6, rowIndex: UInt(index)) - == UInt16(offset + 6)) - #expect( - getArrayValue(rb, colIndex: 7, rowIndex: UInt(index)) - == UInt32(offset + 7)) - #expect( - getArrayValue(rb, colIndex: 8, rowIndex: UInt(index)) - == UInt64(offset + 8)) - #expect( - getArrayValue(rb, colIndex: 9, rowIndex: UInt(index)) - == Float(offset + 9) - ) - if index % 2 == 0 { - #expect( - getArrayValue(rb, colIndex: 10, rowIndex: UInt(index)) - == Double(offset + 10)) - } else { - #expect( - getArrayValue(rb, colIndex: 10, rowIndex: UInt(index)) == Double?(nil) - ) - } - #expect( - getArrayValue(rb, colIndex: 11, rowIndex: UInt(index)) - == String(offset + 11)) - } - } - - @Test func arrowUnkeyedEncoder() throws { - var testMap: [Int8: String?] = [:] - for index in 0..<10 { - testMap[Int8(index)] = "test\(index)" - } - - let rb = try ArrowEncoder.encode(testMap) - #expect(Int(rb.length) == testMap.count) - #expect(rb.columns.count == 2) - #expect(rb.columns[0].type == .int8) - #expect(rb.columns[1].type == .utf8) - for index in 0..<10 { - let key: Int8 = getArrayValue(rb, colIndex: 0, rowIndex: UInt(index))! - let value: String = getArrayValue(rb, colIndex: 1, rowIndex: UInt(index))! - #expect("test\(key)" == value) - } - } - - @Test func arrowSingleEncoder() throws { - var intArray: [Int32?] = [] - for index in 0..<100 { - if index == 10 { - intArray.append(nil) - } else { - intArray.append(Int32(index)) - } - } - - let rb = try ArrowEncoder.encode(intArray)! - #expect(Int(rb.length) == intArray.count) - #expect(rb.columns.count == 1) - #expect(rb.columns[0].type == .int32) - for index in 0..<100 { - if index == 10 { - let anyArray = rb.columns[0] - #expect(anyArray.asAny(UInt(index)) == nil) - } else { - #expect( - getArrayValue(rb, colIndex: 0, rowIndex: UInt(index)) == Int32(index)) - } - } - } -} +//struct CodableTests { +// +// public class TestClass: Codable { +// public var propBool: Bool +// public var propInt8: Int8 +// public var propInt16: Int16 +// public var propInt32: Int32 +// public var propInt64: Int64 +// public var propUInt8: UInt8 +// public var propUInt16: UInt16 +// public var propUInt32: UInt32 +// public var propUInt64: UInt64 +// public var propFloat: Float +// public var propDouble: Double? +// public var propString: String +// public var propDate: Date +// +// public required init() { +// self.propBool = false +// self.propInt8 = 1 +// self.propInt16 = 2 +// self.propInt32 = 3 +// self.propInt64 = 4 +// self.propUInt8 = 5 +// self.propUInt16 = 6 +// self.propUInt32 = 7 +// self.propUInt64 = 8 +// self.propFloat = 9 +// self.propDouble = 10 +// self.propString = "11" +// self.propDate = Date.now +// } +// } +// +// @Test func arrowKeyedDecoder() throws { +// let date1 = Date(timeIntervalSinceReferenceDate: 86400 * 5000 + 352) +// let boolBuilder = try ArrowArrayBuilders.loadBoolArrayBuilder() +// let int8Builder: NumberArrayBuilder = +// try ArrowArrayBuilders.loadNumberArrayBuilder() +// let int16Builder: NumberArrayBuilder = +// try ArrowArrayBuilders.loadNumberArrayBuilder() +// let int32Builder: NumberArrayBuilder = +// try ArrowArrayBuilders.loadNumberArrayBuilder() +// let int64Builder: NumberArrayBuilder = +// try ArrowArrayBuilders.loadNumberArrayBuilder() +// let uint8Builder: NumberArrayBuilder = +// try ArrowArrayBuilders.loadNumberArrayBuilder() +// let uint16Builder: NumberArrayBuilder = +// try ArrowArrayBuilders.loadNumberArrayBuilder() +// let uint32Builder: NumberArrayBuilder = +// try ArrowArrayBuilders.loadNumberArrayBuilder() +// let uint64Builder: NumberArrayBuilder = +// try ArrowArrayBuilders.loadNumberArrayBuilder() +// let floatBuilder: NumberArrayBuilder = +// try ArrowArrayBuilders.loadNumberArrayBuilder() +// let doubleBuilder: NumberArrayBuilder = +// try ArrowArrayBuilders.loadNumberArrayBuilder() +// let stringBuilder = try ArrowArrayBuilders.loadStringArrayBuilder() +// let dateBuilder = try ArrowArrayBuilders.loadDate64ArrayBuilder() +// +// boolBuilder.append(false, true, false) +// int8Builder.append(10, 11, 12) +// int16Builder.append(20, 21, 22) +// int32Builder.append(30, 31, 32) +// int64Builder.append(40, 41, 42) +// uint8Builder.append(50, 51, 52) +// uint16Builder.append(60, 61, 62) +// uint32Builder.append(70, 71, 72) +// uint64Builder.append(80, 81, 82) +// floatBuilder.append(90.1, 91.1, 92.1) +// doubleBuilder.append(101.1, nil, nil) +// stringBuilder.append("test0", "test1", "test2") +// dateBuilder.append(date1, date1, date1) +// let result = RecordBatchX.Builder() +// .addColumn("propBool", arrowArray: try boolBuilder.finish()) +// .addColumn("propInt8", arrowArray: try int8Builder.finish()) +// .addColumn("propInt16", arrowArray: try int16Builder.finish()) +// .addColumn("propInt32", arrowArray: try int32Builder.finish()) +// .addColumn("propInt64", arrowArray: try int64Builder.finish()) +// .addColumn("propUInt8", arrowArray: try uint8Builder.finish()) +// .addColumn("propUInt16", arrowArray: try uint16Builder.finish()) +// .addColumn("propUInt32", arrowArray: try uint32Builder.finish()) +// .addColumn("propUInt64", arrowArray: try uint64Builder.finish()) +// .addColumn("propFloat", arrowArray: try floatBuilder.finish()) +// .addColumn("propDouble", arrowArray: try doubleBuilder.finish()) +// .addColumn("propString", arrowArray: try stringBuilder.finish()) +// .addColumn("propDate", arrowArray: try dateBuilder.finish()) +// .finish() +// switch result { +// case .success(let rb): +// let decoder = ArrowDecoder(rb) +// let testClasses = try decoder.decode(TestClass.self) +// for index in 0.. = +// try ArrowArrayBuilders.loadNumberArrayBuilder() +// int8Builder.append(10, 11, 12) +// let result = RecordBatchX.Builder() +// .addColumn("propInt8", arrowArray: try int8Builder.finish()) +// .finish() +// switch result { +// case .success(let rb): +// let decoder = ArrowDecoder(rb) +// let testData = try decoder.decode(Int8?.self) +// for index in 0.. = +// try ArrowArrayBuilders.loadNumberArrayBuilder() +// int8WNilBuilder.append(10, nil, 12, nil) +// let resultWNil = RecordBatchX.Builder() +// .addColumn( +// "propInt8", +// arrowArray: try int8WNilBuilder.finish() +// ) +// .finish() +// switch resultWNil { +// case .success(let rb): +// let decoder = ArrowDecoder(rb) +// let testData = try decoder.decode(Int8?.self) +// for index in 0.. = +// try ArrowArrayBuilders.loadNumberArrayBuilder() +// let stringBuilder = try ArrowArrayBuilders.loadStringArrayBuilder() +// int8Builder.append(10, 11, 12, 13) +// stringBuilder.append("test10", "test11", "test12", "test13") +// switch RecordBatchX.Builder() +// .addColumn("propInt8", arrowArray: try int8Builder.finish()) +// .addColumn("propString", arrowArray: try stringBuilder.finish()) +// .finish() +// { +// case .success(let rb): +// let decoder = ArrowDecoder(rb) +// let testData = try decoder.decode([Int8: String].self) +// for data in testData { +// #expect("test\(data.key)" == data.value) +// } +// case .failure(let err): +// throw err +// } +// +// switch RecordBatchX.Builder() +// .addColumn("propString", arrowArray: try stringBuilder.finish()) +// .addColumn("propInt8", arrowArray: try int8Builder.finish()) +// .finish() +// { +// case .success(let rb): +// let decoder = ArrowDecoder(rb) +// let testData = try decoder.decode([String: Int8].self) +// for data in testData { +// #expect("test\(data.value)" == data.key) +// } +// case .failure(let err): +// throw err +// } +// } +// +// @Test func arrowMapDecoderWithNull() throws { +// let int8Builder: NumberArrayBuilder = +// try ArrowArrayBuilders.loadNumberArrayBuilder() +// let stringWNilBuilder = try ArrowArrayBuilders.loadStringArrayBuilder() +// int8Builder.append(10, 11, 12, 13) +// stringWNilBuilder.append(nil, "test11", nil, "test13") +// let resultWNil = RecordBatchX.Builder() +// .addColumn("propInt8", arrowArray: try int8Builder.finish()) +// .addColumn("propString", arrowArray: try stringWNilBuilder.finish()) +// .finish() +// switch resultWNil { +// case .success(let rb): +// let decoder = ArrowDecoder(rb) +// let testData = try decoder.decode([Int8: String?].self) +// for data in testData { +// let str = data.value +// if data.key % 2 == 0 { +// #expect(str == nil) +// } else { +// #expect(str == "test\(data.key)") +// } +// } +// case .failure(let err): +// throw err +// } +// } +// +// func getArrayValue(_ rb: RecordBatch, colIndex: Int, rowIndex: UInt) -> T? +// { +// let anyArray = rb.columns[colIndex] +// return anyArray.asAny(UInt(rowIndex)) as? T +// } +// +// @Test func arrowKeyedEncoder() throws { +// var infos: [TestClass] = [] +// for index in 0..<10 { +// let tClass = TestClass() +// let offset = index * 12 +// tClass.propBool = index % 2 == 0 +// tClass.propInt8 = Int8(offset + 1) +// tClass.propInt16 = Int16(offset + 2) +// tClass.propInt32 = Int32(offset + 3) +// tClass.propInt64 = Int64(offset + 4) +// tClass.propUInt8 = UInt8(offset + 5) +// tClass.propUInt16 = UInt16(offset + 6) +// tClass.propUInt32 = UInt32(offset + 7) +// tClass.propUInt64 = UInt64(offset + 8) +// tClass.propFloat = Float(offset + 9) +// tClass.propDouble = index % 2 == 0 ? Double(offset + 10) : nil +// tClass.propString = "\(offset + 11)" +// tClass.propDate = Date.now +// infos.append(tClass) +// } +// +// let rb = try ArrowEncoder.encode(infos)! +// #expect(Int(rb.length) == infos.count) +// #expect(rb.columns.count == 13) +// #expect(rb.columns[0].type == .boolean) +// #expect(rb.columns[1].type == .int8) +// #expect(rb.columns[2].type == .int16) +// #expect(rb.columns[3].type == .int32) +// #expect(rb.columns[4].type == .int64) +// #expect(rb.columns[5].type == .uint8) +// #expect(rb.columns[6].type == .uint16) +// #expect(rb.columns[7].type == .uint32) +// #expect(rb.columns[8].type == .uint64) +// #expect(rb.columns[9].type == .float32) +// #expect(rb.columns[10].type == .float64) +// #expect(rb.columns[11].type == .utf8) +// #expect(rb.columns[12].type == .date64) +// for index in 0..<10 { +// let offset = index * 12 +// #expect( +// getArrayValue(rb, colIndex: 0, rowIndex: UInt(index)) +// == (index % 2 == 0)) +// #expect( +// getArrayValue(rb, colIndex: 1, rowIndex: UInt(index)) +// == Int8(offset + 1)) +// #expect( +// getArrayValue(rb, colIndex: 2, rowIndex: UInt(index)) +// == Int16(offset + 2) +// ) +// #expect( +// getArrayValue(rb, colIndex: 3, rowIndex: UInt(index)) +// == Int32(offset + 3) +// ) +// #expect( +// getArrayValue(rb, colIndex: 4, rowIndex: UInt(index)) +// == Int64(offset + 4) +// ) +// #expect( +// getArrayValue(rb, colIndex: 5, rowIndex: UInt(index)) +// == UInt8(offset + 5) +// ) +// #expect( +// getArrayValue(rb, colIndex: 6, rowIndex: UInt(index)) +// == UInt16(offset + 6)) +// #expect( +// getArrayValue(rb, colIndex: 7, rowIndex: UInt(index)) +// == UInt32(offset + 7)) +// #expect( +// getArrayValue(rb, colIndex: 8, rowIndex: UInt(index)) +// == UInt64(offset + 8)) +// #expect( +// getArrayValue(rb, colIndex: 9, rowIndex: UInt(index)) +// == Float(offset + 9) +// ) +// if index % 2 == 0 { +// #expect( +// getArrayValue(rb, colIndex: 10, rowIndex: UInt(index)) +// == Double(offset + 10)) +// } else { +// #expect( +// getArrayValue(rb, colIndex: 10, rowIndex: UInt(index)) == Double?(nil) +// ) +// } +// #expect( +// getArrayValue(rb, colIndex: 11, rowIndex: UInt(index)) +// == String(offset + 11)) +// } +// } +// +// @Test func arrowUnkeyedEncoder() throws { +// var testMap: [Int8: String?] = [:] +// for index in 0..<10 { +// testMap[Int8(index)] = "test\(index)" +// } +// +// let rb = try ArrowEncoder.encode(testMap) +// #expect(Int(rb.length) == testMap.count) +// #expect(rb.columns.count == 2) +// #expect(rb.columns[0].type == .int8) +// #expect(rb.columns[1].type == .utf8) +// for index in 0..<10 { +// let key: Int8 = getArrayValue(rb, colIndex: 0, rowIndex: UInt(index))! +// let value: String = getArrayValue(rb, colIndex: 1, rowIndex: UInt(index))! +// #expect("test\(key)" == value) +// } +// } +// +// @Test func arrowSingleEncoder() throws { +// var intArray: [Int32?] = [] +// for index in 0..<100 { +// if index == 10 { +// intArray.append(nil) +// } else { +// intArray.append(Int32(index)) +// } +// } +// +// let rb = try ArrowEncoder.encode(intArray)! +// #expect(Int(rb.length) == intArray.count) +// #expect(rb.columns.count == 1) +// #expect(rb.columns[0].type == .int32) +// for index in 0..<100 { +// if index == 10 { +// let anyArray = rb.columns[0] +// #expect(anyArray.asAny(UInt(index)) == nil) +// } else { +// #expect( +// getArrayValue(rb, colIndex: 0, rowIndex: UInt(index)) == Int32(index)) +// } +// } +// } +//} diff --git a/Tests/ArrowTests/Array/FuzzedArrayTests.swift b/Tests/ArrowTests/FuzzedArrayTests.swift similarity index 100% rename from Tests/ArrowTests/Array/FuzzedArrayTests.swift rename to Tests/ArrowTests/FuzzedArrayTests.swift diff --git a/Tests/ArrowTests/IPCTests.swift b/Tests/ArrowTests/IPCTests.swift deleted file mode 100644 index 7faa41c..0000000 --- a/Tests/ArrowTests/IPCTests.swift +++ /dev/null @@ -1,673 +0,0 @@ -// Copyright 2025 The Apache Software Foundation -// -// 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. - -import FlatBuffers -import Foundation -import Testing - -@testable import Arrow - -let currentDate = Date.now - -struct StructTest { - var field0: Bool = false - var field1: Int8 = 0 - var field2: Int16 = 0 - var field: Int32 = 0 - var field4: Int64 = 0 - var field5: UInt8 = 0 - var field6: UInt16 = 0 - var field7: UInt32 = 0 - var field8: UInt64 = 0 - var field9: Double = 0 - var field10: Float = 0 - var field11: String = "" - var field12 = Data() - var field13: Date = currentDate -} - -func loadArrowResource(name: String) throws(ArrowError) -> URL { - if let resource = Bundle.module.url( - forResource: name, - withExtension: "arrow", - subdirectory: "Resources" - ) { - return resource - } else { - throw .runtimeError("Couldn't find \(name).arrow in the test resources.") - } -} - -@discardableResult -func checkBoolRecordBatch( - _ result: Result -) throws(ArrowError) -> [RecordBatchX] { - let recordBatches: [RecordBatchX] - switch result { - case .success(let result): - recordBatches = result.batches - case .failure(let error): - throw error - } - #expect(recordBatches.count == 1) - for recordBatch in recordBatches { - #expect(recordBatch.length == 5) - #expect(recordBatch.columns.count == 2) - #expect(recordBatch.schema.fields.count == 2) - #expect(recordBatch.schema.fields[0].name == "one") - #expect(recordBatch.schema.fields[0].type == .boolean) - #expect(recordBatch.schema.fields[1].name == "two") - #expect(recordBatch.schema.fields[1].type == .utf8) - for index in 0.. -) throws(ArrowError) -> [RecordBatchX] { - let recordBatches: [RecordBatchX] - switch result { - case .success(let result): - recordBatches = result.batches - case .failure(let error): - throw error - } - #expect(recordBatches.count == 1) - for recordBatch in recordBatches { - #expect(recordBatch.length == 3) - #expect(recordBatch.columns.count == 1) - #expect(recordBatch.schema.fields.count == 1) - #expect(recordBatch.schema.fields[0].name == "my struct") - guard case .strct(_) = recordBatch.schema.fields[0].type else { - Issue.record("Expected field 0 to be a struct") - return [] - } - guard let nestedArray = recordBatch.columns[0] as? NestedArray else { - throw .runtimeError("Could not cast to NestedArray") - } - guard let fields = nestedArray.fields else { - throw .runtimeError("NestedArray.fields is nil") - } - #expect(fields.count == 2) - #expect(fields[0].type == .utf8) - #expect(fields[1].type == .boolean) - let column = recordBatch.columns[0] - // guard let str = column else { - // throw .runtimeError("String array is nil") - // } - #expect("\(column.asString(0))" == "{0,false}") - #expect("\(column.asString(1))" == "{1,true}") - #expect(column.asAny(2) == nil) - } - return recordBatches -} - -func currentDirectory(path: String = #file) -> URL { - URL(fileURLWithPath: path).deletingLastPathComponent() -} - -func makeSchema() -> ArrowSchema { - let schemaBuilder = ArrowSchema.Builder() - return - schemaBuilder - .addField("col1", type: .int8, isNullable: true) - .addField("col2", type: .utf8, isNullable: false) - .addField("col3", type: .date32, isNullable: false) - .addField("col4", type: .int32, isNullable: false) - .addField("col5", type: .float32, isNullable: false) - .finish() -} - -func makeStructSchema() throws -> ArrowSchema { - let testObj = StructTest() - var fields: [ArrowField] = [] - let buildStructType = { () -> ArrowType in - let mirror = Mirror(reflecting: testObj) - for (property, value) in mirror.children { - guard let property else { - fatalError("Cannot get field name") - } - let arrowType = try ArrowTypeConverter.infoForType(type(of: value)) - fields.append( - ArrowField(name: property, dataType: arrowType, isNullable: true)) - } - return .strct(fields) - } - - return ArrowSchema.Builder() - .addField("struct1", type: try buildStructType(), isNullable: true) - .finish() -} - -func makeStructRecordBatch() throws -> RecordBatchX { - let testData = StructTest() - let dateNow = Date.now - let structBuilder = try ArrowArrayBuilders.structArrayBuilderForType( - testData - ) - structBuilder.append([ - true, Int8(1), Int16(2), Int32(3), Int64(4), - UInt8(5), UInt16(6), UInt32(7), UInt64(8), Double(9.9), - Float(10.10), "11", Data("12".utf8), dateNow, - ]) - structBuilder.append(nil) - structBuilder.append([ - true, Int8(13), Int16(14), Int32(15), Int64(16), - UInt8(17), UInt16(18), UInt32(19), UInt64(20), Double(21.21), - Float(22.22), "23", Data("24".utf8), dateNow, - ]) - let structArray = try structBuilder.finish() - let result = RecordBatchX.Builder() - .addColumn("struct1", arrowArray: structArray) - .finish() - switch result { - case .success(let recordBatch): - return recordBatch - case .failure(let error): - throw error - } -} - -func makeRecordBatch() throws -> RecordBatchX { - let uint8Builder: NumberArrayBuilder = - try ArrowArrayBuilders.loadNumberArrayBuilder() - uint8Builder.append(10) - uint8Builder.append(nil) - uint8Builder.append(nil) - uint8Builder.append(44) - let stringBuilder = try ArrowArrayBuilders.loadStringArrayBuilder() - stringBuilder.append("test10") - stringBuilder.append("test22") - stringBuilder.append("test33") - stringBuilder.append("test44") - let date32Builder = try ArrowArrayBuilders.loadDate32ArrayBuilder() - let date2 = Date(timeIntervalSinceReferenceDate: 86400 * 1) - let date1 = Date(timeIntervalSinceReferenceDate: 86400 * 5000 + 352) - date32Builder.append(date1) - date32Builder.append(date2) - date32Builder.append(date1) - date32Builder.append(date2) - let int32Builder: NumberArrayBuilder = - try ArrowArrayBuilders.loadNumberArrayBuilder() - int32Builder.append(1) - int32Builder.append(2) - int32Builder.append(3) - int32Builder.append(4) - let floatBuilder: NumberArrayBuilder = - try ArrowArrayBuilders.loadNumberArrayBuilder() - floatBuilder.append(211.112) - floatBuilder.append(322.223) - floatBuilder.append(433.334) - floatBuilder.append(544.445) - - let uint8Array = try uint8Builder.finish() - let stringArray = try stringBuilder.finish() - let date32Array = try date32Builder.finish() - let int32Array = try int32Builder.finish() - let floatArray = try floatBuilder.finish() - let result = RecordBatchX.Builder() - .addColumn("col1", arrowArray: uint8Array) - .addColumn("col2", arrowArray: stringArray) - .addColumn("col3", arrowArray: date32Array) - .addColumn("col4", arrowArray: int32Array) - .addColumn("col5", arrowArray: floatArray) - .finish() - switch result { - case .success(let recordBatch): - return recordBatch - case .failure(let error): - throw error - } -} - -struct IPCStreamReaderTests { - - @Test func recordBatchInMemoryToFromStream() throws { - let schema = makeSchema() - let recordBatch = try makeRecordBatch() - let arrowWriter = ArrowWriter() - let writerInfo = ArrowWriter.Info( - .recordbatch, schema: schema, batches: [recordBatch]) - switch arrowWriter.writeStreaming(writerInfo) { - case .success(let writeData): - let arrowReader = ArrowReaderX() - switch arrowReader.readStreaming(writeData) { - case .success(let result): - let recordBatches = result.batches - #expect(recordBatches.count == 1) - for recordBatch in recordBatches { - #expect(recordBatch.length == 4) - #expect(recordBatch.columns.count == 5) - #expect(recordBatch.schema.fields.count == 5) - #expect(recordBatch.schema.fields[0].name == "col1") - #expect(recordBatch.schema.fields[0].type == .int8) - #expect(recordBatch.schema.fields[1].name == "col2") - #expect(recordBatch.schema.fields[1].type == .utf8) - #expect(recordBatch.schema.fields[2].name == "col3") - #expect(recordBatch.schema.fields[2].type == .date32) - #expect(recordBatch.schema.fields[3].name == "col4") - #expect(recordBatch.schema.fields[3].type == .int32) - #expect(recordBatch.schema.fields[4].name == "col5") - #expect(recordBatch.schema.fields[4].type == .float32) - let columns = recordBatch.columns - #expect(columns[0].nullCount == 2) - let dateVal = "\((columns[2]).asString(0))" - #expect(dateVal == "2014-09-10 00:00:00 +0000") - let stringVal = "\((columns[1]).asString(1))" - #expect(stringVal == "test22") - let uintVal = "\((columns[0]).asString(0))" - #expect(uintVal == "10") - let stringVal2 = "\((columns[1]).asString(3))" - #expect(stringVal2 == "test44") - let uintVal2 = "\((columns[0]).asString(3))" - #expect(uintVal2 == "44") - } - case .failure(let error): - throw error - } - case .failure(let error): - throw error - } - } -} - -struct IPCFileReaderTests { - @Test func fileReader_double() throws { - let fileURL = try loadArrowResource(name: "testdata_double") - let arrowReader = ArrowReaderX() - let result = arrowReader.fromFile(fileURL) - let recordBatches: [RecordBatchX] - switch result { - case .success(let result): - recordBatches = result.batches - case .failure(let error): - throw error - } - - #expect(recordBatches.count == 1) - for recordBatch in recordBatches { - #expect(recordBatch.length == 5) - #expect(recordBatch.columns.count == 2) - #expect(recordBatch.schema.fields.count == 2) - #expect(recordBatch.schema.fields[0].name == "one") - #expect( - recordBatch.schema.fields[0].type == .float64) - #expect(recordBatch.schema.fields[1].name == "two") - #expect( - recordBatch.schema.fields[1].type == .utf8) - for index in 0.. (ArrowSchema, RecordBatchX) { - let schemaBuilder = ArrowSchema.Builder() - let schema = schemaBuilder.addField( - "binary", type: .binary, isNullable: false - ) - .finish() - - let binaryBuilder = try ArrowArrayBuilders.loadBinaryArrayBuilder() - binaryBuilder.append("test10".data(using: .utf8)) - binaryBuilder.append("test22".data(using: .utf8)) - binaryBuilder.append("test33".data(using: .utf8)) - binaryBuilder.append("test44".data(using: .utf8)) - - let binaryArray = try binaryBuilder.finish() - let result = RecordBatchX.Builder() - .addColumn("binary", arrowArray: binaryArray) - .finish() - switch result { - case .success(let recordBatch): - return (schema, recordBatch) - case .failure(let error): - throw error - } - } - - func makeTimeDataset() throws -> (ArrowSchema, RecordBatchX) { - let schemaBuilder = ArrowSchema.Builder() - let schema = schemaBuilder.addField( - "time64", type: .time64(.microsecond), isNullable: false - ) - .addField("time32", type: .time32(.millisecond), isNullable: false) - .finish() - - let time64Builder = try ArrowArrayBuilders.loadTime64ArrayBuilder( - .nanosecond - ) - time64Builder.append(12_345_678) - time64Builder.append(1) - time64Builder.append(nil) - time64Builder.append(98_765_432) - let time32Builder = try ArrowArrayBuilders.loadTime32ArrayBuilder( - .millisecond - ) - time32Builder.append(1) - time32Builder.append(2) - time32Builder.append(nil) - time32Builder.append(3) - let time64Array = try time64Builder.finish() - let time32Array = try time32Builder.finish() - let result = RecordBatchX.Builder() - .addColumn("time64", arrowArray: time64Array) - .addColumn("time32", arrowArray: time32Array) - .finish() - switch result { - case .success(let recordBatch): - return (schema, recordBatch) - case .failure(let error): - throw error - } - } - - @Test func structRecordBatchInMemoryToFromStream() throws { - // read existing file - let schema = try makeStructSchema() - let recordBatch = try makeStructRecordBatch() - let arrowWriter = ArrowWriter() - let writerInfo = ArrowWriter.Info( - .recordbatch, - schema: schema, - batches: [recordBatch] - ) - switch arrowWriter.writeStreaming(writerInfo) { - case .success(let writeData): - let arrowReader = ArrowReaderX() - switch arrowReader.readStreaming(writeData) { - case .success(let result): - let recordBatches = result.batches - #expect(recordBatches.count == 1) - for recordBatch in recordBatches { - #expect(recordBatch.length == 3) - #expect(recordBatch.columns.count == 1) - #expect(recordBatch.schema.fields.count == 1) - #expect(recordBatch.schema.fields[0].name == "struct1") - guard case .strct(let fields) = recordBatch.schema.fields[0].type - else { - Issue.record("Expected Struct") - return - } - #expect(fields.count == 14) - let columns = recordBatch.columns - #expect(columns[0].nullCount == 1) - #expect(columns[0].asAny(1) == nil) - let structVal = "\(columns[0].asString(0))" - #expect( - structVal == "{true,1,2,3,4,5,6,7,8,9.9,10.1,11,12,\(currentDate)}") - let nestedArray = (recordBatch.columns[0] as? NestedArray)! - #expect(nestedArray.length == 3) - #expect(nestedArray.fields != nil) - #expect(nestedArray.fields!.count == 14) - #expect(nestedArray.fields![0].type == .boolean) - #expect(nestedArray.fields![1].type == .int8) - #expect(nestedArray.fields![2].type == .int16) - #expect(nestedArray.fields![3].type == .int32) - #expect(nestedArray.fields![4].type == .int64) - #expect(nestedArray.fields![5].type == .uint8) - #expect(nestedArray.fields![6].type == .uint16) - #expect(nestedArray.fields![7].type == .uint32) - #expect(nestedArray.fields![8].type == .uint64) - #expect(nestedArray.fields![9].type == .float64) - #expect(nestedArray.fields![10].type == .float32) - #expect(nestedArray.fields![11].type == .utf8) - #expect(nestedArray.fields![12].type == .binary) - #expect(nestedArray.fields![13].type == .date64) - } - case .failure(let error): - throw error - } - case .failure(let error): - throw error - } - } - - @Test func binaryInMemoryToFromStream() throws { - let dataset = try makeBinaryDataset() - let writerInfo = ArrowWriter.Info( - .recordbatch, - schema: dataset.0, - batches: [dataset.1] - ) - let arrowWriter = ArrowWriter() - switch arrowWriter.writeFile(writerInfo) { - case .success(let writeData): - let arrowReader = ArrowReaderX() - switch arrowReader.readFile(writeData) { - case .success(let result): - #expect(result.schema != nil) - let schema = result.schema! - #expect(schema.fields.count == 1) - #expect(schema.fields[0].name == "binary") - #expect(schema.fields[0].type == .binary) - #expect(result.batches.count == 1) - let recordBatch = result.batches[0] - #expect(recordBatch.length == 4) - let columns = recordBatch.columns - let stringVal = - "\(columns[0].asString(1))" - #expect(stringVal == "test22") - case .failure(let error): - throw error - } - case .failure(let error): - throw error - } - } - - @Test func timeInMemoryToFromStream() throws { - let dataset = try makeTimeDataset() - let writerInfo = ArrowWriter.Info( - .recordbatch, - schema: dataset.0, - batches: [dataset.1] - ) - let arrowWriter = ArrowWriter() - switch arrowWriter.writeFile(writerInfo) { - case .success(let writeData): - let arrowReader = ArrowReaderX() - switch arrowReader.readFile(writeData) { - case .success(let result): - #expect(result.schema != nil) - let schema = result.schema! - #expect(schema.fields.count == 2) - #expect(schema.fields[0].name == "time64") - #expect(schema.fields[0].type == .time64(.microsecond)) - #expect(schema.fields[1].name == "time32") - #expect(schema.fields[1].type == .time32(.millisecond)) - #expect(result.batches.count == 1) - let recordBatch = result.batches[0] - #expect(recordBatch.length == 4) - let columns = recordBatch.columns - let stringVal = "\(columns[0].asString(0))" - #expect(stringVal == "12345678") - let stringVal2 = - "\(columns[1].asString(3))" - #expect(stringVal2 == "3") - case .failure(let error): - throw error - } - case .failure(let error): - throw error - } - } -} diff --git a/Tests/ArrowTests/Array/ListArrayTests.swift b/Tests/ArrowTests/ListArrayTests.swift similarity index 100% rename from Tests/ArrowTests/Array/ListArrayTests.swift rename to Tests/ArrowTests/ListArrayTests.swift diff --git a/Tests/ArrowTests/Array/ReadmeExamples.swift b/Tests/ArrowTests/ReadmeExamples.swift similarity index 100% rename from Tests/ArrowTests/Array/ReadmeExamples.swift rename to Tests/ArrowTests/ReadmeExamples.swift diff --git a/Tests/ArrowTests/RecordBatchTests.swift b/Tests/ArrowTests/RecordBatchTests.swift index 647f104..937e7b4 100644 --- a/Tests/ArrowTests/RecordBatchTests.swift +++ b/Tests/ArrowTests/RecordBatchTests.swift @@ -18,57 +18,56 @@ import Testing struct RecordBatchTests { @Test func recordBatch() throws { - let uint8Builder: NumberArrayBuilder = - try ArrowArrayBuilders.loadNumberArrayBuilder() + let uint8Builder = ArrayBuilderFixedWidth() uint8Builder.append(10) uint8Builder.append(22) - uint8Builder.append(nil) - let stringBuilder = try ArrowArrayBuilders.loadStringArrayBuilder() + uint8Builder.appendNull() + let stringBuilder = ArrayBuilderVariableLength() stringBuilder.append("test10") stringBuilder.append("test22") stringBuilder.append("test33") - let intArray = try uint8Builder.finish() - let stringArray = try stringBuilder.finish() - let result = RecordBatchX.Builder() - .addColumn("col1", arrowArray: intArray) - .addColumn("col2", arrowArray: stringArray) - .finish() - switch result { - case .success(let recordBatch): - let schema = recordBatch.schema - #expect(schema.fields.count == 2) - #expect(schema.fields[0].name == "col1") - #expect(schema.fields[0].type == .uint8) - #expect(schema.fields[0].isNullable == true) - #expect(schema.fields[1].name == "col2") - #expect(schema.fields[1].type == .utf8) - #expect(schema.fields[1].isNullable == false) - #expect(recordBatch.columns.count == 2) - let col1: any ArrowArray = try recordBatch.data(for: 0) - let col2: any ArrowArray = try recordBatch.data(for: 1) - #expect(col1.length == 3) - #expect(col2.length == 3) - #expect(col1.nullCount == 1) - case .failure(let error): - throw error - } + let intArray = uint8Builder.finish() + let stringArray = stringBuilder.finish() + // let result = RecordBatch.Builder() + // .addColumn("col1", arrowArray: intArray) + // .addColumn("col2", arrowArray: stringArray) + // .finish() + // switch result { + // case .success(let recordBatch): + // let schema = recordBatch.schema + // #expect(schema.fields.count == 2) + // #expect(schema.fields[0].name == "col1") + // #expect(schema.fields[0].type == .uint8) + // #expect(schema.fields[0].isNullable == true) + // #expect(schema.fields[1].name == "col2") + // #expect(schema.fields[1].type == .utf8) + // #expect(schema.fields[1].isNullable == false) + // #expect(recordBatch.columns.count == 2) + // let col1: any ArrowArray = try recordBatch.data(for: 0) + // let col2: any ArrowArray = try recordBatch.data(for: 1) + // #expect(col1.length == 3) + // #expect(col2.length == 3) + // #expect(col1.nullCount == 1) + // case .failure(let error): + // throw error + // } } // Ensure that invalid record batches can't be built. @Test func schemaNullabilityChecked() throws { - let stringBuilder = try ArrowArrayBuilders.loadStringArrayBuilder() - stringBuilder.append("test10") - stringBuilder.append(nil) - stringBuilder.append("test33") - let array = try stringBuilder.finish() - - let field = ArrowField(name: "col1", dataType: .utf8, isNullable: false) - let result = RecordBatchX.Builder() - .addColumn(field, arrowArray: array) - .finish() - if case .success(_) = result { - Issue.record("Record batch should have rejected null data.") - } + // let stringBuilder = try ArrowArrayBuilders.loadStringArrayBuilder() + // stringBuilder.append("test10") + // stringBuilder.append(nil) + // stringBuilder.append("test33") + // let array = try stringBuilder.finish() + // + // let field = ArrowField(name: "col1", dataType: .utf8, isNullable: false) + // let result = RecordBatchX.Builder() + // .addColumn(field, arrowArray: array) + // .finish() + // if case .success(_) = result { + // Issue.record("Record batch should have rejected null data.") + // } } } diff --git a/Tests/ArrowTests/Array/StructArrayTests.swift b/Tests/ArrowTests/StructArrayTests.swift similarity index 100% rename from Tests/ArrowTests/Array/StructArrayTests.swift rename to Tests/ArrowTests/StructArrayTests.swift diff --git a/Tests/ArrowTests/TableTests.swift b/Tests/ArrowTests/TableTests.swift index 59aa61b..87a3386 100644 --- a/Tests/ArrowTests/TableTests.swift +++ b/Tests/ArrowTests/TableTests.swift @@ -37,82 +37,31 @@ struct TableTests { #expect(schema.fields[1].isNullable == false) } - @Test func schemaNested() throws { - struct StructTest { - var field0: Bool = false - var field1: Int8 = 0 - var field2: Int16 = 0 - var field3: Int32 = 0 - var field4: Int64 = 0 - var field5: UInt8 = 0 - var field6: UInt16 = 0 - var field7: UInt32 = 0 - var field8: UInt64 = 0 - var field9: Double = 0 - var field10: Float = 0 - var field11: String = "" - var field12 = Data() - var field13: Date = Date.now - } - - let testObj = StructTest() - var fields: [ArrowField] = [] - let buildStructType = { () -> ArrowType in - let mirror = Mirror(reflecting: testObj) - for (property, value) in mirror.children { - let arrowType = try ArrowTypeConverter.infoForType(type(of: value)) - fields.append( - ArrowField( - name: property!, - dataType: arrowType, - isNullable: true - ) - ) - } - - return .strct(fields) - } - - let structType = try buildStructType() - guard case .strct(let fields) = structType else { - Issue.record("Expected a struct") - return - } - #expect(fields.count == 14) - #expect(fields[0].type == .boolean) - #expect(fields[1].type == .int8) - #expect(fields[2].type == .int16) - #expect(fields[3].type == .int32) - #expect(fields[4].type == .int64) - #expect(fields[5].type == .uint8) - #expect(fields[6].type == .uint16) - #expect(fields[7].type == .uint32) - #expect(fields[8].type == .uint64) - #expect(fields[9].type == .float64) - #expect(fields[10].type == .float32) - #expect(fields[11].type == .utf8) - #expect(fields[12].type == .binary) - #expect(fields[13].type == .date64) - } - @Test func table() throws { - let doubleBuilder: NumberArrayBuilder = - try ArrowArrayBuilders.loadNumberArrayBuilder() + let doubleBuilder: ArrayBuilderFixedWidth = .init() doubleBuilder.append(11.11) doubleBuilder.append(22.22) - let stringBuilder = try ArrowArrayBuilders.loadStringArrayBuilder() + let stringBuilder = ArrayBuilderVariableLength() stringBuilder.append("test10") stringBuilder.append("test22") - let date32Builder: Date32ArrayBuilder = - try ArrowArrayBuilders.loadDate32ArrayBuilder() + let date32Builder: ArrayBuilderDate32 = .init() let date2 = Date(timeIntervalSinceReferenceDate: 86400 * 1) let date1 = Date(timeIntervalSinceReferenceDate: 86400 * 5000 + 352) date32Builder.append(date1) date32Builder.append(date2) let table = try ArrowTable.Builder() - .addColumn("col1", arrowArray: doubleBuilder.finish()) - .addColumn("col2", arrowArray: stringBuilder.finish()) - .addColumn("col3", arrowArray: date32Builder.finish()) + .addColumn( + ArrowField(name: "col1", dataType: .float64, isNullable: false), + arrowArray: doubleBuilder.finish() + ) + .addColumn( + ArrowField(name: "col2", dataType: .utf8, isNullable: false), + arrowArray: stringBuilder.finish() + ) + .addColumn( + ArrowField(name: "col3", dataType: .date32, isNullable: false), + arrowArray: date32Builder.finish() + ) .finish() let schema = table.schema #expect(schema.fields.count == 3) @@ -137,24 +86,20 @@ struct TableTests { } @Test func tableWithChunkedData() throws { - let uint8Builder: NumberArrayBuilder = - try ArrowArrayBuilders.loadNumberArrayBuilder() + let uint8Builder: ArrayBuilderFixedWidth = .init() uint8Builder.append(10) uint8Builder.append(22) - let uint8Builder2: NumberArrayBuilder = - try ArrowArrayBuilders.loadNumberArrayBuilder() + let uint8Builder2: ArrayBuilderFixedWidth = .init() uint8Builder2.append(33) - let uint8Builder3: NumberArrayBuilder = - try ArrowArrayBuilders.loadNumberArrayBuilder() + let uint8Builder3: ArrayBuilderFixedWidth = .init() uint8Builder3.append(44) - let stringBuilder = try ArrowArrayBuilders.loadStringArrayBuilder() + let stringBuilder = ArrayBuilderVariableLength() stringBuilder.append("test10") stringBuilder.append("test22") - let stringBuilder2 = try ArrowArrayBuilders.loadStringArrayBuilder() - stringBuilder.append("test33") - stringBuilder.append("test44") - let date32Builder: Date32ArrayBuilder = - try ArrowArrayBuilders.loadDate32ArrayBuilder() + let stringBuilder2 = ArrayBuilderVariableLength() + stringBuilder2.append("test33") + stringBuilder2.append("test44") + let date32Builder: ArrayBuilderDate32 = .init() let date2 = Date(timeIntervalSinceReferenceDate: 86400 * 1) let date1 = Date(timeIntervalSinceReferenceDate: 86400 * 5000 + 352) date32Builder.append(date1) @@ -169,9 +114,18 @@ struct TableTests { ]) let dateArray = try ChunkedArray([date32Builder.finish()]) let table = ArrowTable.Builder() - .addColumn("col1", chunked: intArray) - .addColumn("col2", chunked: stringArray) - .addColumn("col3", chunked: dateArray) + .addColumn( + ArrowField(name: "col1", dataType: .uint8, isNullable: false), + chunked: intArray + ) + .addColumn( + ArrowField(name: "col2", dataType: .utf8, isNullable: false), + chunked: stringArray + ) + .addColumn( + ArrowField(name: "col3", dataType: .date32, isNullable: false), + chunked: dateArray + ) .finish() let schema = table.schema #expect(schema.fields.count == 3) @@ -197,39 +151,39 @@ struct TableTests { #expect(col2.asString(2) == "test33") } - @Test func tableToRecordBatch() throws { - let uint8Builder: NumberArrayBuilder = - try ArrowArrayBuilders.loadNumberArrayBuilder() - uint8Builder.append(10) - uint8Builder.append(22) - let stringBuilder = try ArrowArrayBuilders.loadStringArrayBuilder() - stringBuilder.append("test10") - stringBuilder.append("test22") - let intArray = try uint8Builder.finish() - let stringArray = try stringBuilder.finish() - let result = RecordBatchX.Builder() - .addColumn("col1", arrowArray: intArray) - .addColumn("col2", arrowArray: stringArray) - .finish().flatMap({ rb in - ArrowTable.from(recordBatches: [rb]) - }) - switch result { - case .success(let table): - let schema = table.schema - #expect(schema.fields.count == 2) - #expect(schema.fields[0].name == "col1") - #expect(schema.fields[0].type == .uint8) - #expect(schema.fields[0].isNullable == false) - #expect(schema.fields[1].name == "col2") - #expect(schema.fields[1].type == .utf8) - #expect(schema.fields[1].isNullable == false) - #expect(table.columns.count == 2) - let col1: ChunkedArray = try table.columns[0].data() - let col2: ChunkedArray = try table.columns[1].data() - #expect(col1.length == 2) - #expect(col2.length == 2) - case .failure(let error): - throw error - } - } + // @Test func tableToRecordBatch() throws { + // let uint8Builder: NumberArrayBuilder = + // try ArrowArrayBuilders.loadNumberArrayBuilder() + // uint8Builder.append(10) + // uint8Builder.append(22) + // let stringBuilder = try ArrowArrayBuilders.loadStringArrayBuilder() + // stringBuilder.append("test10") + // stringBuilder.append("test22") + // let intArray = try uint8Builder.finish() + // let stringArray = try stringBuilder.finish() + // let result = RecordBatchX.Builder() + // .addColumn("col1", arrowArray: intArray) + // .addColumn("col2", arrowArray: stringArray) + // .finish().flatMap({ rb in + // ArrowTable.from(recordBatches: [rb]) + // }) + // switch result { + // case .success(let table): + // let schema = table.schema + // #expect(schema.fields.count == 2) + // #expect(schema.fields[0].name == "col1") + // #expect(schema.fields[0].type == .uint8) + // #expect(schema.fields[0].isNullable == false) + // #expect(schema.fields[1].name == "col2") + // #expect(schema.fields[1].type == .utf8) + // #expect(schema.fields[1].isNullable == false) + // #expect(table.columns.count == 2) + // let col1: ChunkedArray = try table.columns[0].data() + // let col2: ChunkedArray = try table.columns[1].data() + // #expect(col1.length == 2) + // #expect(col2.length == 2) + // case .failure(let error): + // throw error + // } + // } }