From 9aac99a99606231cb1f69fbf4701c54ce2ca3732 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 10 Mar 2025 22:58:34 -0700 Subject: [PATCH] [SPARK-51465] Use `Apache Arrow Swift` 19.0.1 --- Sources/SparkConnect/ArrowArray.swift | 331 +++ Sources/SparkConnect/ArrowArrayBuilder.swift | 340 +++ Sources/SparkConnect/ArrowBuffer.swift | 90 + Sources/SparkConnect/ArrowBufferBuilder.swift | 384 +++ Sources/SparkConnect/ArrowData.swift | 71 + Sources/SparkConnect/ArrowDecoder.swift | 372 +++ Sources/SparkConnect/ArrowEncoder.swift | 459 ++++ Sources/SparkConnect/ArrowReader.swift | 375 +++ Sources/SparkConnect/ArrowReaderHelper.swift | 313 +++ Sources/SparkConnect/ArrowSchema.swift | 74 + Sources/SparkConnect/ArrowTable.swift | 203 ++ Sources/SparkConnect/ArrowType.swift | 406 ++++ Sources/SparkConnect/ArrowWriter.swift | 364 +++ Sources/SparkConnect/ArrowWriterHelper.swift | 142 ++ Sources/SparkConnect/BitUtility.swift | 40 + Sources/SparkConnect/ChunkedArray.swift | 149 ++ Sources/SparkConnect/File_generated.swift | 258 ++ Sources/SparkConnect/Flight.pb.swift | 1366 +++++++++++ Sources/SparkConnect/FlightData.swift | 49 + Sources/SparkConnect/FlightDescriptor.swift | 56 + Sources/SparkConnect/MemoryAllocator.swift | 31 + Sources/SparkConnect/Message_generated.swift | 629 +++++ Sources/SparkConnect/ProtoUtil.swift | 72 + Sources/SparkConnect/Schema_generated.swift | 2113 +++++++++++++++++ .../SparkConnect/SparseTensor_generated.swift | 863 +++++++ Sources/SparkConnect/Tensor_generated.swift | 309 +++ 26 files changed, 9859 insertions(+) create mode 100644 Sources/SparkConnect/ArrowArray.swift create mode 100644 Sources/SparkConnect/ArrowArrayBuilder.swift create mode 100644 Sources/SparkConnect/ArrowBuffer.swift create mode 100644 Sources/SparkConnect/ArrowBufferBuilder.swift create mode 100644 Sources/SparkConnect/ArrowData.swift create mode 100644 Sources/SparkConnect/ArrowDecoder.swift create mode 100644 Sources/SparkConnect/ArrowEncoder.swift create mode 100644 Sources/SparkConnect/ArrowReader.swift create mode 100644 Sources/SparkConnect/ArrowReaderHelper.swift create mode 100644 Sources/SparkConnect/ArrowSchema.swift create mode 100644 Sources/SparkConnect/ArrowTable.swift create mode 100644 Sources/SparkConnect/ArrowType.swift create mode 100644 Sources/SparkConnect/ArrowWriter.swift create mode 100644 Sources/SparkConnect/ArrowWriterHelper.swift create mode 100644 Sources/SparkConnect/BitUtility.swift create mode 100644 Sources/SparkConnect/ChunkedArray.swift create mode 100644 Sources/SparkConnect/File_generated.swift create mode 100644 Sources/SparkConnect/Flight.pb.swift create mode 100644 Sources/SparkConnect/FlightData.swift create mode 100644 Sources/SparkConnect/FlightDescriptor.swift create mode 100644 Sources/SparkConnect/MemoryAllocator.swift create mode 100644 Sources/SparkConnect/Message_generated.swift create mode 100644 Sources/SparkConnect/ProtoUtil.swift create mode 100644 Sources/SparkConnect/Schema_generated.swift create mode 100644 Sources/SparkConnect/SparseTensor_generated.swift create mode 100644 Sources/SparkConnect/Tensor_generated.swift diff --git a/Sources/SparkConnect/ArrowArray.swift b/Sources/SparkConnect/ArrowArray.swift new file mode 100644 index 0000000..1f96a1d --- /dev/null +++ b/Sources/SparkConnect/ArrowArray.swift @@ -0,0 +1,331 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +import Foundation + +public protocol ArrowArrayHolder { + var type: ArrowType { get } + var length: UInt { get } + var nullCount: UInt { get } + var array: AnyArray { get } + var data: ArrowData { get } + var getBufferData: () -> [Data] { get } + var getBufferDataSizes: () -> [Int] { get } + var getArrowColumn: (ArrowField, [ArrowArrayHolder]) throws -> ArrowColumn { get } +} + +public class ArrowArrayHolderImpl: ArrowArrayHolder { + public let data: ArrowData + public let type: ArrowType + public let length: UInt + public let nullCount: UInt + public let array: AnyArray + public let getBufferData: () -> [Data] + public let getBufferDataSizes: () -> [Int] + public let getArrowColumn: (ArrowField, [ArrowArrayHolder]) throws -> ArrowColumn + public init(_ arrowArray: ArrowArray) { + self.array = arrowArray + self.data = arrowArray.arrowData + self.length = arrowArray.length + self.type = arrowArray.arrowData.type + self.nullCount = arrowArray.nullCount + self.getBufferData = { () -> [Data] in + var bufferData = [Data]() + for buffer in arrowArray.arrowData.buffers { + bufferData.append(Data()) + buffer.append(to: &bufferData[bufferData.count - 1]) + } + + return bufferData + } + + self.getBufferDataSizes = { () -> [Int] in + var bufferDataSizes = [Int]() + for buffer in arrowArray.arrowData.buffers { + bufferDataSizes.append(Int(buffer.capacity)) + } + + return bufferDataSizes + } + + self.getArrowColumn = { + (field: ArrowField, arrayHolders: [ArrowArrayHolder]) throws -> ArrowColumn in + var arrays = [ArrowArray]() + for arrayHolder in arrayHolders { + if let array = arrayHolder.array as? ArrowArray { + arrays.append(array) + } + } + + return ArrowColumn(field, chunked: ChunkedArrayHolder(try ChunkedArray(arrays))) + } + } + + public static func loadArray( // swiftlint:disable:this cyclomatic_complexity + _ arrowType: ArrowType, with: ArrowData + ) throws -> ArrowArrayHolder { + switch arrowType.id { + case .int8: + return try ArrowArrayHolderImpl(FixedArray(with)) + case .int16: + return try ArrowArrayHolderImpl(FixedArray(with)) + case .int32: + return try ArrowArrayHolderImpl(FixedArray(with)) + case .int64: + return try ArrowArrayHolderImpl(FixedArray(with)) + case .uint8: + return try ArrowArrayHolderImpl(FixedArray(with)) + case .uint16: + return try ArrowArrayHolderImpl(FixedArray(with)) + case .uint32: + return try ArrowArrayHolderImpl(FixedArray(with)) + case .uint64: + return try ArrowArrayHolderImpl(FixedArray(with)) + case .double: + return try ArrowArrayHolderImpl(FixedArray(with)) + case .float: + return try ArrowArrayHolderImpl(FixedArray(with)) + case .date32: + return try ArrowArrayHolderImpl(Date32Array(with)) + case .date64: + return try ArrowArrayHolderImpl(Date64Array(with)) + case .time32: + return try ArrowArrayHolderImpl(Time32Array(with)) + case .time64: + return try ArrowArrayHolderImpl(Time64Array(with)) + case .string: + return try ArrowArrayHolderImpl(StringArray(with)) + case .boolean: + return try ArrowArrayHolderImpl(BoolArray(with)) + case .binary: + return try ArrowArrayHolderImpl(BinaryArray(with)) + case .strct: + return try ArrowArrayHolderImpl(StructArray(with)) + default: + throw ArrowError.invalid("Array not found for type: \(arrowType)") + } + } +} + +public class ArrowArray: AsString, AnyArray { + public typealias ItemType = T + public let arrowData: ArrowData + public var nullCount: UInt { return self.arrowData.nullCount } + public var length: UInt { return self.arrowData.length } + + public required init(_ arrowData: ArrowData) throws { + self.arrowData = arrowData + } + + public func isNull(_ at: UInt) throws -> Bool { + if at >= self.length { + throw ArrowError.outOfBounds(index: Int64(at)) + } + + return self.arrowData.isNull(at) + } + + public subscript(_ index: UInt) -> T? { + fatalError("subscript() has not been implemented") + } + + public func asString(_ index: UInt) -> String { + if self[index] == nil { + return "" + } + + return "\(self[index]!)" + } + + public func asAny(_ index: UInt) -> Any? { + if self[index] == nil { + return nil + } + + return self[index]! + } +} + +public class FixedArray: ArrowArray { + public override subscript(_ index: UInt) -> T? { + if self.arrowData.isNull(index) { + return nil + } + + let byteOffset = self.arrowData.stride * Int(index) + return self.arrowData.buffers[1].rawPointer.advanced(by: byteOffset).load(as: T.self) + } +} + +public class StringArray: ArrowArray { + public override subscript(_ index: UInt) -> String? { + let offsetIndex = MemoryLayout.stride * Int(index) + if self.arrowData.isNull(index) { + return nil + } + + let offsets = self.arrowData.buffers[1] + let values = self.arrowData.buffers[2] + + 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) + let arrayLength = Int(endIndex - startIndex) + let rawPointer = values.rawPointer.advanced(by: Int(startIndex)) + .bindMemory(to: UInt8.self, capacity: arrayLength) + let buffer = UnsafeBufferPointer(start: rawPointer, count: arrayLength) + let byteArray = Array(buffer) + return String(data: Data(byteArray), encoding: .utf8) + } +} + +public class BoolArray: ArrowArray { + public override subscript(_ index: UInt) -> Bool? { + if self.arrowData.isNull(index) { + return nil + } + + let valueBuffer = self.arrowData.buffers[1] + return BitUtility.isSet(index, buffer: valueBuffer) + } +} + +public class Date32Array: ArrowArray { + public override subscript(_ index: UInt) -> Date? { + if self.arrowData.isNull(index) { + return nil + } + + let byteOffset = self.arrowData.stride * Int(index) + let milliseconds = self.arrowData.buffers[1].rawPointer.advanced(by: byteOffset).load( + as: UInt32.self) + return Date(timeIntervalSince1970: TimeInterval(milliseconds * 86400)) + } +} + +public class Date64Array: ArrowArray { + public override subscript(_ index: UInt) -> Date? { + if self.arrowData.isNull(index) { + return nil + } + + let byteOffset = self.arrowData.stride * Int(index) + let milliseconds = self.arrowData.buffers[1].rawPointer.advanced(by: byteOffset).load( + as: UInt64.self) + return Date(timeIntervalSince1970: TimeInterval(milliseconds / 1000)) + } +} + +public class Time32Array: FixedArray {} +public class Time64Array: FixedArray {} + +public class BinaryArray: ArrowArray { + public struct Options { + public var printAsHex = false + public var printEncoding: String.Encoding = .utf8 + } + + public var options = Options() + + public override subscript(_ index: UInt) -> Data? { + let offsetIndex = MemoryLayout.stride * Int(index) + if self.arrowData.isNull(index) { + return nil + } + + let offsets = self.arrowData.buffers[1] + let values = self.arrowData.buffers[2] + 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) + let arrayLength = Int(endIndex - startIndex) + let rawPointer = values.rawPointer.advanced(by: Int(startIndex)) + .bindMemory(to: UInt8.self, capacity: arrayLength) + let buffer = UnsafeBufferPointer(start: rawPointer, count: arrayLength) + let byteArray = Array(buffer) + return Data(byteArray) + } + + public override func asString(_ index: UInt) -> String { + if self[index] == nil { return "" } + let data = self[index]! + if options.printAsHex { + return data.hexEncodedString() + } else { + return String(data: data, encoding: .utf8)! + } + } +} + +public class StructArray: ArrowArray<[Any?]> { + public private(set) var arrowFields: [ArrowArrayHolder]? + public required init(_ arrowData: ArrowData) throws { + try super.init(arrowData) + var fields = [ArrowArrayHolder]() + for child in arrowData.children { + fields.append(try ArrowArrayHolderImpl.loadArray(child.type, with: child)) + } + + self.arrowFields = fields + } + + public override subscript(_ index: UInt) -> [Any?]? { + if self.arrowData.isNull(index) { + return nil + } + + if let fields = arrowFields { + var result = [Any?]() + for field in fields { + result.append(field.array.asAny(index)) + } + + return result + } + + return nil + } + + public override func asString(_ index: UInt) -> String { + if self.arrowData.isNull(index) { + return "" + } + + var output = "{" + if let fields = arrowFields { + for fieldIndex in 0.. ArrowArrayHolder + func appendAny(_ val: Any?) +} + +public class ArrowArrayBuilder>: + ArrowArrayHolderBuilder +{ + let type: ArrowType + let bufferBuilder: T + public var length: UInt { return self.bufferBuilder.length } + public var capacity: UInt { return self.bufferBuilder.capacity } + public var nullCount: UInt { return self.bufferBuilder.nullCount } + public var offset: UInt { return self.bufferBuilder.offset } + + fileprivate init(_ type: ArrowType) throws { + self.type = type + self.bufferBuilder = try T() + } + + public func append(_ vals: T.ItemType?...) { + for val in vals { + self.bufferBuilder.append(val) + } + } + + public func append(_ vals: [T.ItemType?]) { + for val in vals { + self.bufferBuilder.append(val) + } + } + + public func append(_ val: T.ItemType?) { + self.bufferBuilder.append(val) + } + + public func appendAny(_ val: Any?) { + self.bufferBuilder.append(val as? T.ItemType) + } + + public func finish() throws -> ArrowArray { + let buffers = self.bufferBuilder.finish() + let arrowData = try ArrowData(self.type, buffers: buffers, nullCount: self.nullCount) + let array = try U(arrowData) + return array + } + + public func getStride() -> Int { + return self.type.getStride() + } + + public func toHolder() throws -> ArrowArrayHolder { + return try ArrowArrayHolderImpl(self.finish()) + } +} + +public class NumberArrayBuilder: ArrowArrayBuilder, FixedArray> { + fileprivate convenience init() throws { + try self.init(ArrowType(ArrowType.infoForNumericType(T.self))) + } +} + +public class StringArrayBuilder: ArrowArrayBuilder, StringArray> { + fileprivate convenience init() throws { + try self.init(ArrowType(ArrowType.ArrowString)) + } +} + +public class BinaryArrayBuilder: ArrowArrayBuilder, BinaryArray> { + fileprivate convenience init() throws { + try self.init(ArrowType(ArrowType.ArrowBinary)) + } +} + +public class BoolArrayBuilder: ArrowArrayBuilder { + fileprivate convenience init() throws { + try self.init(ArrowType(ArrowType.ArrowBool)) + } +} + +public class Date32ArrayBuilder: ArrowArrayBuilder { + fileprivate convenience init() throws { + try self.init(ArrowType(ArrowType.ArrowDate32)) + } +} + +public class Date64ArrayBuilder: ArrowArrayBuilder { + fileprivate convenience init() throws { + try self.init(ArrowType(ArrowType.ArrowDate64)) + } +} + +public class Time32ArrayBuilder: ArrowArrayBuilder, Time32Array> { + fileprivate convenience init(_ unit: ArrowTime32Unit) throws { + try self.init(ArrowTypeTime32(unit)) + } +} + +public class Time64ArrayBuilder: ArrowArrayBuilder, Time64Array> { + fileprivate convenience init(_ unit: ArrowTime64Unit) throws { + try self.init(ArrowTypeTime64(unit)) + } +} + +public class StructArrayBuilder: ArrowArrayBuilder { + let builders: [any ArrowArrayHolderBuilder] + let fields: [ArrowField] + public init(_ fields: [ArrowField], builders: [any ArrowArrayHolderBuilder]) throws { + self.fields = fields + self.builders = builders + try super.init(ArrowNestedType(ArrowType.ArrowStruct, fields: fields)) + self.bufferBuilder.initializeTypeInfo(fields) + } + + public init(_ fields: [ArrowField]) throws { + self.fields = fields + var builders = [any ArrowArrayHolderBuilder]() + for field in fields { + builders.append(try ArrowArrayBuilders.loadBuilder(arrowType: field.type)) + } + + self.builders = builders + try super.init(ArrowNestedType(ArrowType.ArrowStruct, fields: fields)) + } + + public override func append(_ values: [Any?]?) { + self.bufferBuilder.append(values) + if let anyValues = values { + for index in 0.. StructArray { + let buffers = self.bufferBuilder.finish() + var childData = [ArrowData]() + for builder in self.builders { + childData.append(try builder.toHolder().array.arrowData) + } + + let arrowData = try ArrowData( + self.type, buffers: buffers, + children: childData, nullCount: self.nullCount, + length: self.length) + let structArray = try StructArray(arrowData) + return structArray + } +} + +public class ArrowArrayBuilders { + public static func loadBuilder( // swiftlint:disable:this cyclomatic_complexity + _ builderType: Any.Type + ) throws -> ArrowArrayHolderBuilder { + 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 ArrowError.invalid("Invalid type for builder: \(builderType)") + } + } + + public static func isValidBuilderType(_ type: T.Type) -> Bool { + return 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 loadStructArrayBuilderForType(_ obj: T) throws -> StructArrayBuilder { + let mirror = Mirror(reflecting: obj) + var builders = [ArrowArrayHolderBuilder]() + var fields = [ArrowField]() + for (property, value) in mirror.children { + guard let propertyName = property else { + continue + } + + let builderType = type(of: value) + let arrowType = ArrowType(ArrowType.infoForType(builderType)) + fields.append(ArrowField(propertyName, type: arrowType, isNullable: true)) + builders.append(try loadBuilder(arrowType: arrowType)) + } + + return try StructArrayBuilder(fields, builders: builders) + } + + public static func loadBuilder( // swiftlint:disable:this cyclomatic_complexity + arrowType: ArrowType + ) throws -> ArrowArrayHolderBuilder { + switch arrowType.id { + 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 .double: + return try loadNumberArrayBuilder() as NumberArrayBuilder + case .float: + return try loadNumberArrayBuilder() as NumberArrayBuilder + case .string: + 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: + guard let timeType = arrowType as? ArrowTypeTime32 else { + throw ArrowError.invalid("Expected arrow type for \(arrowType.id) not found") + } + return try Time32ArrayBuilder(timeType.unit) + case .time64: + guard let timeType = arrowType as? ArrowTypeTime64 else { + throw ArrowError.invalid("Expected arrow type for \(arrowType.id) not found") + } + return try Time64ArrayBuilder(timeType.unit) + default: + throw ArrowError.unknownType("Builder not found for arrow type: \(arrowType.id)") + } + } + + public static func loadNumberArrayBuilder() throws -> 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 -> StringArrayBuilder { + return try StringArrayBuilder() + } + + public static func loadBoolArrayBuilder() throws -> BoolArrayBuilder { + return try BoolArrayBuilder() + } + + public static func loadDate32ArrayBuilder() throws -> Date32ArrayBuilder { + return try Date32ArrayBuilder() + } + + public static func loadDate64ArrayBuilder() throws -> Date64ArrayBuilder { + return try Date64ArrayBuilder() + } + + public static func loadBinaryArrayBuilder() throws -> BinaryArrayBuilder { + return try BinaryArrayBuilder() + } + + public static func loadTime32ArrayBuilder(_ unit: ArrowTime32Unit) throws -> Time32ArrayBuilder { + return try Time32ArrayBuilder(unit) + } + + public static func loadTime64ArrayBuilder(_ unit: ArrowTime64Unit) throws -> Time64ArrayBuilder { + return try Time64ArrayBuilder(unit) + } +} diff --git a/Sources/SparkConnect/ArrowBuffer.swift b/Sources/SparkConnect/ArrowBuffer.swift new file mode 100644 index 0000000..e6736af --- /dev/null +++ b/Sources/SparkConnect/ArrowBuffer.swift @@ -0,0 +1,90 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +import 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 { + return 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 memory = MemoryAllocator(64) + let rawPointer = memory.allocateArray(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 memory = MemoryAllocator(64) + let rawPointer = memory.allocateArray(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)) + } + + private static func alignTo64(_ length: UInt) -> UInt { + let bufAlignment = length % 64 + if bufAlignment != 0 { + return length + (64 - bufAlignment) + 8 + } + + return length + 8 + } +} diff --git a/Sources/SparkConnect/ArrowBufferBuilder.swift b/Sources/SparkConnect/ArrowBufferBuilder.swift new file mode 100644 index 0000000..e835f3a --- /dev/null +++ b/Sources/SparkConnect/ArrowBufferBuilder.swift @@ -0,0 +1,384 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +import Foundation + +public protocol ArrowBufferBuilder { + associatedtype ItemType + var capacity: UInt { get } + var length: UInt { get } + var nullCount: UInt { get } + var offset: UInt { get } + init() throws + 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 { return 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 { + return 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 { return self.values.capacity } + + init(values: ArrowBuffer, nulls: ArrowBuffer, stride: Int = MemoryLayout.stride) { + self.stride = stride + self.values = values + super.init(nulls) + } +} + +public class FixedBufferBuilder: ValuesBufferBuilder, ArrowBufferBuilder { + public typealias ItemType = T + private let defaultVal: ItemType + public required init() throws { + self.defaultVal = try FixedBufferBuilder.defaultValueForType() + 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] + } + + fileprivate static func defaultValueForType() throws -> T { + let type = T.self + if type == Int8.self { + return Int8(0) as! T // swiftlint:disable:this force_cast + } else if type == Int16.self { + return Int16(0) as! T // swiftlint:disable:this force_cast + } else if type == Int32.self { + return Int32(0) as! T // swiftlint:disable:this force_cast + } else if type == Int64.self { + return Int64(0) as! T // swiftlint:disable:this force_cast + } else if type == UInt8.self { + return UInt8(0) as! T // swiftlint:disable:this force_cast + } else if type == UInt16.self { + return UInt16(0) as! T // swiftlint:disable:this force_cast + } else if type == UInt32.self { + return UInt32(0) as! T // swiftlint:disable:this force_cast + } else if type == UInt64.self { + return UInt64(0) as! T // swiftlint:disable:this force_cast + } else if type == Float.self { + return Float(0) as! T // swiftlint:disable:this force_cast + } else if type == Double.self { + return Double(0) as! T // swiftlint:disable:this force_cast + } + + throw ArrowError.unknownType("Unable to determine default value") + } +} + +public class BoolBufferBuilder: ValuesBufferBuilder, ArrowBufferBuilder { + public typealias ItemType = Bool + public required init() throws { + 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 { + public typealias ItemType = T + var offsets: ArrowBuffer + let binaryStride = MemoryLayout.stride + public required init() throws { + 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 binData: Data + var isNull = false + if let val = newValue { + binData = getBytesFor(val)! + } else { + var nullVal = 0 + isNull = true + binData = Data(bytes: &nullVal, count: MemoryLayout.size) + } + + var currentIndex: Int32 = 0 + var currentOffset: Int32 = Int32(binData.count) + if index > 0 { + currentIndex = self.offsets.rawPointer.advanced(by: offsetIndex).load(as: Int32.self) + currentOffset += currentIndex + if currentOffset > self.values.length { + self.value_resize(UInt(currentOffset)) + } + } + + if isNull { + self.nullCount += 1 + BitUtility.clearBit(index + self.offset, buffer: self.nulls) + } else { + BitUtility.setBit(index + self.offset, buffer: self.nulls) + } + + binData.withUnsafeBytes { bufferPointer in + let rawPointer = bufferPointer.baseAddress! + self.values.rawPointer.advanced(by: Int(currentIndex)) + .copyMemory(from: rawPointer, byteCount: binData.count) + } + + self.offsets.rawPointer.advanced(by: (offsetIndex + MemoryLayout.stride)) + .storeBytes(of: currentOffset, as: Int32.self) + } + + public func value_resize(_ 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 { + public typealias ItemType = T + public var capacity: UInt { return self.bufferBuilder.capacity } + public var length: UInt { return self.bufferBuilder.length } + public var nullCount: UInt { return self.bufferBuilder.nullCount } + public var offset: UInt { return self.bufferBuilder.offset } + let bufferBuilder: FixedBufferBuilder + public required init() throws { + self.bufferBuilder = try FixedBufferBuilder() + } + + public func append(_ newValue: ItemType?) { + fatalError("Method is not implemented") + } + + public func isNull(_ index: UInt) -> Bool { + return self.bufferBuilder.isNull(index) + } + + public func resize(_ length: UInt) { + self.bufferBuilder.resize(length) + } + + public func finish() -> [ArrowBuffer] { + return 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: ArrowNestedType? + public init() throws { + let nulls = ArrowBuffer.createBuffer(0, size: UInt(MemoryLayout.stride)) + super.init(nulls) + } + + public func initializeTypeInfo(_ fields: [ArrowField]) { + info = ArrowNestedType(ArrowType.ArrowStruct, fields: fields) + } + + public func append(_ newValue: [Any?]?) { + let index = UInt(self.length) + self.length += 1 + if 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] + } +} diff --git a/Sources/SparkConnect/ArrowData.swift b/Sources/SparkConnect/ArrowData.swift new file mode 100644 index 0000000..1817e35 --- /dev/null +++ b/Sources/SparkConnect/ArrowData.swift @@ -0,0 +1,71 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +import Foundation + +public class ArrowData { + public let type: ArrowType + public let buffers: [ArrowBuffer] + public let children: [ArrowData] + public let nullCount: UInt + public let length: UInt + public let stride: Int + + convenience init(_ arrowType: ArrowType, buffers: [ArrowBuffer], nullCount: UInt) throws { + try self.init( + arrowType, buffers: buffers, + children: [ArrowData](), nullCount: nullCount, + length: buffers[1].length) + } + + init( + _ arrowType: ArrowType, buffers: [ArrowBuffer], children: [ArrowData], nullCount: UInt, + length: UInt + ) throws { + let infoType = arrowType.info + switch infoType { + case let .primitiveInfo(typeId): + if typeId == ArrowTypeId.unknown { + throw ArrowError.unknownType("Unknown primitive type for data") + } + case let .variableInfo(typeId): + if typeId == ArrowTypeId.unknown { + throw ArrowError.unknownType("Unknown variable type for data") + } + case let .timeInfo(typeId): + if typeId == ArrowTypeId.unknown { + throw ArrowError.unknownType("Unknown time type for data") + } + case let .complexInfo(typeId): + if typeId == ArrowTypeId.unknown { + throw ArrowError.unknownType("Unknown complex type for data") + } + } + + self.type = arrowType + self.buffers = buffers + self.children = children + self.nullCount = nullCount + self.length = length + self.stride = arrowType.getStride() + } + + public func isNull(_ at: UInt) -> Bool { + let nullBuffer = buffers[0] + return nullBuffer.length > 0 && !BitUtility.isSet(at, buffer: nullBuffer) + } +} diff --git a/Sources/SparkConnect/ArrowDecoder.swift b/Sources/SparkConnect/ArrowDecoder.swift new file mode 100644 index 0000000..6d68fa4 --- /dev/null +++ b/Sources/SparkConnect/ArrowDecoder.swift @@ -0,0 +1,372 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +import 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: RecordBatch + public let nameToCol: [String: ArrowArrayHolder] + public let columns: [ArrowArrayHolder] + 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: RecordBatch) { + self.rb = rb + var colMapping = [String: ArrowArrayHolder]() + var columns = [ArrowArrayHolder]() + 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 { + return ArrowUnkeyedDecoding(self, codingPath: codingPath) + } + + public func singleValueContainer() -> SingleValueDecodingContainer { + return ArrowSingleValueDecoding(self, codingPath: codingPath) + } + + func getCol(_ name: String) throws -> AnyArray { + guard let col = self.nameToCol[name] else { + throw ArrowError.invalid("Column for key \"\(name)\" not found") + } + + return col.array + } + + func getCol(_ index: Int) throws -> AnyArray { + if index >= self.columns.count { + throw ArrowError.outOfBounds(index: Int64(index)) + } + + return self.columns[index].array + } + + func doDecode(_ key: CodingKey) throws -> T? { + let array: AnyArray = try self.getCol(key.stringValue) + return array.asAny(self.rbIndex) as? T + } + + func doDecode(_ col: Int) throws -> T? { + let array: AnyArray = try self.getCol(col) + return array.asAny(self.rbIndex) as? T + } + + func isNull(_ key: CodingKey) throws -> Bool { + let array: AnyArray = try self.getCol(key.stringValue) + return array.asAny(self.rbIndex) == nil + } + + func isNull(_ col: Int) throws -> Bool { + let array: AnyArray = 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! + } + + 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() } + return try self.decoder.doDecode(self.currentIndex)! + } 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 { + return 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 { + return try self.decoder.doDecode(key)! + } + + func decode(_ type: String.Type, forKey key: Key) throws -> String { + return try self.decoder.doDecode(key)! + } + + func decode(_ type: Double.Type, forKey key: Key) throws -> Double { + return try self.decoder.doDecode(key)! + } + + func decode(_ type: Float.Type, forKey key: Key) throws -> Float { + return try self.decoder.doDecode(key)! + } + + 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 { + return try self.decoder.doDecode(key)! + } + + func decode(_ type: Int16.Type, forKey key: Key) throws -> Int16 { + return try self.decoder.doDecode(key)! + } + + func decode(_ type: Int32.Type, forKey key: Key) throws -> Int32 { + return try self.decoder.doDecode(key)! + } + + func decode(_ type: Int64.Type, forKey key: Key) throws -> Int64 { + return try self.decoder.doDecode(key)! + } + + 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 { + return try self.decoder.doDecode(key)! + } + + func decode(_ type: UInt16.Type, forKey key: Key) throws -> UInt16 { + return try self.decoder.doDecode(key)! + } + + func decode(_ type: UInt32.Type, forKey key: Key) throws -> UInt32 { + return try self.decoder.doDecode(key)! + } + + func decode(_ type: UInt64.Type, forKey key: Key) throws -> UInt64 { + return try self.decoder.doDecode(key)! + } + + func decode(_ type: T.Type, forKey key: Key) throws -> T where T: Decodable { + if ArrowArrayBuilders.isValidBuilderType(type) || type == Date.self { + return try self.decoder.doDecode(key)! + } 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 { + return try self.decoder.doDecode(self.decoder.singleRBCol)! + } + + func decode(_ type: String.Type) throws -> String { + return try self.decoder.doDecode(self.decoder.singleRBCol)! + } + + func decode(_ type: Double.Type) throws -> Double { + return try self.decoder.doDecode(self.decoder.singleRBCol)! + } + + func decode(_ type: Float.Type) throws -> Float { + return try self.decoder.doDecode(self.decoder.singleRBCol)! + } + + 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 { + return try self.decoder.doDecode(self.decoder.singleRBCol)! + } + + func decode(_ type: Int16.Type) throws -> Int16 { + return try self.decoder.doDecode(self.decoder.singleRBCol)! + } + + func decode(_ type: Int32.Type) throws -> Int32 { + return try self.decoder.doDecode(self.decoder.singleRBCol)! + } + + func decode(_ type: Int64.Type) throws -> Int64 { + return try self.decoder.doDecode(self.decoder.singleRBCol)! + } + + 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 { + return try self.decoder.doDecode(self.decoder.singleRBCol)! + } + + func decode(_ type: UInt16.Type) throws -> UInt16 { + return try self.decoder.doDecode(self.decoder.singleRBCol)! + } + + func decode(_ type: UInt32.Type) throws -> UInt32 { + return try self.decoder.doDecode(self.decoder.singleRBCol)! + } + + func decode(_ type: UInt64.Type) throws -> UInt64 { + return try self.decoder.doDecode(self.decoder.singleRBCol)! + } + + func decode(_ type: T.Type) throws -> T where T: Decodable { + if ArrowArrayBuilders.isValidBuilderType(type) || type == Date.self { + return try self.decoder.doDecode(self.decoder.singleRBCol)! + } else { + throw ArrowError.invalid("Type \(type) is currently not supported") + } + } +} diff --git a/Sources/SparkConnect/ArrowEncoder.swift b/Sources/SparkConnect/ArrowEncoder.swift new file mode 100644 index 0000000..7b21f78 --- /dev/null +++ b/Sources/SparkConnect/ArrowEncoder.swift @@ -0,0 +1,459 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +import Foundation + +public class ArrowEncoder: Encoder { + public private(set) var builders = [String: ArrowArrayHolderBuilder]() + 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 there 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: ArrowArrayHolderBuilder], byIndex: [String]) { + self.builders = builders + self.byIndex = byIndex + } + + public static func encode(_ data: T) throws -> RecordBatch { + let encoder = try loadEncoder(data) + try data.encode(to: encoder) + return try encoder.finish() + } + + public static func encode(_ rows: [T]) throws -> RecordBatch? { + 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.loadBuilder(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 -> RecordBatch { + try throwIfInvalid() + let batchBuilder = RecordBatch.Builder() + for key in byIndex { + batchBuilder.addColumn(key, arrowArray: try builders[key]!.toHolder()) + } + + switch batchBuilder.finish() { + case .success(let rb): + return rb + case .failure(let error): + throw error + } + } + + public func container(keyedBy type: Key.Type) -> KeyedEncodingContainer + where Key: CodingKey { + var container = ArrowKeyedEncoding(self) + container.codingPath = codingPath + return KeyedEncodingContainer(container) + } + + public func unkeyedContainer() -> UnkeyedEncodingContainer { + return ArrowUnkeyedEncoding(self, codingPath: self.codingPath) + } + + public func singleValueContainer() -> SingleValueEncodingContainer { + return 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 -> ArrowArrayHolderBuilder { + try throwIfInvalid() + var builder = builders[key] + if builder == nil { + builder = try ArrowArrayBuilders.loadBuilder(T.self) + builders[key] = builder + byIndex.append(key) + } + + return builder! + } + + func getIndex(_ index: Int) -> Int { + return self.modForIndex == nil ? index : index % self.modForIndex! + } + + func doEncodeNil(_ keyIndex: Int) throws { + try throwIfInvalid() + let index = self.getIndex(keyIndex) + if index >= builders.count { + throw ArrowError.outOfBounds(index: Int64(index)) + } + + builders[byIndex[index]]!.appendAny(nil) + } + + func doEncode(_ value: T, key: CodingKey) throws { + try throwIfInvalid() + let builder = try ensureColumnExists(value, key: key.stringValue) + builder.appendAny(value) + } + + func doEncode(_ value: T, keyIndex: Int) throws { + try throwIfInvalid() + let index = self.getIndex(keyIndex) + if index >= builders.count { + if index == builders.count { + try ensureColumnExists(value, key: "col\(index)") + } else { + throw ArrowError.outOfBounds(index: Int64(index)) + } + } + + builders[byIndex[index]]!.appendAny(value) + } + + func throwIfInvalid() throws { + 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 { + return 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/SparkConnect/ArrowReader.swift b/Sources/SparkConnect/ArrowReader.swift new file mode 100644 index 0000000..8a98db9 --- /dev/null +++ b/Sources/SparkConnect/ArrowReader.swift @@ -0,0 +1,375 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +import FlatBuffers +import Foundation + +let FILEMARKER = "ARROW1" +let CONTINUATIONMARKER = -1 + +public class ArrowReader { // swiftlint:disable:this type_body_length + private class RecordBatchData { + let schema: org_apache_arrow_flatbuf_Schema + let recordBatch: org_apache_arrow_flatbuf_RecordBatch + private var fieldIndex: Int32 = 0 + private var nodeIndex: Int32 = 0 + private var bufferIndex: Int32 = 0 + init( + _ recordBatch: org_apache_arrow_flatbuf_RecordBatch, + schema: org_apache_arrow_flatbuf_Schema + ) { + self.recordBatch = recordBatch + self.schema = schema + } + + func nextNode() -> org_apache_arrow_flatbuf_FieldNode? { + if nodeIndex >= self.recordBatch.nodesCount { return nil } + defer { nodeIndex += 1 } + return self.recordBatch.nodes(at: nodeIndex) + } + + func nextBuffer() -> org_apache_arrow_flatbuf_Buffer? { + if bufferIndex >= self.recordBatch.buffersCount { return nil } + defer { bufferIndex += 1 } + return self.recordBatch.buffers(at: bufferIndex) + } + + func nextField() -> org_apache_arrow_flatbuf_Field? { + if fieldIndex >= self.schema.fieldsCount { return nil } + defer { fieldIndex += 1 } + return self.schema.fields(at: fieldIndex) + } + + func isDone() -> Bool { + return nodeIndex >= self.recordBatch.nodesCount + } + } + + private struct DataLoadInfo { + let fileData: Data + let messageOffset: Int64 + var batchData: RecordBatchData + } + + public class ArrowReaderResult { + fileprivate var messageSchema: org_apache_arrow_flatbuf_Schema? + public var schema: ArrowSchema? + public var batches = [RecordBatch]() + } + + public init() {} + + private func loadSchema(_ schema: org_apache_arrow_flatbuf_Schema) -> Result< + ArrowSchema, ArrowError + > { + 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 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)) + } + + private func loadVariableData( + _ loadInfo: DataLoadInfo, + field: org_apache_arrow_flatbuf_Field + ) + -> 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: org_apache_arrow_flatbuf_Field + ) + -> Result + { + if isNestedType(field.typeType) { + return loadStructData(loadInfo, field: field) + } else if isFixedPrimitive(field.typeType) { + return loadPrimitiveData(loadInfo, field: field) + } else { + return loadVariableData(loadInfo, field: field) + } + } + + private func loadRecordBatch( + _ recordBatch: org_apache_arrow_flatbuf_RecordBatch, + schema: org_apache_arrow_flatbuf_Schema, + arrowSchema: ArrowSchema, + data: Data, + messageEndOffset: Int64 + ) -> Result { + var columns: [ArrowArrayHolder] = [] + 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(RecordBatch(arrowSchema, columns: columns)) + } + + public func fromStream( // swiftlint:disable:this function_body_length + _ 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...] + let footerBuffer = ByteBuffer( + data: footerData, + allowReadingUnalignedBuffers: useUnalignedBuffers) + let footer = org_apache_arrow_flatbuf_Footer.getRootAsFooter(bb: footerBuffer) + let schemaResult = loadSchema(footer.schema!) + switch schemaResult { + case .success(let schema): + result.schema = schema + case .failure(let error): + return .failure(error) + } + + for index in 0...size)), + as: Int32.self) + } + } + + let messageStartOffset = + recordBatch.offset + (Int64(MemoryLayout.size) * messageOffset) + let messageEndOffset = messageStartOffset + Int64(messageLength) + let recordBatchData = fileData[messageStartOffset.. Result { + do { + let fileData = try Data(contentsOf: fileURL) + if !validateFileData(fileData) { + return .failure(.ioError("Not a valid arrow file.")) + } + + let markerLength = FILEMARKER.utf8.count + let footerLengthEnd = Int(fileData.count - markerLength) + let data = fileData[..<(footerLengthEnd)] + return fromStream(data) + } catch { + return .failure(.unknownError("Error loading file: \(error)")) + } + } + + static public func makeArrowReaderResult() -> ArrowReaderResult { + return ArrowReaderResult() + } + + public func fromMessage( + _ dataHeader: Data, + dataBody: Data, + result: ArrowReaderResult, + useUnalignedBuffers: Bool = false + ) -> Result { + let mbb = ByteBuffer( + data: dataHeader, + allowReadingUnalignedBuffers: useUnalignedBuffers) + let message = org_apache_arrow_flatbuf_Message.getRootAsMessage(bb: mbb) + switch message.headerType { + case .schema: + let sMessage = message.header(type: org_apache_arrow_flatbuf_Schema.self)! + switch loadSchema(sMessage) { + case .success(let schema): + result.schema = schema + result.messageSchema = sMessage + return .success(()) + case .failure(let error): + return .failure(error) + } + case .recordbatch: + let rbMessage = message.header(type: org_apache_arrow_flatbuf_RecordBatch.self)! + do { + let recordBatch = try loadRecordBatch( + rbMessage, schema: result.messageSchema!, arrowSchema: result.schema!, + data: dataBody, messageEndOffset: 0 + ).get() + result.batches.append(recordBatch) + return .success(()) + } catch let error as ArrowError { + return .failure(error) + } catch { + return .failure(.unknownError("Unexpected error: \(error)")) + } + + default: + return .failure(.unknownError("Unhandled header type: \(message.headerType)")) + } + } + +} diff --git a/Sources/SparkConnect/ArrowReaderHelper.swift b/Sources/SparkConnect/ArrowReaderHelper.swift new file mode 100644 index 0000000..110ef15 --- /dev/null +++ b/Sources/SparkConnect/ArrowReaderHelper.swift @@ -0,0 +1,313 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +import FlatBuffers +import Foundation + +private func makeBinaryHolder( + _ buffers: [ArrowBuffer], + nullCount: UInt +) -> Result { + do { + let arrowType = ArrowType(ArrowType.ArrowBinary) + let arrowData = try ArrowData(arrowType, buffers: buffers, nullCount: nullCount) + return .success(ArrowArrayHolderImpl(try BinaryArray(arrowData))) + } catch let error as ArrowError { + return .failure(error) + } catch { + return .failure(.unknownError("\(error)")) + } +} + +private func makeStringHolder( + _ buffers: [ArrowBuffer], + nullCount: UInt +) -> Result { + do { + let arrowType = ArrowType(ArrowType.ArrowString) + let arrowData = try ArrowData(arrowType, buffers: buffers, nullCount: nullCount) + return .success(ArrowArrayHolderImpl(try StringArray(arrowData))) + } catch let error as ArrowError { + return .failure(error) + } catch { + return .failure(.unknownError("\(error)")) + } +} + +private func makeDateHolder( + _ field: ArrowField, + buffers: [ArrowBuffer], + nullCount: UInt +) -> Result { + do { + if field.type.id == .date32 { + let arrowData = try ArrowData(field.type, buffers: buffers, nullCount: nullCount) + return .success(ArrowArrayHolderImpl(try Date32Array(arrowData))) + } + + let arrowData = try ArrowData(field.type, buffers: buffers, nullCount: nullCount) + return .success(ArrowArrayHolderImpl(try Date64Array(arrowData))) + } catch let error as ArrowError { + return .failure(error) + } catch { + return .failure(.unknownError("\(error)")) + } +} + +private func makeTimeHolder( + _ field: ArrowField, + buffers: [ArrowBuffer], + nullCount: UInt +) -> Result { + do { + if field.type.id == .time32 { + if let arrowType = field.type as? ArrowTypeTime32 { + let arrowData = try ArrowData(arrowType, buffers: buffers, nullCount: nullCount) + return .success(ArrowArrayHolderImpl(try FixedArray(arrowData))) + } else { + return .failure(.invalid("Incorrect field type for time: \(field.type)")) + } + } + + if let arrowType = field.type as? ArrowTypeTime64 { + let arrowData = try ArrowData(arrowType, buffers: buffers, nullCount: nullCount) + return .success(ArrowArrayHolderImpl(try FixedArray(arrowData))) + } else { + return .failure(.invalid("Incorrect field type for time: \(field.type)")) + } + } catch let error as ArrowError { + return .failure(error) + } catch { + return .failure(.unknownError("\(error)")) + } +} + +private func makeBoolHolder( + _ buffers: [ArrowBuffer], + nullCount: UInt +) -> Result { + do { + let arrowType = ArrowType(ArrowType.ArrowBool) + let arrowData = try ArrowData(arrowType, buffers: buffers, nullCount: nullCount) + return .success(ArrowArrayHolderImpl(try BoolArray(arrowData))) + } catch let error as ArrowError { + return .failure(error) + } catch { + return .failure(.unknownError("\(error)")) + } +} + +private func makeFixedHolder( + _: T.Type, field: ArrowField, buffers: [ArrowBuffer], + nullCount: UInt +) -> Result { + do { + let arrowData = try ArrowData(field.type, buffers: buffers, nullCount: nullCount) + return .success(ArrowArrayHolderImpl(try FixedArray(arrowData))) + } catch let error as ArrowError { + return .failure(error) + } catch { + return .failure(.unknownError("\(error)")) + } +} + +func makeStructHolder( + _ field: ArrowField, + buffers: [ArrowBuffer], + nullCount: UInt, + children: [ArrowData], + rbLength: UInt +) -> Result { + do { + let arrowData = try ArrowData( + field.type, + buffers: buffers, children: children, + nullCount: nullCount, length: rbLength) + return .success(ArrowArrayHolderImpl(try StructArray(arrowData))) + } catch let error as ArrowError { + return .failure(error) + } catch { + return .failure(.unknownError("\(error)")) + } +} + +func makeArrayHolder( + _ field: org_apache_arrow_flatbuf_Field, + buffers: [ArrowBuffer], + nullCount: UInt, + children: [ArrowData]?, + rbLength: UInt +) -> Result { + let arrowField = fromProto(field: field) + return makeArrayHolder( + arrowField, buffers: buffers, nullCount: nullCount, children: children, rbLength: rbLength) +} + +func makeArrayHolder( // swiftlint:disable:this cyclomatic_complexity + _ field: ArrowField, + buffers: [ArrowBuffer], + nullCount: UInt, + children: [ArrowData]?, + rbLength: UInt +) -> Result { + let typeId = field.type.id + switch typeId { + case .int8: + return makeFixedHolder(Int8.self, field: field, buffers: buffers, nullCount: nullCount) + case .uint8: + return makeFixedHolder(UInt8.self, field: field, buffers: buffers, nullCount: nullCount) + case .int16: + return makeFixedHolder(Int16.self, field: field, buffers: buffers, nullCount: nullCount) + case .uint16: + return makeFixedHolder(UInt16.self, field: field, buffers: buffers, nullCount: nullCount) + case .int32: + return makeFixedHolder(Int32.self, field: field, buffers: buffers, nullCount: nullCount) + case .uint32: + return makeFixedHolder(UInt32.self, field: field, buffers: buffers, nullCount: nullCount) + case .int64: + return makeFixedHolder(Int64.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 .float: + return makeFixedHolder(Float.self, field: field, buffers: buffers, nullCount: nullCount) + case .double: + return makeFixedHolder(Double.self, field: field, buffers: buffers, nullCount: nullCount) + case .string: + return makeStringHolder(buffers, nullCount: nullCount) + case .binary: + return makeBinaryHolder(buffers, nullCount: nullCount) + case .date32, .date64: + return makeDateHolder(field, buffers: buffers, nullCount: nullCount) + case .time32, .time64: + return makeTimeHolder(field, buffers: buffers, nullCount: nullCount) + case .strct: + return makeStructHolder( + field, buffers: buffers, nullCount: nullCount, children: children!, rbLength: rbLength) + default: + return .failure(.unknownType("Type \(typeId) currently not supported")) + } +} + +func makeBuffer( + _ buffer: org_apache_arrow_flatbuf_Buffer, fileData: Data, + length: UInt, messageOffset: Int64 +) -> ArrowBuffer { + let startOffset = messageOffset + buffer.offset + let endOffset = startOffset + buffer.length + let bufferData = [UInt8](fileData[startOffset.. Bool { + switch type { + case .int, .bool, .floatingpoint, .date, .time: + return true + default: + return false + } +} + +func isNestedType(_ type: org_apache_arrow_flatbuf_Type_) -> Bool { + switch type { + case .struct_: + return true + default: + return false + } +} + +func findArrowType( // swiftlint:disable:this cyclomatic_complexity function_body_length + _ field: org_apache_arrow_flatbuf_Field +) -> ArrowType { + let type = field.typeType + switch type { + case .int: + let intType = field.type(type: org_apache_arrow_flatbuf_Int.self)! + let bitWidth = intType.bitWidth + if bitWidth == 8 { + return ArrowType(intType.isSigned ? ArrowType.ArrowInt8 : ArrowType.ArrowUInt8) + } + if bitWidth == 16 { + return ArrowType(intType.isSigned ? ArrowType.ArrowInt16 : ArrowType.ArrowUInt16) + } + if bitWidth == 32 { + return ArrowType(intType.isSigned ? ArrowType.ArrowInt32 : ArrowType.ArrowUInt32) + } + if bitWidth == 64 { + return ArrowType(intType.isSigned ? ArrowType.ArrowInt64 : ArrowType.ArrowUInt64) + } + return ArrowType(ArrowType.ArrowUnknown) + case .bool: + return ArrowType(ArrowType.ArrowBool) + case .floatingpoint: + let floatType = field.type(type: org_apache_arrow_flatbuf_FloatingPoint.self)! + switch floatType.precision { + case .single: + return ArrowType(ArrowType.ArrowFloat) + case .double: + return ArrowType(ArrowType.ArrowDouble) + default: + return ArrowType(ArrowType.ArrowUnknown) + } + case .utf8: + return ArrowType(ArrowType.ArrowString) + case .binary: + return ArrowType(ArrowType.ArrowBinary) + case .date: + let dateType = field.type(type: org_apache_arrow_flatbuf_Date.self)! + if dateType.unit == .day { + return ArrowType(ArrowType.ArrowDate32) + } + + return ArrowType(ArrowType.ArrowDate64) + case .time: + let timeType = field.type(type: org_apache_arrow_flatbuf_Time.self)! + if timeType.unit == .second || timeType.unit == .millisecond { + return ArrowTypeTime32(timeType.unit == .second ? .seconds : .milliseconds) + } + + return ArrowTypeTime64(timeType.unit == .microsecond ? .microseconds : .nanoseconds) + case .struct_: + _ = field.type(type: org_apache_arrow_flatbuf_Struct_.self)! + var fields = [ArrowField]() + for index in 0..= recordBatch.buffersCount { + throw ArrowError.outOfBounds(index: Int64(index)) + } +} + +func validateFileData(_ data: Data) -> Bool { + let markerLength = FILEMARKER.utf8.count + let startString = String(decoding: data[.. ArrowField { + return self.fields[index] + } + + public func fieldIndex(_ name: String) -> Int? { + return self.fieldLookup[name] + } + + public class Builder { + private var fields: [ArrowField] = [] + + public init() {} + + @discardableResult + public func addField(_ field: ArrowField) -> Builder { + fields.append(field) + return self + } + + @discardableResult + public func addField(_ name: String, type: ArrowType, isNullable: Bool) -> Builder { + fields.append(ArrowField(name, type: type, isNullable: isNullable)) + return self + } + + public func finish() -> ArrowSchema { + return ArrowSchema(fields) + } + } +} diff --git a/Sources/SparkConnect/ArrowTable.swift b/Sources/SparkConnect/ArrowTable.swift new file mode 100644 index 0000000..bcd3170 --- /dev/null +++ b/Sources/SparkConnect/ArrowTable.swift @@ -0,0 +1,203 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +import Foundation + +public class ArrowColumn { + public let field: ArrowField + fileprivate let dataHolder: ChunkedArrayHolder + public var type: ArrowType { return self.dataHolder.type } + public var length: UInt { return self.dataHolder.length } + public var nullCount: UInt { return self.dataHolder.nullCount } + + public func data() -> ChunkedArray { + return (self.dataHolder.holder as! ChunkedArray) // swiftlint:disable:this force_cast + } + + public var name: String { return field.name } + public init(_ field: ArrowField, chunked: ChunkedArrayHolder) { + self.field = field + self.dataHolder = chunked + } +} + +public class ArrowTable { + public let schema: ArrowSchema + public var columnCount: UInt { return 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 + } + + public static func from(recordBatches: [RecordBatch]) -> Result { + if recordBatches.isEmpty { + return .failure(.arrayHasNoElements) + } + + var holders = [[ArrowArrayHolder]]() + let schema = recordBatches[0].schema + for recordBatch in recordBatches { + for index in 0.. Result { + do { + return .success(try holders[0].getArrowColumn(field, holders)) + } catch { + return .failure(.runtimeError("\(error)")) + } + } + + public class Builder { + let schemaBuilder = ArrowSchema.Builder() + var columns = [ArrowColumn]() + + public init() {} + + @discardableResult + public func addColumn(_ fieldName: String, arrowArray: ArrowArray) throws -> Builder { + return self.addColumn(fieldName, chunked: try ChunkedArray([arrowArray])) + } + + @discardableResult + public func addColumn(_ fieldName: String, chunked: ChunkedArray) -> Builder { + let field = ArrowField(fieldName, type: chunked.type, isNullable: chunked.nullCount != 0) + self.schemaBuilder.addField(field) + self.columns.append(ArrowColumn(field, chunked: ChunkedArrayHolder(chunked))) + return self + } + + @discardableResult + public func addColumn(_ field: ArrowField, arrowArray: 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) + self.columns.append(ArrowColumn(field, chunked: ChunkedArrayHolder(chunked))) + return self + } + + @discardableResult + public func addColumn(_ column: ArrowColumn) -> Builder { + self.schemaBuilder.addField(column.field) + self.columns.append(column) + return self + } + + public func finish() -> ArrowTable { + return ArrowTable(self.schemaBuilder.finish(), columns: self.columns) + } + } +} + +public class RecordBatch { + public let schema: ArrowSchema + public var columnCount: UInt { return UInt(self.columns.count) } + public let columns: [ArrowArrayHolder] + public let length: UInt + public init(_ schema: ArrowSchema, columns: [ArrowArrayHolder]) { + self.schema = schema + self.columns = columns + self.length = columns[0].length + } + + public class Builder { + let schemaBuilder = ArrowSchema.Builder() + var columns = [ArrowArrayHolder]() + + public init() {} + + @discardableResult + public func addColumn(_ fieldName: String, arrowArray: ArrowArrayHolder) -> Builder { + let field = ArrowField( + fieldName, type: arrowArray.type, isNullable: arrowArray.nullCount != 0) + self.schemaBuilder.addField(field) + self.columns.append(arrowArray) + return self + } + + @discardableResult + public func addColumn(_ field: ArrowField, arrowArray: ArrowArrayHolder) -> 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 { // swiftlint:disable:this for_where + return .failure(.runtimeError("Columns have different sizes")) + } + } + } + return .success(RecordBatch(self.schemaBuilder.finish(), columns: self.columns)) + } + } + + public func data(for columnIndex: Int) -> ArrowArray { + let arrayHolder = column(columnIndex) + return (arrayHolder.array as! ArrowArray) // swiftlint:disable:this force_cast + } + + public func anyData(for columnIndex: Int) -> AnyArray { + let arrayHolder = column(columnIndex) + return arrayHolder.array + } + + public func column(_ index: Int) -> ArrowArrayHolder { + return self.columns[index] + } + + public func column(_ name: String) -> ArrowArrayHolder? { + if let index = self.schema.fieldIndex(name) { + return self.columns[index] + } + + return nil + } +} diff --git a/Sources/SparkConnect/ArrowType.swift b/Sources/SparkConnect/ArrowType.swift new file mode 100644 index 0000000..5272779 --- /dev/null +++ b/Sources/SparkConnect/ArrowType.swift @@ -0,0 +1,406 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +import Foundation + +public typealias Time32 = Int32 +public typealias Time64 = Int64 +public typealias Date32 = Int32 +public typealias Date64 = Int64 + +func FlatBuffersVersion_23_1_4() { // swiftlint:disable:this identifier_name +} + +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 enum ArrowTypeId: Sendable { + case binary + case boolean + case date32 + case date64 + case dateType + case decimal128 + case decimal256 + case dictionary + case double + case fixedSizeBinary + case fixedWidthType + case float + // case HalfFloatType + case int16 + case int32 + case int64 + case int8 + case integer + case intervalUnit + case list + case nested + case null + case number + case string + case strct + case time32 + case time64 + case time + case uint16 + case uint32 + case uint64 + case uint8 + case union + case unknown +} + +public enum ArrowTime32Unit { + case seconds + case milliseconds +} + +public enum ArrowTime64Unit { + case microseconds + case nanoseconds +} + +public class ArrowTypeTime32: ArrowType { + let unit: ArrowTime32Unit + public init(_ unit: ArrowTime32Unit) { + self.unit = unit + super.init(ArrowType.ArrowTime32) + } + + public override var cDataFormatId: String { + get throws { + switch self.unit { + case .milliseconds: + return "ttm" + case .seconds: + return "tts" + } + } + } +} + +public class ArrowTypeTime64: ArrowType { + let unit: ArrowTime64Unit + public init(_ unit: ArrowTime64Unit) { + self.unit = unit + super.init(ArrowType.ArrowTime64) + } + + public override var cDataFormatId: String { + get throws { + switch self.unit { + case .microseconds: + return "ttu" + case .nanoseconds: + return "ttn" + } + } + } +} + +public class ArrowNestedType: ArrowType { + let fields: [ArrowField] + public init(_ info: ArrowType.Info, fields: [ArrowField]) { + self.fields = fields + super.init(info) + } +} + +public class ArrowType { + public private(set) var info: ArrowType.Info + public static let ArrowInt8 = Info.primitiveInfo(ArrowTypeId.int8) + public static let ArrowInt16 = Info.primitiveInfo(ArrowTypeId.int16) + public static let ArrowInt32 = Info.primitiveInfo(ArrowTypeId.int32) + public static let ArrowInt64 = Info.primitiveInfo(ArrowTypeId.int64) + public static let ArrowUInt8 = Info.primitiveInfo(ArrowTypeId.uint8) + public static let ArrowUInt16 = Info.primitiveInfo(ArrowTypeId.uint16) + public static let ArrowUInt32 = Info.primitiveInfo(ArrowTypeId.uint32) + public static let ArrowUInt64 = Info.primitiveInfo(ArrowTypeId.uint64) + public static let ArrowFloat = Info.primitiveInfo(ArrowTypeId.float) + public static let ArrowDouble = Info.primitiveInfo(ArrowTypeId.double) + public static let ArrowUnknown = Info.primitiveInfo(ArrowTypeId.unknown) + public static let ArrowString = Info.variableInfo(ArrowTypeId.string) + public static let ArrowBool = Info.primitiveInfo(ArrowTypeId.boolean) + public static let ArrowDate32 = Info.primitiveInfo(ArrowTypeId.date32) + public static let ArrowDate64 = Info.primitiveInfo(ArrowTypeId.date64) + public static let ArrowBinary = Info.variableInfo(ArrowTypeId.binary) + public static let ArrowTime32 = Info.timeInfo(ArrowTypeId.time32) + public static let ArrowTime64 = Info.timeInfo(ArrowTypeId.time64) + public static let ArrowStruct = Info.complexInfo(ArrowTypeId.strct) + + public init(_ info: ArrowType.Info) { + self.info = info + } + + public var id: ArrowTypeId { + switch self.info { + case .primitiveInfo(let id): + return id + case .timeInfo(let id): + return id + case .variableInfo(let id): + return id + case .complexInfo(let id): + return id + } + } + + public enum Info: Sendable { + case primitiveInfo(ArrowTypeId) + case variableInfo(ArrowTypeId) + case timeInfo(ArrowTypeId) + case complexInfo(ArrowTypeId) + } + + public static func infoForType( // swiftlint:disable:this cyclomatic_complexity + _ type: Any.Type + ) -> ArrowType.Info { + if type == String.self { + return ArrowType.ArrowString + } else if type == Date.self { + return ArrowType.ArrowDate64 + } else if type == Bool.self { + return ArrowType.ArrowBool + } else if type == Data.self { + return ArrowType.ArrowBinary + } else if type == Int8.self { + return ArrowType.ArrowInt8 + } else if type == Int16.self { + return ArrowType.ArrowInt16 + } else if type == Int32.self { + return ArrowType.ArrowInt32 + } else if type == Int64.self { + return ArrowType.ArrowInt64 + } else if type == UInt8.self { + return ArrowType.ArrowUInt8 + } else if type == UInt16.self { + return ArrowType.ArrowUInt16 + } else if type == UInt32.self { + return ArrowType.ArrowUInt32 + } else if type == UInt64.self { + return ArrowType.ArrowUInt64 + } else if type == Float.self { + return ArrowType.ArrowFloat + } else if type == Double.self { + return ArrowType.ArrowDouble + } else { + return ArrowType.ArrowUnknown + } + } + + public static func infoForNumericType(_ type: T.Type) -> ArrowType.Info { + if type == Int8.self { + return ArrowType.ArrowInt8 + } else if type == Int16.self { + return ArrowType.ArrowInt16 + } else if type == Int32.self { + return ArrowType.ArrowInt32 + } else if type == Int64.self { + return ArrowType.ArrowInt64 + } else if type == UInt8.self { + return ArrowType.ArrowUInt8 + } else if type == UInt16.self { + return ArrowType.ArrowUInt16 + } else if type == UInt32.self { + return ArrowType.ArrowUInt32 + } else if type == UInt64.self { + return ArrowType.ArrowUInt64 + } else if type == Float.self { + return ArrowType.ArrowFloat + } else if type == Double.self { + return ArrowType.ArrowDouble + } else { + return ArrowType.ArrowUnknown + } + } + + public func getStride( // swiftlint:disable:this cyclomatic_complexity + ) -> Int + { + switch self.id { + case .int8: + return MemoryLayout.stride + case .int16: + return MemoryLayout.stride + case .int32: + return MemoryLayout.stride + case .int64: + return MemoryLayout.stride + case .uint8: + return MemoryLayout.stride + case .uint16: + return MemoryLayout.stride + case .uint32: + return MemoryLayout.stride + case .uint64: + return MemoryLayout.stride + case .float: + return MemoryLayout.stride + case .double: + return MemoryLayout.stride + case .boolean: + return MemoryLayout.stride + case .date32: + return MemoryLayout.stride + case .date64: + return MemoryLayout.stride + case .time32: + return MemoryLayout.stride + case .time64: + return MemoryLayout.stride + case .binary: + return MemoryLayout.stride + case .string: + return MemoryLayout.stride + case .strct: + return 0 + default: + fatalError("Stride requested for unknown type: \(self)") + } + } + + public var cDataFormatId: String { + get throws { + switch self.id { + case ArrowTypeId.int8: + return "c" + case ArrowTypeId.int16: + return "s" + case ArrowTypeId.int32: + return "i" + case ArrowTypeId.int64: + return "l" + case ArrowTypeId.uint8: + return "C" + case ArrowTypeId.uint16: + return "S" + case ArrowTypeId.uint32: + return "I" + case ArrowTypeId.uint64: + return "L" + case ArrowTypeId.float: + return "f" + case ArrowTypeId.double: + return "g" + case ArrowTypeId.boolean: + return "b" + case ArrowTypeId.date32: + return "tdD" + case ArrowTypeId.date64: + return "tdm" + case ArrowTypeId.time32: + if let time32 = self as? ArrowTypeTime32 { + return try time32.cDataFormatId + } + return "tts" + case ArrowTypeId.time64: + if let time64 = self as? ArrowTypeTime64 { + return try time64.cDataFormatId + } + return "ttu" + case ArrowTypeId.binary: + return "z" + case ArrowTypeId.string: + return "u" + default: + throw ArrowError.notImplemented + } + } + } + + public static func fromCDataFormatId( // swiftlint:disable:this cyclomatic_complexity + _ from: String + ) throws -> ArrowType { + if from == "c" { + return ArrowType(ArrowType.ArrowInt8) + } else if from == "s" { + return ArrowType(ArrowType.ArrowInt16) + } else if from == "i" { + return ArrowType(ArrowType.ArrowInt32) + } else if from == "l" { + return ArrowType(ArrowType.ArrowInt64) + } else if from == "C" { + return ArrowType(ArrowType.ArrowUInt8) + } else if from == "S" { + return ArrowType(ArrowType.ArrowUInt16) + } else if from == "I" { + return ArrowType(ArrowType.ArrowUInt32) + } else if from == "L" { + return ArrowType(ArrowType.ArrowUInt64) + } else if from == "f" { + return ArrowType(ArrowType.ArrowFloat) + } else if from == "g" { + return ArrowType(ArrowType.ArrowDouble) + } else if from == "b" { + return ArrowType(ArrowType.ArrowBool) + } else if from == "tdD" { + return ArrowType(ArrowType.ArrowDate32) + } else if from == "tdm" { + return ArrowType(ArrowType.ArrowDate64) + } else if from == "tts" { + return ArrowTypeTime32(.seconds) + } else if from == "ttm" { + return ArrowTypeTime32(.milliseconds) + } else if from == "ttu" { + return ArrowTypeTime64(.microseconds) + } else if from == "ttn" { + return ArrowTypeTime64(.nanoseconds) + } else if from == "z" { + return ArrowType(ArrowType.ArrowBinary) + } else if from == "u" { + return ArrowType(ArrowType.ArrowString) + } + + throw ArrowError.notImplemented + } +} + +extension ArrowType.Info: Equatable { + public static func == (lhs: ArrowType.Info, rhs: ArrowType.Info) -> Bool { + switch (lhs, rhs) { + case (.primitiveInfo(let lhsId), .primitiveInfo(let rhsId)): + return lhsId == rhsId + case (.variableInfo(let lhsId), .variableInfo(let rhsId)): + return lhsId == rhsId + case (.timeInfo(let lhsId), .timeInfo(let rhsId)): + return lhsId == rhsId + default: + return false + } + } +} + +func getBytesFor(_ data: T) -> Data? { + if let temp = data as? String { + return temp.data(using: .utf8) + } else if T.self == Data.self { + return data as? Data + } else { + return nil + } +} +// swiftlint:disable:this file_length diff --git a/Sources/SparkConnect/ArrowWriter.swift b/Sources/SparkConnect/ArrowWriter.swift new file mode 100644 index 0000000..84bc955 --- /dev/null +++ b/Sources/SparkConnect/ArrowWriter.swift @@ -0,0 +1,364 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +import FlatBuffers +import Foundation + +public protocol DataWriter { + var count: Int { get } + func append(_ data: Data) +} + +public class ArrowWriter { // swiftlint:disable:this type_body_length + public class InMemDataWriter: DataWriter { + public private(set) var data: Data + public var count: Int { return 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 { return currentSize } + public init(_ handle: FileHandle) { + self.handle = handle + } + + public func append(_ data: Data) { + self.handle.write(data) + self.currentSize += data.count + } + } + + public class Info { + public let type: org_apache_arrow_flatbuf_MessageHeader + public let schema: ArrowSchema + public let batches: [RecordBatch] + public init( + _ type: org_apache_arrow_flatbuf_MessageHeader, schema: ArrowSchema, batches: [RecordBatch] + ) { + self.type = type + self.schema = schema + self.batches = batches + } + + public convenience init(_ type: org_apache_arrow_flatbuf_MessageHeader, schema: ArrowSchema) { + self.init(type, schema: schema, batches: [RecordBatch]()) + } + } + + public init() {} + + private func writeField(_ fbb: inout FlatBufferBuilder, field: ArrowField) -> Result< + Offset, ArrowError + > { + let nameOffset = fbb.create(string: field.name) + let fieldTypeOffsetResult = toFBType(&fbb, arrowType: field.type) + let startOffset = org_apache_arrow_flatbuf_Field.startField(&fbb) + org_apache_arrow_flatbuf_Field.add(name: nameOffset, &fbb) + org_apache_arrow_flatbuf_Field.add(nullable: field.isNullable, &fbb) + switch toFBTypeEnum(field.type) { + case .success(let type): + org_apache_arrow_flatbuf_Field.add(typeType: type, &fbb) + case .failure(let error): + return .failure(error) + } + + switch fieldTypeOffsetResult { + case .success(let offset): + org_apache_arrow_flatbuf_Field.add(type: offset, &fbb) + return .success(org_apache_arrow_flatbuf_Field.endField(&fbb, start: startOffset)) + case .failure(let error): + return .failure(error) + } + } + + private func writeSchema(_ fbb: inout FlatBufferBuilder, schema: ArrowSchema) -> Result< + Offset, ArrowError + > { + 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 = + org_apache_arrow_flatbuf_Schema.createSchema( + &fbb, + endianness: .little, + fieldsVectorOffset: fieldsOffset) + return .success(schemaOffset) + + } + + private func writeRecordBatches( + _ writer: inout DataWriter, + batches: [RecordBatch] + ) -> Result<[org_apache_arrow_flatbuf_Block], ArrowError> { + var rbBlocks = [org_apache_arrow_flatbuf_Block]() + + for batch in batches { + let startIndex = writer.count + switch writeRecordBatch(batch: batch) { + case .success(let rbResult): + withUnsafeBytes(of: rbResult.1.o.littleEndian) { writer.append(Data($0)) } + writer.append(rbResult.0) + switch writeRecordBatchData(&writer, batch: batch) { + case .success: + rbBlocks.append( + org_apache_arrow_flatbuf_Block( + offset: Int64(startIndex), + metaDataLength: Int32(0), + bodyLength: Int64(rbResult.1.o))) + case .failure(let error): + return .failure(error) + } + case .failure(let error): + return .failure(error) + } + } + + return .success(rbBlocks) + } + + private func writeRecordBatch(batch: RecordBatch) -> Result<(Data, Offset), ArrowError> { + let schema = batch.schema + var fbb = FlatBufferBuilder() + + // write out field nodes + var fieldNodeOffsets = [Offset]() + fbb.startVector( + schema.fields.count, elementSize: MemoryLayout.size) + for index in (0.. Result< + Bool, ArrowError + > { + for index in 0.. Result { + var fbb: FlatBufferBuilder = FlatBufferBuilder() + 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 = org_apache_arrow_flatbuf_Footer.startFooter(&fbb) + org_apache_arrow_flatbuf_Footer.add(schema: schemaOffset, &fbb) + org_apache_arrow_flatbuf_Footer.addVectorOf(recordBatches: rbBlkEnd, &fbb) + let footerOffset = org_apache_arrow_flatbuf_Footer.endFooter(&fbb, start: footerStartOffset) + fbb.finish(offset: footerOffset) + case .failure(let error): + return .failure(error) + } + + return .success(fbb.data) + } + + private func writeStream(_ writer: inout DataWriter, info: ArrowWriter.Info) -> Result< + Bool, ArrowError + > { + var fbb: FlatBufferBuilder = FlatBufferBuilder() + switch writeSchema(&fbb, schema: info.schema) { + case .success(let schemaOffset): + fbb.finish(offset: schemaOffset) + writer.append(fbb.data) + 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 toStream(_ info: ArrowWriter.Info) -> Result { + var writer: any DataWriter = InMemDataWriter() + switch writeStream(&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)")) + } + + let fileHandle = FileHandle(forUpdatingAtPath: fileName.path)! + defer { fileHandle.closeFile() } + + var markerData = FILEMARKER.data(using: .utf8)! + addPadForAlignment(&markerData) + + var writer: any DataWriter = FileDataWriter(fileHandle) + writer.append(FILEMARKER.data(using: .utf8)!) + switch writeStream(&writer, info: info) { + case .success: + writer.append(FILEMARKER.data(using: .utf8)!) + case .failure(let error): + return .failure(error) + } + + return .success(true) + } + + public func toMessage(_ batch: RecordBatch) -> Result<[Data], ArrowError> { + var writer: any DataWriter = InMemDataWriter() + switch writeRecordBatch(batch: batch) { + case .success(let message): + writer.append(message.0) + addPadForAlignment(&writer) + var dataWriter: any DataWriter = InMemDataWriter() + switch writeRecordBatchData(&dataWriter, batch: batch) { + case .success: + return .success([ + (writer as! InMemDataWriter).data, // swiftlint:disable:this force_cast + (dataWriter as! InMemDataWriter).data, // swiftlint:disable:this force_cast + ]) + 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 = org_apache_arrow_flatbuf_Message.startMessage(&fbb) + org_apache_arrow_flatbuf_Message.add(bodyLength: Int64(0), &fbb) + org_apache_arrow_flatbuf_Message.add(headerType: .schema, &fbb) + org_apache_arrow_flatbuf_Message.add(header: Offset(offset: UOffset(schemaSize)), &fbb) + org_apache_arrow_flatbuf_Message.add(version: .max, &fbb) + let messageOffset = org_apache_arrow_flatbuf_Message.endMessage(&fbb, start: startMessage) + fbb.finish(offset: messageOffset) + return .success(fbb.data) + } +} diff --git a/Sources/SparkConnect/ArrowWriterHelper.swift b/Sources/SparkConnect/ArrowWriterHelper.swift new file mode 100644 index 0000000..7702e48 --- /dev/null +++ b/Sources/SparkConnect/ArrowWriterHelper.swift @@ -0,0 +1,142 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +import FlatBuffers +import Foundation + +extension Data { + func hexEncodedString() -> String { + return map { String(format: "%02hhx", $0) }.joined() + } +} + +func toFBTypeEnum(_ arrowType: ArrowType) -> Result { + let infoType = arrowType.info + if infoType == ArrowType.ArrowInt8 || infoType == ArrowType.ArrowInt16 + || infoType == ArrowType.ArrowInt64 || infoType == ArrowType.ArrowUInt8 + || infoType == ArrowType.ArrowUInt16 || infoType == ArrowType.ArrowUInt32 + || infoType == ArrowType.ArrowUInt64 || infoType == ArrowType.ArrowInt32 + { + return .success(org_apache_arrow_flatbuf_Type_.int) + } else if infoType == ArrowType.ArrowFloat || infoType == ArrowType.ArrowDouble { + return .success(org_apache_arrow_flatbuf_Type_.floatingpoint) + } else if infoType == ArrowType.ArrowString { + return .success(org_apache_arrow_flatbuf_Type_.utf8) + } else if infoType == ArrowType.ArrowBinary { + return .success(org_apache_arrow_flatbuf_Type_.binary) + } else if infoType == ArrowType.ArrowBool { + return .success(org_apache_arrow_flatbuf_Type_.bool) + } else if infoType == ArrowType.ArrowDate32 || infoType == ArrowType.ArrowDate64 { + return .success(org_apache_arrow_flatbuf_Type_.date) + } else if infoType == ArrowType.ArrowTime32 || infoType == ArrowType.ArrowTime64 { + return .success(org_apache_arrow_flatbuf_Type_.time) + } + return .failure(.unknownType("Unable to find flatbuf type for Arrow type: \(infoType)")) +} + +func toFBType( // swiftlint:disable:this cyclomatic_complexity + _ fbb: inout FlatBufferBuilder, + arrowType: ArrowType +) -> Result { + let infoType = arrowType.info + if infoType == ArrowType.ArrowInt8 || infoType == ArrowType.ArrowUInt8 { + return .success( + org_apache_arrow_flatbuf_Int.createInt( + &fbb, bitWidth: 8, isSigned: infoType == ArrowType.ArrowInt8)) + } else if infoType == ArrowType.ArrowInt16 || infoType == ArrowType.ArrowUInt16 { + return .success( + org_apache_arrow_flatbuf_Int.createInt( + &fbb, bitWidth: 16, isSigned: infoType == ArrowType.ArrowInt16)) + } else if infoType == ArrowType.ArrowInt32 || infoType == ArrowType.ArrowUInt32 { + return .success( + org_apache_arrow_flatbuf_Int.createInt( + &fbb, bitWidth: 32, isSigned: infoType == ArrowType.ArrowInt32)) + } else if infoType == ArrowType.ArrowInt64 || infoType == ArrowType.ArrowUInt64 { + return .success( + org_apache_arrow_flatbuf_Int.createInt( + &fbb, bitWidth: 64, isSigned: infoType == ArrowType.ArrowInt64)) + } else if infoType == ArrowType.ArrowFloat { + return .success( + org_apache_arrow_flatbuf_FloatingPoint.createFloatingPoint(&fbb, precision: .single)) + } else if infoType == ArrowType.ArrowDouble { + return .success( + org_apache_arrow_flatbuf_FloatingPoint.createFloatingPoint(&fbb, precision: .double)) + } else if infoType == ArrowType.ArrowString { + return .success( + org_apache_arrow_flatbuf_Utf8.endUtf8( + &fbb, start: org_apache_arrow_flatbuf_Utf8.startUtf8(&fbb))) + } else if infoType == ArrowType.ArrowBinary { + return .success( + org_apache_arrow_flatbuf_Binary.endBinary( + &fbb, start: org_apache_arrow_flatbuf_Binary.startBinary(&fbb))) + } else if infoType == ArrowType.ArrowBool { + return .success( + org_apache_arrow_flatbuf_Bool.endBool( + &fbb, start: org_apache_arrow_flatbuf_Bool.startBool(&fbb))) + } else if infoType == ArrowType.ArrowDate32 { + let startOffset = org_apache_arrow_flatbuf_Date.startDate(&fbb) + org_apache_arrow_flatbuf_Date.add(unit: .day, &fbb) + return .success(org_apache_arrow_flatbuf_Date.endDate(&fbb, start: startOffset)) + } else if infoType == ArrowType.ArrowDate64 { + let startOffset = org_apache_arrow_flatbuf_Date.startDate(&fbb) + org_apache_arrow_flatbuf_Date.add(unit: .millisecond, &fbb) + return .success(org_apache_arrow_flatbuf_Date.endDate(&fbb, start: startOffset)) + } else if infoType == ArrowType.ArrowTime32 { + let startOffset = org_apache_arrow_flatbuf_Time.startTime(&fbb) + if let timeType = arrowType as? ArrowTypeTime32 { + org_apache_arrow_flatbuf_Time.add( + unit: timeType.unit == .seconds ? .second : .millisecond, &fbb) + return .success(org_apache_arrow_flatbuf_Time.endTime(&fbb, start: startOffset)) + } + + return .failure(.invalid("Unable to case to Time32")) + } else if infoType == ArrowType.ArrowTime64 { + let startOffset = org_apache_arrow_flatbuf_Time.startTime(&fbb) + if let timeType = arrowType as? ArrowTypeTime64 { + org_apache_arrow_flatbuf_Time.add( + unit: timeType.unit == .microseconds ? .microsecond : .nanosecond, &fbb) + return .success(org_apache_arrow_flatbuf_Time.endTime(&fbb, start: startOffset)) + } + + return .failure(.invalid("Unable to case to Time64")) + } + + return .failure(.unknownType("Unable to add flatbuf type for Arrow type: \(infoType)")) +} + +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/SparkConnect/BitUtility.swift b/Sources/SparkConnect/BitUtility.swift new file mode 100644 index 0000000..f461d65 --- /dev/null +++ b/Sources/SparkConnect/BitUtility.swift @@ -0,0 +1,40 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +import 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/SparkConnect/ChunkedArray.swift b/Sources/SparkConnect/ChunkedArray.swift new file mode 100644 index 0000000..859ff2e --- /dev/null +++ b/Sources/SparkConnect/ChunkedArray.swift @@ -0,0 +1,149 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +import Foundation + +public protocol AnyArray { + var arrowData: ArrowData { get } + func asAny(_ index: UInt) -> Any? + var length: UInt { get } +} + +public protocol AsString { + func asString(_ index: UInt) -> String +} + +public class ChunkedArrayHolder { + public let type: ArrowType + public let length: UInt + public let nullCount: UInt + public let holder: Any + + public let getBufferData: () -> Result<[Data], ArrowError> + public let getBufferDataSizes: () -> Result<[Int], ArrowError> + public init(_ chunked: ChunkedArray) { // swiftlint:disable:this cyclomatic_complexity + self.holder = chunked + self.length = chunked.length + self.type = chunked.type + self.nullCount = chunked.nullCount + self.getBufferData = { () -> Result<[Data], ArrowError> in + var bufferData = [Data]() + var numBuffers = 2 + switch toFBTypeEnum(chunked.type) { + case .success(let fbType): + if !isFixedPrimitive(fbType) { + numBuffers = 3 + } + case .failure(let error): + return .failure(error) + } + + for _ in 0.. Result<[Int], ArrowError> in + var bufferDataSizes = [Int]() + var numBuffers = 2 + + switch toFBTypeEnum(chunked.type) { + case .success(let fbType): + if !isFixedPrimitive(fbType) { + numBuffers = 3 + } + case .failure(let error): + return .failure(error) + } + + for _ in 0..: AsString { + public let arrays: [ArrowArray] + public let type: ArrowType + public let nullCount: UInt + public let length: UInt + public var arrayCount: UInt { return UInt(self.arrays.count) } + + public init(_ arrays: [ArrowArray]) throws { + if arrays.count == 0 { + throw ArrowError.arrayHasNoElements + } + + self.type = arrays[0].arrowData.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 { + if self[index] == nil { + return "" + } + + return "\(self[index]!)" + } +} diff --git a/Sources/SparkConnect/File_generated.swift b/Sources/SparkConnect/File_generated.swift new file mode 100644 index 0000000..e6403cf --- /dev/null +++ b/Sources/SparkConnect/File_generated.swift @@ -0,0 +1,258 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +// automatically generated by the FlatBuffers compiler, do not modify +// swiftlint:disable all +// swiftformat:disable all + +import FlatBuffers + +public struct org_apache_arrow_flatbuf_Block: NativeStruct, Verifiable, FlatbuffersInitializable { + + static func validateVersion() { FlatBuffersVersion_23_1_4() } + + /// Index to the start of the RecordBlock (note this is past the Message header) + private var _offset: Int64 + /// Length of the metadata + private var _metaDataLength: Int32 + private let padding0__: UInt32 = 0 + /// Length of the data (this is aligned so there can be a gap between this and + /// the metadata). + private var _bodyLength: Int64 + + public init(_ bb: ByteBuffer, o: Int32) { + let _accessor = Struct(bb: bb, position: o) + _offset = _accessor.readBuffer(of: Int64.self, at: 0) + _metaDataLength = _accessor.readBuffer(of: Int32.self, at: 8) + _bodyLength = _accessor.readBuffer(of: Int64.self, at: 16) + } + + public init(offset: Int64, metaDataLength: Int32, bodyLength: Int64) { + _offset = offset + _metaDataLength = metaDataLength + _bodyLength = bodyLength + } + + public init() { + _offset = 0 + _metaDataLength = 0 + _bodyLength = 0 + } + + /// Index to the start of the RecordBlock (note this is past the Message header) + public var offset: Int64 { _offset } + /// Length of the metadata + public var metaDataLength: Int32 { _metaDataLength } + /// Length of the data (this is aligned so there can be a gap between this and + /// the metadata). + public var bodyLength: Int64 { _bodyLength } + + public static func verify(_ verifier: inout Verifier, at position: Int, of type: T.Type) throws + where T: Verifiable { + try verifier.inBuffer(position: position, of: org_apache_arrow_flatbuf_Block.self) + } +} + +public struct org_apache_arrow_flatbuf_Block_Mutable: FlatBufferObject { + + static func validateVersion() { FlatBuffersVersion_23_1_4() } + public var __buffer: ByteBuffer! { return _accessor.bb } + private var _accessor: Struct + + public init(_ bb: ByteBuffer, o: Int32) { _accessor = Struct(bb: bb, position: o) } + + public var offset: Int64 { return _accessor.readBuffer(of: Int64.self, at: 0) } + public var metaDataLength: Int32 { return _accessor.readBuffer(of: Int32.self, at: 8) } + public var bodyLength: Int64 { return _accessor.readBuffer(of: Int64.self, at: 16) } +} + +/// ---------------------------------------------------------------------- +/// Arrow File metadata +/// +public struct org_apache_arrow_flatbuf_Footer: FlatBufferObject, Verifiable { + + static func validateVersion() { FlatBuffersVersion_23_1_4() } + public var __buffer: ByteBuffer! { return _accessor.bb } + private var _accessor: Table + + public static func getRootAsFooter(bb: ByteBuffer) -> org_apache_arrow_flatbuf_Footer { + return org_apache_arrow_flatbuf_Footer( + Table( + bb: bb, position: Int32(bb.read(def: UOffset.self, position: bb.reader)) + Int32(bb.reader)) + ) + } + + private init(_ t: Table) { _accessor = t } + public init(_ bb: ByteBuffer, o: Int32) { _accessor = Table(bb: bb, position: o) } + + private enum VTOFFSET: VOffset { + case version = 4 + case schema = 6 + case dictionaries = 8 + case recordBatches = 10 + case customMetadata = 12 + var v: Int32 { Int32(self.rawValue) } + var p: VOffset { self.rawValue } + } + + public var version: org_apache_arrow_flatbuf_MetadataVersion { + let o = _accessor.offset(VTOFFSET.version.v) + return o == 0 + ? .v1 + : org_apache_arrow_flatbuf_MetadataVersion( + rawValue: _accessor.readBuffer(of: Int16.self, at: o)) ?? .v1 + } + public var schema: org_apache_arrow_flatbuf_Schema? { + let o = _accessor.offset(VTOFFSET.schema.v) + return o == 0 + ? nil + : org_apache_arrow_flatbuf_Schema(_accessor.bb, o: _accessor.indirect(o + _accessor.postion)) + } + public var hasDictionaries: Bool { + let o = _accessor.offset(VTOFFSET.dictionaries.v) + return o == 0 ? false : true + } + public var dictionariesCount: Int32 { + let o = _accessor.offset(VTOFFSET.dictionaries.v) + return o == 0 ? 0 : _accessor.vector(count: o) + } + public func dictionaries(at index: Int32) -> org_apache_arrow_flatbuf_Block? { + let o = _accessor.offset(VTOFFSET.dictionaries.v) + return o == 0 + ? nil + : _accessor.directRead( + of: org_apache_arrow_flatbuf_Block.self, offset: _accessor.vector(at: o) + index * 24) + } + public func mutableDictionaries(at index: Int32) -> org_apache_arrow_flatbuf_Block_Mutable? { + let o = _accessor.offset(VTOFFSET.dictionaries.v) + return o == 0 + ? nil + : org_apache_arrow_flatbuf_Block_Mutable( + _accessor.bb, o: _accessor.vector(at: o) + index * 24) + } + public var hasRecordBatches: Bool { + let o = _accessor.offset(VTOFFSET.recordBatches.v) + return o == 0 ? false : true + } + public var recordBatchesCount: Int32 { + let o = _accessor.offset(VTOFFSET.recordBatches.v) + return o == 0 ? 0 : _accessor.vector(count: o) + } + public func recordBatches(at index: Int32) -> org_apache_arrow_flatbuf_Block? { + let o = _accessor.offset(VTOFFSET.recordBatches.v) + return o == 0 + ? nil + : _accessor.directRead( + of: org_apache_arrow_flatbuf_Block.self, offset: _accessor.vector(at: o) + index * 24) + } + public func mutableRecordBatches(at index: Int32) -> org_apache_arrow_flatbuf_Block_Mutable? { + let o = _accessor.offset(VTOFFSET.recordBatches.v) + return o == 0 + ? nil + : org_apache_arrow_flatbuf_Block_Mutable( + _accessor.bb, o: _accessor.vector(at: o) + index * 24) + } + /// User-defined metadata + public var hasCustomMetadata: Bool { + let o = _accessor.offset(VTOFFSET.customMetadata.v) + return o == 0 ? false : true + } + public var customMetadataCount: Int32 { + let o = _accessor.offset(VTOFFSET.customMetadata.v) + return o == 0 ? 0 : _accessor.vector(count: o) + } + public func customMetadata(at index: Int32) -> org_apache_arrow_flatbuf_KeyValue? { + let o = _accessor.offset(VTOFFSET.customMetadata.v) + return o == 0 + ? nil + : org_apache_arrow_flatbuf_KeyValue( + _accessor.bb, o: _accessor.indirect(_accessor.vector(at: o) + index * 4)) + } + public static func startFooter(_ fbb: inout FlatBufferBuilder) -> UOffset { + fbb.startTable(with: 5) + } + public static func add( + version: org_apache_arrow_flatbuf_MetadataVersion, _ fbb: inout FlatBufferBuilder + ) { fbb.add(element: version.rawValue, def: 0, at: VTOFFSET.version.p) } + public static func add(schema: Offset, _ fbb: inout FlatBufferBuilder) { + fbb.add(offset: schema, at: VTOFFSET.schema.p) + } + public static func addVectorOf(dictionaries: Offset, _ fbb: inout FlatBufferBuilder) { + fbb.add(offset: dictionaries, at: VTOFFSET.dictionaries.p) + } + public static func startVectorOfDictionaries(_ size: Int, in builder: inout FlatBufferBuilder) { + builder.startVector( + size * MemoryLayout.size, + elementSize: MemoryLayout.alignment) + } + public static func addVectorOf(recordBatches: Offset, _ fbb: inout FlatBufferBuilder) { + fbb.add(offset: recordBatches, at: VTOFFSET.recordBatches.p) + } + public static func startVectorOfRecordBatches(_ size: Int, in builder: inout FlatBufferBuilder) { + builder.startVector( + size * MemoryLayout.size, + elementSize: MemoryLayout.alignment) + } + public static func addVectorOf(customMetadata: Offset, _ fbb: inout FlatBufferBuilder) { + fbb.add(offset: customMetadata, at: VTOFFSET.customMetadata.p) + } + public static func endFooter(_ fbb: inout FlatBufferBuilder, start: UOffset) -> Offset { + let end = Offset(offset: fbb.endTable(at: start)) + return end + } + public static func createFooter( + _ fbb: inout FlatBufferBuilder, + version: org_apache_arrow_flatbuf_MetadataVersion = .v1, + schemaOffset schema: Offset = Offset(), + dictionariesVectorOffset dictionaries: Offset = Offset(), + recordBatchesVectorOffset recordBatches: Offset = Offset(), + customMetadataVectorOffset customMetadata: Offset = Offset() + ) -> Offset { + let __start = org_apache_arrow_flatbuf_Footer.startFooter(&fbb) + org_apache_arrow_flatbuf_Footer.add(version: version, &fbb) + org_apache_arrow_flatbuf_Footer.add(schema: schema, &fbb) + org_apache_arrow_flatbuf_Footer.addVectorOf(dictionaries: dictionaries, &fbb) + org_apache_arrow_flatbuf_Footer.addVectorOf(recordBatches: recordBatches, &fbb) + org_apache_arrow_flatbuf_Footer.addVectorOf(customMetadata: customMetadata, &fbb) + return org_apache_arrow_flatbuf_Footer.endFooter(&fbb, start: __start) + } + + public static func verify(_ verifier: inout Verifier, at position: Int, of type: T.Type) throws + where T: Verifiable { + var _v = try verifier.visitTable(at: position) + try _v.visit( + field: VTOFFSET.version.p, fieldName: "version", required: false, + type: org_apache_arrow_flatbuf_MetadataVersion.self) + try _v.visit( + field: VTOFFSET.schema.p, fieldName: "schema", required: false, + type: ForwardOffset.self) + try _v.visit( + field: VTOFFSET.dictionaries.p, fieldName: "dictionaries", required: false, + type: ForwardOffset> + .self) + try _v.visit( + field: VTOFFSET.recordBatches.p, fieldName: "recordBatches", required: false, + type: ForwardOffset> + .self) + try _v.visit( + field: VTOFFSET.customMetadata.p, fieldName: "customMetadata", required: false, + type: ForwardOffset< + Vector, org_apache_arrow_flatbuf_KeyValue> + >.self) + _v.finish() + } +} diff --git a/Sources/SparkConnect/Flight.pb.swift b/Sources/SparkConnect/Flight.pb.swift new file mode 100644 index 0000000..629a85b --- /dev/null +++ b/Sources/SparkConnect/Flight.pb.swift @@ -0,0 +1,1366 @@ +// DO NOT EDIT. +// swift-format-ignore-file +// +// Generated by the Swift generator plugin for the protocol buffer compiler. +// Source: Flight.proto +// +// For information on using the generated types, please see the documentation: +// https://github.com/apple/swift-protobuf/ + +// +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +//

+// http://www.apache.org/licenses/LICENSE-2.0 +//

+// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +import Foundation +import SwiftProtobuf + +// If the compiler emits an error on this type, it is because this file +// was generated by a version of the `protoc` Swift plug-in that is +// incompatible with the version of SwiftProtobuf to which you are linking. +// Please ensure that you are building against the same version of the API +// that was used to generate this file. +fileprivate struct _GeneratedWithProtocGenSwiftVersion: SwiftProtobuf.ProtobufAPIVersionCheck { + struct _2: SwiftProtobuf.ProtobufAPIVersion_2 {} + typealias Version = _2 +} + +/// +/// The result of a cancel operation. +/// +/// This is used by CancelFlightInfoResult.status. +enum Arrow_Flight_Protocol_CancelStatus: SwiftProtobuf.Enum { + typealias RawValue = Int + + /// The cancellation status is unknown. Servers should avoid using + /// this value (send a NOT_FOUND error if the requested query is + /// not known). Clients can retry the request. + case unspecified // = 0 + + /// The cancellation request is complete. Subsequent requests with + /// the same payload may return CANCELLED or a NOT_FOUND error. + case cancelled // = 1 + + /// The cancellation request is in progress. The client may retry + /// the cancellation request. + case cancelling // = 2 + + /// The query is not cancellable. The client should not retry the + /// cancellation request. + case notCancellable // = 3 + case UNRECOGNIZED(Int) + + init() { + self = .unspecified + } + + init?(rawValue: Int) { + switch rawValue { + case 0: self = .unspecified + case 1: self = .cancelled + case 2: self = .cancelling + case 3: self = .notCancellable + default: self = .UNRECOGNIZED(rawValue) + } + } + + var rawValue: Int { + switch self { + case .unspecified: return 0 + case .cancelled: return 1 + case .cancelling: return 2 + case .notCancellable: return 3 + case .UNRECOGNIZED(let i): return i + } + } + +} + +#if swift(>=4.2) + +extension Arrow_Flight_Protocol_CancelStatus: CaseIterable { + // The compiler won't synthesize support with the UNRECOGNIZED case. + static let allCases: [Arrow_Flight_Protocol_CancelStatus] = [ + .unspecified, + .cancelled, + .cancelling, + .notCancellable, + ] +} + +#endif // swift(>=4.2) + +/// +/// The request that a client provides to a server on handshake. +struct Arrow_Flight_Protocol_HandshakeRequest { + // SwiftProtobuf.Message conformance is added in an extension below. See the + // `Message` and `Message+*Additions` files in the SwiftProtobuf library for + // methods supported on all messages. + + /// + /// A defined protocol version + var protocolVersion: UInt64 = 0 + + /// + /// Arbitrary auth/handshake info. + var payload: Data = Data() + + var unknownFields = SwiftProtobuf.UnknownStorage() + + init() {} +} + +struct Arrow_Flight_Protocol_HandshakeResponse { + // SwiftProtobuf.Message conformance is added in an extension below. See the + // `Message` and `Message+*Additions` files in the SwiftProtobuf library for + // methods supported on all messages. + + /// + /// A defined protocol version + var protocolVersion: UInt64 = 0 + + /// + /// Arbitrary auth/handshake info. + var payload: Data = Data() + + var unknownFields = SwiftProtobuf.UnknownStorage() + + init() {} +} + +/// +/// A message for doing simple auth. +struct Arrow_Flight_Protocol_BasicAuth { + // SwiftProtobuf.Message conformance is added in an extension below. See the + // `Message` and `Message+*Additions` files in the SwiftProtobuf library for + // methods supported on all messages. + + var username: String = String() + + var password: String = String() + + var unknownFields = SwiftProtobuf.UnknownStorage() + + init() {} +} + +struct Arrow_Flight_Protocol_Empty { + // SwiftProtobuf.Message conformance is added in an extension below. See the + // `Message` and `Message+*Additions` files in the SwiftProtobuf library for + // methods supported on all messages. + + var unknownFields = SwiftProtobuf.UnknownStorage() + + init() {} +} + +/// +/// Describes an available action, including both the name used for execution +/// along with a short description of the purpose of the action. +struct Arrow_Flight_Protocol_ActionType { + // SwiftProtobuf.Message conformance is added in an extension below. See the + // `Message` and `Message+*Additions` files in the SwiftProtobuf library for + // methods supported on all messages. + + var type: String = String() + + var description_p: String = String() + + var unknownFields = SwiftProtobuf.UnknownStorage() + + init() {} +} + +/// +/// A service specific expression that can be used to return a limited set +/// of available Arrow Flight streams. +struct Arrow_Flight_Protocol_Criteria { + // SwiftProtobuf.Message conformance is added in an extension below. See the + // `Message` and `Message+*Additions` files in the SwiftProtobuf library for + // methods supported on all messages. + + var expression: Data = Data() + + var unknownFields = SwiftProtobuf.UnknownStorage() + + init() {} +} + +/// +/// An opaque action specific for the service. +struct Arrow_Flight_Protocol_Action { + // SwiftProtobuf.Message conformance is added in an extension below. See the + // `Message` and `Message+*Additions` files in the SwiftProtobuf library for + // methods supported on all messages. + + var type: String = String() + + var body: Data = Data() + + var unknownFields = SwiftProtobuf.UnknownStorage() + + init() {} +} + +/// +/// The request of the CancelFlightInfo action. +/// +/// The request should be stored in Action.body. +struct Arrow_Flight_Protocol_CancelFlightInfoRequest { + // SwiftProtobuf.Message conformance is added in an extension below. See the + // `Message` and `Message+*Additions` files in the SwiftProtobuf library for + // methods supported on all messages. + + var info: Arrow_Flight_Protocol_FlightInfo { + get {return _info ?? Arrow_Flight_Protocol_FlightInfo()} + set {_info = newValue} + } + /// Returns true if `info` has been explicitly set. + var hasInfo: Bool {return self._info != nil} + /// Clears the value of `info`. Subsequent reads from it will return its default value. + mutating func clearInfo() {self._info = nil} + + var unknownFields = SwiftProtobuf.UnknownStorage() + + init() {} + + fileprivate var _info: Arrow_Flight_Protocol_FlightInfo? = nil +} + +/// +/// The request of the RenewFlightEndpoint action. +/// +/// The request should be stored in Action.body. +struct Arrow_Flight_Protocol_RenewFlightEndpointRequest { + // SwiftProtobuf.Message conformance is added in an extension below. See the + // `Message` and `Message+*Additions` files in the SwiftProtobuf library for + // methods supported on all messages. + + var endpoint: Arrow_Flight_Protocol_FlightEndpoint { + get {return _endpoint ?? Arrow_Flight_Protocol_FlightEndpoint()} + set {_endpoint = newValue} + } + /// Returns true if `endpoint` has been explicitly set. + var hasEndpoint: Bool {return self._endpoint != nil} + /// Clears the value of `endpoint`. Subsequent reads from it will return its default value. + mutating func clearEndpoint() {self._endpoint = nil} + + var unknownFields = SwiftProtobuf.UnknownStorage() + + init() {} + + fileprivate var _endpoint: Arrow_Flight_Protocol_FlightEndpoint? = nil +} + +/// +/// An opaque result returned after executing an action. +struct Arrow_Flight_Protocol_Result { + // SwiftProtobuf.Message conformance is added in an extension below. See the + // `Message` and `Message+*Additions` files in the SwiftProtobuf library for + // methods supported on all messages. + + var body: Data = Data() + + var unknownFields = SwiftProtobuf.UnknownStorage() + + init() {} +} + +/// +/// The result of the CancelFlightInfo action. +/// +/// The result should be stored in Result.body. +struct Arrow_Flight_Protocol_CancelFlightInfoResult { + // SwiftProtobuf.Message conformance is added in an extension below. See the + // `Message` and `Message+*Additions` files in the SwiftProtobuf library for + // methods supported on all messages. + + var status: Arrow_Flight_Protocol_CancelStatus = .unspecified + + var unknownFields = SwiftProtobuf.UnknownStorage() + + init() {} +} + +/// +/// Wrap the result of a getSchema call +struct Arrow_Flight_Protocol_SchemaResult { + // SwiftProtobuf.Message conformance is added in an extension below. See the + // `Message` and `Message+*Additions` files in the SwiftProtobuf library for + // methods supported on all messages. + + /// The schema of the dataset in its IPC form: + /// 4 bytes - an optional IPC_CONTINUATION_TOKEN prefix + /// 4 bytes - the byte length of the payload + /// a flatbuffer Message whose header is the Schema + var schema: Data = Data() + + var unknownFields = SwiftProtobuf.UnknownStorage() + + init() {} +} + +/// +/// The name or tag for a Flight. May be used as a way to retrieve or generate +/// a flight or be used to expose a set of previously defined flights. +struct Arrow_Flight_Protocol_FlightDescriptor { + // SwiftProtobuf.Message conformance is added in an extension below. See the + // `Message` and `Message+*Additions` files in the SwiftProtobuf library for + // methods supported on all messages. + + var type: Arrow_Flight_Protocol_FlightDescriptor.DescriptorType = .unknown + + /// + /// Opaque value used to express a command. Should only be defined when + /// type = CMD. + var cmd: Data = Data() + + /// + /// List of strings identifying a particular dataset. Should only be defined + /// when type = PATH. + var path: [String] = [] + + var unknownFields = SwiftProtobuf.UnknownStorage() + + /// + /// Describes what type of descriptor is defined. + enum DescriptorType: SwiftProtobuf.Enum { + typealias RawValue = Int + + /// Protobuf pattern, not used. + case unknown // = 0 + + /// + /// A named path that identifies a dataset. A path is composed of a string + /// or list of strings describing a particular dataset. This is conceptually + /// similar to a path inside a filesystem. + case path // = 1 + + /// + /// An opaque command to generate a dataset. + case cmd // = 2 + case UNRECOGNIZED(Int) + + init() { + self = .unknown + } + + init?(rawValue: Int) { + switch rawValue { + case 0: self = .unknown + case 1: self = .path + case 2: self = .cmd + default: self = .UNRECOGNIZED(rawValue) + } + } + + var rawValue: Int { + switch self { + case .unknown: return 0 + case .path: return 1 + case .cmd: return 2 + case .UNRECOGNIZED(let i): return i + } + } + + } + + init() {} +} + +#if swift(>=4.2) + +extension Arrow_Flight_Protocol_FlightDescriptor.DescriptorType: CaseIterable { + // The compiler won't synthesize support with the UNRECOGNIZED case. + static let allCases: [Arrow_Flight_Protocol_FlightDescriptor.DescriptorType] = [ + .unknown, + .path, + .cmd, + ] +} + +#endif // swift(>=4.2) + +/// +/// The access coordinates for retrieval of a dataset. With a FlightInfo, a +/// consumer is able to determine how to retrieve a dataset. +struct Arrow_Flight_Protocol_FlightInfo { + // SwiftProtobuf.Message conformance is added in an extension below. See the + // `Message` and `Message+*Additions` files in the SwiftProtobuf library for + // methods supported on all messages. + + /// The schema of the dataset in its IPC form: + /// 4 bytes - an optional IPC_CONTINUATION_TOKEN prefix + /// 4 bytes - the byte length of the payload + /// a flatbuffer Message whose header is the Schema + var schema: Data = Data() + + /// + /// The descriptor associated with this info. + var flightDescriptor: Arrow_Flight_Protocol_FlightDescriptor { + get {return _flightDescriptor ?? Arrow_Flight_Protocol_FlightDescriptor()} + set {_flightDescriptor = newValue} + } + /// Returns true if `flightDescriptor` has been explicitly set. + var hasFlightDescriptor: Bool {return self._flightDescriptor != nil} + /// Clears the value of `flightDescriptor`. Subsequent reads from it will return its default value. + mutating func clearFlightDescriptor() {self._flightDescriptor = nil} + + /// + /// A list of endpoints associated with the flight. To consume the + /// whole flight, all endpoints (and hence all Tickets) must be + /// consumed. Endpoints can be consumed in any order. + /// + /// In other words, an application can use multiple endpoints to + /// represent partitioned data. + /// + /// If the returned data has an ordering, an application can use + /// "FlightInfo.ordered = true" or should return the all data in a + /// single endpoint. Otherwise, there is no ordering defined on + /// endpoints or the data within. + /// + /// A client can read ordered data by reading data from returned + /// endpoints, in order, from front to back. + /// + /// Note that a client may ignore "FlightInfo.ordered = true". If an + /// ordering is important for an application, an application must + /// choose one of them: + /// + /// * An application requires that all clients must read data in + /// returned endpoints order. + /// * An application must return the all data in a single endpoint. + var endpoint: [Arrow_Flight_Protocol_FlightEndpoint] = [] + + /// Set these to -1 if unknown. + var totalRecords: Int64 = 0 + + var totalBytes: Int64 = 0 + + /// + /// FlightEndpoints are in the same order as the data. + var ordered: Bool = false + + var unknownFields = SwiftProtobuf.UnknownStorage() + + init() {} + + fileprivate var _flightDescriptor: Arrow_Flight_Protocol_FlightDescriptor? = nil +} + +/// +/// A particular stream or split associated with a flight. +struct Arrow_Flight_Protocol_FlightEndpoint { + // SwiftProtobuf.Message conformance is added in an extension below. See the + // `Message` and `Message+*Additions` files in the SwiftProtobuf library for + // methods supported on all messages. + + /// + /// Token used to retrieve this stream. + var ticket: Arrow_Flight_Protocol_Ticket { + get {return _ticket ?? Arrow_Flight_Protocol_Ticket()} + set {_ticket = newValue} + } + /// Returns true if `ticket` has been explicitly set. + var hasTicket: Bool {return self._ticket != nil} + /// Clears the value of `ticket`. Subsequent reads from it will return its default value. + mutating func clearTicket() {self._ticket = nil} + + /// + /// A list of URIs where this ticket can be redeemed via DoGet(). + /// + /// If the list is empty, the expectation is that the ticket can only + /// be redeemed on the current service where the ticket was + /// generated. + /// + /// If the list is not empty, the expectation is that the ticket can + /// be redeemed at any of the locations, and that the data returned + /// will be equivalent. In this case, the ticket may only be redeemed + /// at one of the given locations, and not (necessarily) on the + /// current service. + /// + /// In other words, an application can use multiple locations to + /// represent redundant and/or load balanced services. + var location: [Arrow_Flight_Protocol_Location] = [] + + /// + /// Expiration time of this stream. If present, clients may assume + /// they can retry DoGet requests. Otherwise, it is + /// application-defined whether DoGet requests may be retried. + var expirationTime: SwiftProtobuf.Google_Protobuf_Timestamp { + get {return _expirationTime ?? SwiftProtobuf.Google_Protobuf_Timestamp()} + set {_expirationTime = newValue} + } + /// Returns true if `expirationTime` has been explicitly set. + var hasExpirationTime: Bool {return self._expirationTime != nil} + /// Clears the value of `expirationTime`. Subsequent reads from it will return its default value. + mutating func clearExpirationTime() {self._expirationTime = nil} + + var unknownFields = SwiftProtobuf.UnknownStorage() + + init() {} + + fileprivate var _ticket: Arrow_Flight_Protocol_Ticket? = nil + fileprivate var _expirationTime: SwiftProtobuf.Google_Protobuf_Timestamp? = nil +} + +/// +/// A location where a Flight service will accept retrieval of a particular +/// stream given a ticket. +struct Arrow_Flight_Protocol_Location { + // SwiftProtobuf.Message conformance is added in an extension below. See the + // `Message` and `Message+*Additions` files in the SwiftProtobuf library for + // methods supported on all messages. + + var uri: String = String() + + var unknownFields = SwiftProtobuf.UnknownStorage() + + init() {} +} + +/// +/// An opaque identifier that the service can use to retrieve a particular +/// portion of a stream. +/// +/// Tickets are meant to be single use. It is an error/application-defined +/// behavior to reuse a ticket. +struct Arrow_Flight_Protocol_Ticket { + // SwiftProtobuf.Message conformance is added in an extension below. See the + // `Message` and `Message+*Additions` files in the SwiftProtobuf library for + // methods supported on all messages. + + var ticket: Data = Data() + + var unknownFields = SwiftProtobuf.UnknownStorage() + + init() {} +} + +/// +/// A batch of Arrow data as part of a stream of batches. +struct Arrow_Flight_Protocol_FlightData { + // SwiftProtobuf.Message conformance is added in an extension below. See the + // `Message` and `Message+*Additions` files in the SwiftProtobuf library for + // methods supported on all messages. + + /// + /// The descriptor of the data. This is only relevant when a client is + /// starting a new DoPut stream. + var flightDescriptor: Arrow_Flight_Protocol_FlightDescriptor { + get {return _flightDescriptor ?? Arrow_Flight_Protocol_FlightDescriptor()} + set {_flightDescriptor = newValue} + } + /// Returns true if `flightDescriptor` has been explicitly set. + var hasFlightDescriptor: Bool {return self._flightDescriptor != nil} + /// Clears the value of `flightDescriptor`. Subsequent reads from it will return its default value. + mutating func clearFlightDescriptor() {self._flightDescriptor = nil} + + /// + /// Header for message data as described in Message.fbs::Message. + var dataHeader: Data = Data() + + /// + /// Application-defined metadata. + var appMetadata: Data = Data() + + /// + /// The actual batch of Arrow data. Preferably handled with minimal-copies + /// coming last in the definition to help with sidecar patterns (it is + /// expected that some implementations will fetch this field off the wire + /// with specialized code to avoid extra memory copies). + var dataBody: Data = Data() + + var unknownFields = SwiftProtobuf.UnknownStorage() + + init() {} + + fileprivate var _flightDescriptor: Arrow_Flight_Protocol_FlightDescriptor? = nil +} + +///* +/// The response message associated with the submission of a DoPut. +struct Arrow_Flight_Protocol_PutResult { + // SwiftProtobuf.Message conformance is added in an extension below. See the + // `Message` and `Message+*Additions` files in the SwiftProtobuf library for + // methods supported on all messages. + + var appMetadata: Data = Data() + + var unknownFields = SwiftProtobuf.UnknownStorage() + + init() {} +} + +#if swift(>=5.5) && canImport(_Concurrency) +extension Arrow_Flight_Protocol_CancelStatus: @unchecked Sendable {} +extension Arrow_Flight_Protocol_HandshakeRequest: @unchecked Sendable {} +extension Arrow_Flight_Protocol_HandshakeResponse: @unchecked Sendable {} +extension Arrow_Flight_Protocol_BasicAuth: @unchecked Sendable {} +extension Arrow_Flight_Protocol_Empty: @unchecked Sendable {} +extension Arrow_Flight_Protocol_ActionType: @unchecked Sendable {} +extension Arrow_Flight_Protocol_Criteria: @unchecked Sendable {} +extension Arrow_Flight_Protocol_Action: @unchecked Sendable {} +extension Arrow_Flight_Protocol_CancelFlightInfoRequest: @unchecked Sendable {} +extension Arrow_Flight_Protocol_RenewFlightEndpointRequest: @unchecked Sendable {} +extension Arrow_Flight_Protocol_Result: @unchecked Sendable {} +extension Arrow_Flight_Protocol_CancelFlightInfoResult: @unchecked Sendable {} +extension Arrow_Flight_Protocol_SchemaResult: @unchecked Sendable {} +extension Arrow_Flight_Protocol_FlightDescriptor: @unchecked Sendable {} +extension Arrow_Flight_Protocol_FlightDescriptor.DescriptorType: @unchecked Sendable {} +extension Arrow_Flight_Protocol_FlightInfo: @unchecked Sendable {} +extension Arrow_Flight_Protocol_FlightEndpoint: @unchecked Sendable {} +extension Arrow_Flight_Protocol_Location: @unchecked Sendable {} +extension Arrow_Flight_Protocol_Ticket: @unchecked Sendable {} +extension Arrow_Flight_Protocol_FlightData: @unchecked Sendable {} +extension Arrow_Flight_Protocol_PutResult: @unchecked Sendable {} +#endif // swift(>=5.5) && canImport(_Concurrency) + +// MARK: - Code below here is support for the SwiftProtobuf runtime. + +fileprivate let _protobuf_package = "arrow.flight.protocol" + +extension Arrow_Flight_Protocol_CancelStatus: SwiftProtobuf._ProtoNameProviding { + static let _protobuf_nameMap: SwiftProtobuf._NameMap = [ + 0: .same(proto: "CANCEL_STATUS_UNSPECIFIED"), + 1: .same(proto: "CANCEL_STATUS_CANCELLED"), + 2: .same(proto: "CANCEL_STATUS_CANCELLING"), + 3: .same(proto: "CANCEL_STATUS_NOT_CANCELLABLE"), + ] +} + +extension Arrow_Flight_Protocol_HandshakeRequest: SwiftProtobuf.Message, SwiftProtobuf._MessageImplementationBase, SwiftProtobuf._ProtoNameProviding { + static let protoMessageName: String = _protobuf_package + ".HandshakeRequest" + static let _protobuf_nameMap: SwiftProtobuf._NameMap = [ + 1: .standard(proto: "protocol_version"), + 2: .same(proto: "payload"), + ] + + mutating func decodeMessage(decoder: inout D) throws { + while let fieldNumber = try decoder.nextFieldNumber() { + // The use of inline closures is to circumvent an issue where the compiler + // allocates stack space for every case branch when no optimizations are + // enabled. https://github.com/apple/swift-protobuf/issues/1034 + switch fieldNumber { + case 1: try { try decoder.decodeSingularUInt64Field(value: &self.protocolVersion) }() + case 2: try { try decoder.decodeSingularBytesField(value: &self.payload) }() + default: break + } + } + } + + func traverse(visitor: inout V) throws { + if self.protocolVersion != 0 { + try visitor.visitSingularUInt64Field(value: self.protocolVersion, fieldNumber: 1) + } + if !self.payload.isEmpty { + try visitor.visitSingularBytesField(value: self.payload, fieldNumber: 2) + } + try unknownFields.traverse(visitor: &visitor) + } + + static func ==(lhs: Arrow_Flight_Protocol_HandshakeRequest, rhs: Arrow_Flight_Protocol_HandshakeRequest) -> Bool { + if lhs.protocolVersion != rhs.protocolVersion {return false} + if lhs.payload != rhs.payload {return false} + if lhs.unknownFields != rhs.unknownFields {return false} + return true + } +} + +extension Arrow_Flight_Protocol_HandshakeResponse: SwiftProtobuf.Message, SwiftProtobuf._MessageImplementationBase, SwiftProtobuf._ProtoNameProviding { + static let protoMessageName: String = _protobuf_package + ".HandshakeResponse" + static let _protobuf_nameMap: SwiftProtobuf._NameMap = [ + 1: .standard(proto: "protocol_version"), + 2: .same(proto: "payload"), + ] + + mutating func decodeMessage(decoder: inout D) throws { + while let fieldNumber = try decoder.nextFieldNumber() { + // The use of inline closures is to circumvent an issue where the compiler + // allocates stack space for every case branch when no optimizations are + // enabled. https://github.com/apple/swift-protobuf/issues/1034 + switch fieldNumber { + case 1: try { try decoder.decodeSingularUInt64Field(value: &self.protocolVersion) }() + case 2: try { try decoder.decodeSingularBytesField(value: &self.payload) }() + default: break + } + } + } + + func traverse(visitor: inout V) throws { + if self.protocolVersion != 0 { + try visitor.visitSingularUInt64Field(value: self.protocolVersion, fieldNumber: 1) + } + if !self.payload.isEmpty { + try visitor.visitSingularBytesField(value: self.payload, fieldNumber: 2) + } + try unknownFields.traverse(visitor: &visitor) + } + + static func ==(lhs: Arrow_Flight_Protocol_HandshakeResponse, rhs: Arrow_Flight_Protocol_HandshakeResponse) -> Bool { + if lhs.protocolVersion != rhs.protocolVersion {return false} + if lhs.payload != rhs.payload {return false} + if lhs.unknownFields != rhs.unknownFields {return false} + return true + } +} + +extension Arrow_Flight_Protocol_BasicAuth: SwiftProtobuf.Message, SwiftProtobuf._MessageImplementationBase, SwiftProtobuf._ProtoNameProviding { + static let protoMessageName: String = _protobuf_package + ".BasicAuth" + static let _protobuf_nameMap: SwiftProtobuf._NameMap = [ + 2: .same(proto: "username"), + 3: .same(proto: "password"), + ] + + mutating func decodeMessage(decoder: inout D) throws { + while let fieldNumber = try decoder.nextFieldNumber() { + // The use of inline closures is to circumvent an issue where the compiler + // allocates stack space for every case branch when no optimizations are + // enabled. https://github.com/apple/swift-protobuf/issues/1034 + switch fieldNumber { + case 2: try { try decoder.decodeSingularStringField(value: &self.username) }() + case 3: try { try decoder.decodeSingularStringField(value: &self.password) }() + default: break + } + } + } + + func traverse(visitor: inout V) throws { + if !self.username.isEmpty { + try visitor.visitSingularStringField(value: self.username, fieldNumber: 2) + } + if !self.password.isEmpty { + try visitor.visitSingularStringField(value: self.password, fieldNumber: 3) + } + try unknownFields.traverse(visitor: &visitor) + } + + static func ==(lhs: Arrow_Flight_Protocol_BasicAuth, rhs: Arrow_Flight_Protocol_BasicAuth) -> Bool { + if lhs.username != rhs.username {return false} + if lhs.password != rhs.password {return false} + if lhs.unknownFields != rhs.unknownFields {return false} + return true + } +} + +extension Arrow_Flight_Protocol_Empty: SwiftProtobuf.Message, SwiftProtobuf._MessageImplementationBase, SwiftProtobuf._ProtoNameProviding { + static let protoMessageName: String = _protobuf_package + ".Empty" + static let _protobuf_nameMap = SwiftProtobuf._NameMap() + + mutating func decodeMessage(decoder: inout D) throws { + while let _ = try decoder.nextFieldNumber() { + } + } + + func traverse(visitor: inout V) throws { + try unknownFields.traverse(visitor: &visitor) + } + + static func ==(lhs: Arrow_Flight_Protocol_Empty, rhs: Arrow_Flight_Protocol_Empty) -> Bool { + if lhs.unknownFields != rhs.unknownFields {return false} + return true + } +} + +extension Arrow_Flight_Protocol_ActionType: SwiftProtobuf.Message, SwiftProtobuf._MessageImplementationBase, SwiftProtobuf._ProtoNameProviding { + static let protoMessageName: String = _protobuf_package + ".ActionType" + static let _protobuf_nameMap: SwiftProtobuf._NameMap = [ + 1: .same(proto: "type"), + 2: .same(proto: "description"), + ] + + mutating func decodeMessage(decoder: inout D) throws { + while let fieldNumber = try decoder.nextFieldNumber() { + // The use of inline closures is to circumvent an issue where the compiler + // allocates stack space for every case branch when no optimizations are + // enabled. https://github.com/apple/swift-protobuf/issues/1034 + switch fieldNumber { + case 1: try { try decoder.decodeSingularStringField(value: &self.type) }() + case 2: try { try decoder.decodeSingularStringField(value: &self.description_p) }() + default: break + } + } + } + + func traverse(visitor: inout V) throws { + if !self.type.isEmpty { + try visitor.visitSingularStringField(value: self.type, fieldNumber: 1) + } + if !self.description_p.isEmpty { + try visitor.visitSingularStringField(value: self.description_p, fieldNumber: 2) + } + try unknownFields.traverse(visitor: &visitor) + } + + static func ==(lhs: Arrow_Flight_Protocol_ActionType, rhs: Arrow_Flight_Protocol_ActionType) -> Bool { + if lhs.type != rhs.type {return false} + if lhs.description_p != rhs.description_p {return false} + if lhs.unknownFields != rhs.unknownFields {return false} + return true + } +} + +extension Arrow_Flight_Protocol_Criteria: SwiftProtobuf.Message, SwiftProtobuf._MessageImplementationBase, SwiftProtobuf._ProtoNameProviding { + static let protoMessageName: String = _protobuf_package + ".Criteria" + static let _protobuf_nameMap: SwiftProtobuf._NameMap = [ + 1: .same(proto: "expression"), + ] + + mutating func decodeMessage(decoder: inout D) throws { + while let fieldNumber = try decoder.nextFieldNumber() { + // The use of inline closures is to circumvent an issue where the compiler + // allocates stack space for every case branch when no optimizations are + // enabled. https://github.com/apple/swift-protobuf/issues/1034 + switch fieldNumber { + case 1: try { try decoder.decodeSingularBytesField(value: &self.expression) }() + default: break + } + } + } + + func traverse(visitor: inout V) throws { + if !self.expression.isEmpty { + try visitor.visitSingularBytesField(value: self.expression, fieldNumber: 1) + } + try unknownFields.traverse(visitor: &visitor) + } + + static func ==(lhs: Arrow_Flight_Protocol_Criteria, rhs: Arrow_Flight_Protocol_Criteria) -> Bool { + if lhs.expression != rhs.expression {return false} + if lhs.unknownFields != rhs.unknownFields {return false} + return true + } +} + +extension Arrow_Flight_Protocol_Action: SwiftProtobuf.Message, SwiftProtobuf._MessageImplementationBase, SwiftProtobuf._ProtoNameProviding { + static let protoMessageName: String = _protobuf_package + ".Action" + static let _protobuf_nameMap: SwiftProtobuf._NameMap = [ + 1: .same(proto: "type"), + 2: .same(proto: "body"), + ] + + mutating func decodeMessage(decoder: inout D) throws { + while let fieldNumber = try decoder.nextFieldNumber() { + // The use of inline closures is to circumvent an issue where the compiler + // allocates stack space for every case branch when no optimizations are + // enabled. https://github.com/apple/swift-protobuf/issues/1034 + switch fieldNumber { + case 1: try { try decoder.decodeSingularStringField(value: &self.type) }() + case 2: try { try decoder.decodeSingularBytesField(value: &self.body) }() + default: break + } + } + } + + func traverse(visitor: inout V) throws { + if !self.type.isEmpty { + try visitor.visitSingularStringField(value: self.type, fieldNumber: 1) + } + if !self.body.isEmpty { + try visitor.visitSingularBytesField(value: self.body, fieldNumber: 2) + } + try unknownFields.traverse(visitor: &visitor) + } + + static func ==(lhs: Arrow_Flight_Protocol_Action, rhs: Arrow_Flight_Protocol_Action) -> Bool { + if lhs.type != rhs.type {return false} + if lhs.body != rhs.body {return false} + if lhs.unknownFields != rhs.unknownFields {return false} + return true + } +} + +extension Arrow_Flight_Protocol_CancelFlightInfoRequest: SwiftProtobuf.Message, SwiftProtobuf._MessageImplementationBase, SwiftProtobuf._ProtoNameProviding { + static let protoMessageName: String = _protobuf_package + ".CancelFlightInfoRequest" + static let _protobuf_nameMap: SwiftProtobuf._NameMap = [ + 1: .same(proto: "info"), + ] + + mutating func decodeMessage(decoder: inout D) throws { + while let fieldNumber = try decoder.nextFieldNumber() { + // The use of inline closures is to circumvent an issue where the compiler + // allocates stack space for every case branch when no optimizations are + // enabled. https://github.com/apple/swift-protobuf/issues/1034 + switch fieldNumber { + case 1: try { try decoder.decodeSingularMessageField(value: &self._info) }() + default: break + } + } + } + + func traverse(visitor: inout V) throws { + // The use of inline closures is to circumvent an issue where the compiler + // allocates stack space for every if/case branch local when no optimizations + // are enabled. https://github.com/apple/swift-protobuf/issues/1034 and + // https://github.com/apple/swift-protobuf/issues/1182 + try { if let v = self._info { + try visitor.visitSingularMessageField(value: v, fieldNumber: 1) + } }() + try unknownFields.traverse(visitor: &visitor) + } + + static func ==(lhs: Arrow_Flight_Protocol_CancelFlightInfoRequest, rhs: Arrow_Flight_Protocol_CancelFlightInfoRequest) -> Bool { + if lhs._info != rhs._info {return false} + if lhs.unknownFields != rhs.unknownFields {return false} + return true + } +} + +extension Arrow_Flight_Protocol_RenewFlightEndpointRequest: SwiftProtobuf.Message, SwiftProtobuf._MessageImplementationBase, SwiftProtobuf._ProtoNameProviding { + static let protoMessageName: String = _protobuf_package + ".RenewFlightEndpointRequest" + static let _protobuf_nameMap: SwiftProtobuf._NameMap = [ + 1: .same(proto: "endpoint"), + ] + + mutating func decodeMessage(decoder: inout D) throws { + while let fieldNumber = try decoder.nextFieldNumber() { + // The use of inline closures is to circumvent an issue where the compiler + // allocates stack space for every case branch when no optimizations are + // enabled. https://github.com/apple/swift-protobuf/issues/1034 + switch fieldNumber { + case 1: try { try decoder.decodeSingularMessageField(value: &self._endpoint) }() + default: break + } + } + } + + func traverse(visitor: inout V) throws { + // The use of inline closures is to circumvent an issue where the compiler + // allocates stack space for every if/case branch local when no optimizations + // are enabled. https://github.com/apple/swift-protobuf/issues/1034 and + // https://github.com/apple/swift-protobuf/issues/1182 + try { if let v = self._endpoint { + try visitor.visitSingularMessageField(value: v, fieldNumber: 1) + } }() + try unknownFields.traverse(visitor: &visitor) + } + + static func ==(lhs: Arrow_Flight_Protocol_RenewFlightEndpointRequest, rhs: Arrow_Flight_Protocol_RenewFlightEndpointRequest) -> Bool { + if lhs._endpoint != rhs._endpoint {return false} + if lhs.unknownFields != rhs.unknownFields {return false} + return true + } +} + +extension Arrow_Flight_Protocol_Result: SwiftProtobuf.Message, SwiftProtobuf._MessageImplementationBase, SwiftProtobuf._ProtoNameProviding { + static let protoMessageName: String = _protobuf_package + ".Result" + static let _protobuf_nameMap: SwiftProtobuf._NameMap = [ + 1: .same(proto: "body"), + ] + + mutating func decodeMessage(decoder: inout D) throws { + while let fieldNumber = try decoder.nextFieldNumber() { + // The use of inline closures is to circumvent an issue where the compiler + // allocates stack space for every case branch when no optimizations are + // enabled. https://github.com/apple/swift-protobuf/issues/1034 + switch fieldNumber { + case 1: try { try decoder.decodeSingularBytesField(value: &self.body) }() + default: break + } + } + } + + func traverse(visitor: inout V) throws { + if !self.body.isEmpty { + try visitor.visitSingularBytesField(value: self.body, fieldNumber: 1) + } + try unknownFields.traverse(visitor: &visitor) + } + + static func ==(lhs: Arrow_Flight_Protocol_Result, rhs: Arrow_Flight_Protocol_Result) -> Bool { + if lhs.body != rhs.body {return false} + if lhs.unknownFields != rhs.unknownFields {return false} + return true + } +} + +extension Arrow_Flight_Protocol_CancelFlightInfoResult: SwiftProtobuf.Message, SwiftProtobuf._MessageImplementationBase, SwiftProtobuf._ProtoNameProviding { + static let protoMessageName: String = _protobuf_package + ".CancelFlightInfoResult" + static let _protobuf_nameMap: SwiftProtobuf._NameMap = [ + 1: .same(proto: "status"), + ] + + mutating func decodeMessage(decoder: inout D) throws { + while let fieldNumber = try decoder.nextFieldNumber() { + // The use of inline closures is to circumvent an issue where the compiler + // allocates stack space for every case branch when no optimizations are + // enabled. https://github.com/apple/swift-protobuf/issues/1034 + switch fieldNumber { + case 1: try { try decoder.decodeSingularEnumField(value: &self.status) }() + default: break + } + } + } + + func traverse(visitor: inout V) throws { + if self.status != .unspecified { + try visitor.visitSingularEnumField(value: self.status, fieldNumber: 1) + } + try unknownFields.traverse(visitor: &visitor) + } + + static func ==(lhs: Arrow_Flight_Protocol_CancelFlightInfoResult, rhs: Arrow_Flight_Protocol_CancelFlightInfoResult) -> Bool { + if lhs.status != rhs.status {return false} + if lhs.unknownFields != rhs.unknownFields {return false} + return true + } +} + +extension Arrow_Flight_Protocol_SchemaResult: SwiftProtobuf.Message, SwiftProtobuf._MessageImplementationBase, SwiftProtobuf._ProtoNameProviding { + static let protoMessageName: String = _protobuf_package + ".SchemaResult" + static let _protobuf_nameMap: SwiftProtobuf._NameMap = [ + 1: .same(proto: "schema"), + ] + + mutating func decodeMessage(decoder: inout D) throws { + while let fieldNumber = try decoder.nextFieldNumber() { + // The use of inline closures is to circumvent an issue where the compiler + // allocates stack space for every case branch when no optimizations are + // enabled. https://github.com/apple/swift-protobuf/issues/1034 + switch fieldNumber { + case 1: try { try decoder.decodeSingularBytesField(value: &self.schema) }() + default: break + } + } + } + + func traverse(visitor: inout V) throws { + if !self.schema.isEmpty { + try visitor.visitSingularBytesField(value: self.schema, fieldNumber: 1) + } + try unknownFields.traverse(visitor: &visitor) + } + + static func ==(lhs: Arrow_Flight_Protocol_SchemaResult, rhs: Arrow_Flight_Protocol_SchemaResult) -> Bool { + if lhs.schema != rhs.schema {return false} + if lhs.unknownFields != rhs.unknownFields {return false} + return true + } +} + +extension Arrow_Flight_Protocol_FlightDescriptor: SwiftProtobuf.Message, SwiftProtobuf._MessageImplementationBase, SwiftProtobuf._ProtoNameProviding { + static let protoMessageName: String = _protobuf_package + ".FlightDescriptor" + static let _protobuf_nameMap: SwiftProtobuf._NameMap = [ + 1: .same(proto: "type"), + 2: .same(proto: "cmd"), + 3: .same(proto: "path"), + ] + + mutating func decodeMessage(decoder: inout D) throws { + while let fieldNumber = try decoder.nextFieldNumber() { + // The use of inline closures is to circumvent an issue where the compiler + // allocates stack space for every case branch when no optimizations are + // enabled. https://github.com/apple/swift-protobuf/issues/1034 + switch fieldNumber { + case 1: try { try decoder.decodeSingularEnumField(value: &self.type) }() + case 2: try { try decoder.decodeSingularBytesField(value: &self.cmd) }() + case 3: try { try decoder.decodeRepeatedStringField(value: &self.path) }() + default: break + } + } + } + + func traverse(visitor: inout V) throws { + if self.type != .unknown { + try visitor.visitSingularEnumField(value: self.type, fieldNumber: 1) + } + if !self.cmd.isEmpty { + try visitor.visitSingularBytesField(value: self.cmd, fieldNumber: 2) + } + if !self.path.isEmpty { + try visitor.visitRepeatedStringField(value: self.path, fieldNumber: 3) + } + try unknownFields.traverse(visitor: &visitor) + } + + static func ==(lhs: Arrow_Flight_Protocol_FlightDescriptor, rhs: Arrow_Flight_Protocol_FlightDescriptor) -> Bool { + if lhs.type != rhs.type {return false} + if lhs.cmd != rhs.cmd {return false} + if lhs.path != rhs.path {return false} + if lhs.unknownFields != rhs.unknownFields {return false} + return true + } +} + +extension Arrow_Flight_Protocol_FlightDescriptor.DescriptorType: SwiftProtobuf._ProtoNameProviding { + static let _protobuf_nameMap: SwiftProtobuf._NameMap = [ + 0: .same(proto: "UNKNOWN"), + 1: .same(proto: "PATH"), + 2: .same(proto: "CMD"), + ] +} + +extension Arrow_Flight_Protocol_FlightInfo: SwiftProtobuf.Message, SwiftProtobuf._MessageImplementationBase, SwiftProtobuf._ProtoNameProviding { + static let protoMessageName: String = _protobuf_package + ".FlightInfo" + static let _protobuf_nameMap: SwiftProtobuf._NameMap = [ + 1: .same(proto: "schema"), + 2: .standard(proto: "flight_descriptor"), + 3: .same(proto: "endpoint"), + 4: .standard(proto: "total_records"), + 5: .standard(proto: "total_bytes"), + 6: .same(proto: "ordered"), + ] + + mutating func decodeMessage(decoder: inout D) throws { + while let fieldNumber = try decoder.nextFieldNumber() { + // The use of inline closures is to circumvent an issue where the compiler + // allocates stack space for every case branch when no optimizations are + // enabled. https://github.com/apple/swift-protobuf/issues/1034 + switch fieldNumber { + case 1: try { try decoder.decodeSingularBytesField(value: &self.schema) }() + case 2: try { try decoder.decodeSingularMessageField(value: &self._flightDescriptor) }() + case 3: try { try decoder.decodeRepeatedMessageField(value: &self.endpoint) }() + case 4: try { try decoder.decodeSingularInt64Field(value: &self.totalRecords) }() + case 5: try { try decoder.decodeSingularInt64Field(value: &self.totalBytes) }() + case 6: try { try decoder.decodeSingularBoolField(value: &self.ordered) }() + default: break + } + } + } + + func traverse(visitor: inout V) throws { + // The use of inline closures is to circumvent an issue where the compiler + // allocates stack space for every if/case branch local when no optimizations + // are enabled. https://github.com/apple/swift-protobuf/issues/1034 and + // https://github.com/apple/swift-protobuf/issues/1182 + if !self.schema.isEmpty { + try visitor.visitSingularBytesField(value: self.schema, fieldNumber: 1) + } + try { if let v = self._flightDescriptor { + try visitor.visitSingularMessageField(value: v, fieldNumber: 2) + } }() + if !self.endpoint.isEmpty { + try visitor.visitRepeatedMessageField(value: self.endpoint, fieldNumber: 3) + } + if self.totalRecords != 0 { + try visitor.visitSingularInt64Field(value: self.totalRecords, fieldNumber: 4) + } + if self.totalBytes != 0 { + try visitor.visitSingularInt64Field(value: self.totalBytes, fieldNumber: 5) + } + if self.ordered != false { + try visitor.visitSingularBoolField(value: self.ordered, fieldNumber: 6) + } + try unknownFields.traverse(visitor: &visitor) + } + + static func ==(lhs: Arrow_Flight_Protocol_FlightInfo, rhs: Arrow_Flight_Protocol_FlightInfo) -> Bool { + if lhs.schema != rhs.schema {return false} + if lhs._flightDescriptor != rhs._flightDescriptor {return false} + if lhs.endpoint != rhs.endpoint {return false} + if lhs.totalRecords != rhs.totalRecords {return false} + if lhs.totalBytes != rhs.totalBytes {return false} + if lhs.ordered != rhs.ordered {return false} + if lhs.unknownFields != rhs.unknownFields {return false} + return true + } +} + +extension Arrow_Flight_Protocol_FlightEndpoint: SwiftProtobuf.Message, SwiftProtobuf._MessageImplementationBase, SwiftProtobuf._ProtoNameProviding { + static let protoMessageName: String = _protobuf_package + ".FlightEndpoint" + static let _protobuf_nameMap: SwiftProtobuf._NameMap = [ + 1: .same(proto: "ticket"), + 2: .same(proto: "location"), + 3: .standard(proto: "expiration_time"), + ] + + mutating func decodeMessage(decoder: inout D) throws { + while let fieldNumber = try decoder.nextFieldNumber() { + // The use of inline closures is to circumvent an issue where the compiler + // allocates stack space for every case branch when no optimizations are + // enabled. https://github.com/apple/swift-protobuf/issues/1034 + switch fieldNumber { + case 1: try { try decoder.decodeSingularMessageField(value: &self._ticket) }() + case 2: try { try decoder.decodeRepeatedMessageField(value: &self.location) }() + case 3: try { try decoder.decodeSingularMessageField(value: &self._expirationTime) }() + default: break + } + } + } + + func traverse(visitor: inout V) throws { + // The use of inline closures is to circumvent an issue where the compiler + // allocates stack space for every if/case branch local when no optimizations + // are enabled. https://github.com/apple/swift-protobuf/issues/1034 and + // https://github.com/apple/swift-protobuf/issues/1182 + try { if let v = self._ticket { + try visitor.visitSingularMessageField(value: v, fieldNumber: 1) + } }() + if !self.location.isEmpty { + try visitor.visitRepeatedMessageField(value: self.location, fieldNumber: 2) + } + try { if let v = self._expirationTime { + try visitor.visitSingularMessageField(value: v, fieldNumber: 3) + } }() + try unknownFields.traverse(visitor: &visitor) + } + + static func ==(lhs: Arrow_Flight_Protocol_FlightEndpoint, rhs: Arrow_Flight_Protocol_FlightEndpoint) -> Bool { + if lhs._ticket != rhs._ticket {return false} + if lhs.location != rhs.location {return false} + if lhs._expirationTime != rhs._expirationTime {return false} + if lhs.unknownFields != rhs.unknownFields {return false} + return true + } +} + +extension Arrow_Flight_Protocol_Location: SwiftProtobuf.Message, SwiftProtobuf._MessageImplementationBase, SwiftProtobuf._ProtoNameProviding { + static let protoMessageName: String = _protobuf_package + ".Location" + static let _protobuf_nameMap: SwiftProtobuf._NameMap = [ + 1: .same(proto: "uri"), + ] + + mutating func decodeMessage(decoder: inout D) throws { + while let fieldNumber = try decoder.nextFieldNumber() { + // The use of inline closures is to circumvent an issue where the compiler + // allocates stack space for every case branch when no optimizations are + // enabled. https://github.com/apple/swift-protobuf/issues/1034 + switch fieldNumber { + case 1: try { try decoder.decodeSingularStringField(value: &self.uri) }() + default: break + } + } + } + + func traverse(visitor: inout V) throws { + if !self.uri.isEmpty { + try visitor.visitSingularStringField(value: self.uri, fieldNumber: 1) + } + try unknownFields.traverse(visitor: &visitor) + } + + static func ==(lhs: Arrow_Flight_Protocol_Location, rhs: Arrow_Flight_Protocol_Location) -> Bool { + if lhs.uri != rhs.uri {return false} + if lhs.unknownFields != rhs.unknownFields {return false} + return true + } +} + +extension Arrow_Flight_Protocol_Ticket: SwiftProtobuf.Message, SwiftProtobuf._MessageImplementationBase, SwiftProtobuf._ProtoNameProviding { + static let protoMessageName: String = _protobuf_package + ".Ticket" + static let _protobuf_nameMap: SwiftProtobuf._NameMap = [ + 1: .same(proto: "ticket"), + ] + + mutating func decodeMessage(decoder: inout D) throws { + while let fieldNumber = try decoder.nextFieldNumber() { + // The use of inline closures is to circumvent an issue where the compiler + // allocates stack space for every case branch when no optimizations are + // enabled. https://github.com/apple/swift-protobuf/issues/1034 + switch fieldNumber { + case 1: try { try decoder.decodeSingularBytesField(value: &self.ticket) }() + default: break + } + } + } + + func traverse(visitor: inout V) throws { + if !self.ticket.isEmpty { + try visitor.visitSingularBytesField(value: self.ticket, fieldNumber: 1) + } + try unknownFields.traverse(visitor: &visitor) + } + + static func ==(lhs: Arrow_Flight_Protocol_Ticket, rhs: Arrow_Flight_Protocol_Ticket) -> Bool { + if lhs.ticket != rhs.ticket {return false} + if lhs.unknownFields != rhs.unknownFields {return false} + return true + } +} + +extension Arrow_Flight_Protocol_FlightData: SwiftProtobuf.Message, SwiftProtobuf._MessageImplementationBase, SwiftProtobuf._ProtoNameProviding { + static let protoMessageName: String = _protobuf_package + ".FlightData" + static let _protobuf_nameMap: SwiftProtobuf._NameMap = [ + 1: .standard(proto: "flight_descriptor"), + 2: .standard(proto: "data_header"), + 3: .standard(proto: "app_metadata"), + 1000: .standard(proto: "data_body"), + ] + + mutating func decodeMessage(decoder: inout D) throws { + while let fieldNumber = try decoder.nextFieldNumber() { + // The use of inline closures is to circumvent an issue where the compiler + // allocates stack space for every case branch when no optimizations are + // enabled. https://github.com/apple/swift-protobuf/issues/1034 + switch fieldNumber { + case 1: try { try decoder.decodeSingularMessageField(value: &self._flightDescriptor) }() + case 2: try { try decoder.decodeSingularBytesField(value: &self.dataHeader) }() + case 3: try { try decoder.decodeSingularBytesField(value: &self.appMetadata) }() + case 1000: try { try decoder.decodeSingularBytesField(value: &self.dataBody) }() + default: break + } + } + } + + func traverse(visitor: inout V) throws { + // The use of inline closures is to circumvent an issue where the compiler + // allocates stack space for every if/case branch local when no optimizations + // are enabled. https://github.com/apple/swift-protobuf/issues/1034 and + // https://github.com/apple/swift-protobuf/issues/1182 + try { if let v = self._flightDescriptor { + try visitor.visitSingularMessageField(value: v, fieldNumber: 1) + } }() + if !self.dataHeader.isEmpty { + try visitor.visitSingularBytesField(value: self.dataHeader, fieldNumber: 2) + } + if !self.appMetadata.isEmpty { + try visitor.visitSingularBytesField(value: self.appMetadata, fieldNumber: 3) + } + if !self.dataBody.isEmpty { + try visitor.visitSingularBytesField(value: self.dataBody, fieldNumber: 1000) + } + try unknownFields.traverse(visitor: &visitor) + } + + static func ==(lhs: Arrow_Flight_Protocol_FlightData, rhs: Arrow_Flight_Protocol_FlightData) -> Bool { + if lhs._flightDescriptor != rhs._flightDescriptor {return false} + if lhs.dataHeader != rhs.dataHeader {return false} + if lhs.appMetadata != rhs.appMetadata {return false} + if lhs.dataBody != rhs.dataBody {return false} + if lhs.unknownFields != rhs.unknownFields {return false} + return true + } +} + +extension Arrow_Flight_Protocol_PutResult: SwiftProtobuf.Message, SwiftProtobuf._MessageImplementationBase, SwiftProtobuf._ProtoNameProviding { + static let protoMessageName: String = _protobuf_package + ".PutResult" + static let _protobuf_nameMap: SwiftProtobuf._NameMap = [ + 1: .standard(proto: "app_metadata"), + ] + + mutating func decodeMessage(decoder: inout D) throws { + while let fieldNumber = try decoder.nextFieldNumber() { + // The use of inline closures is to circumvent an issue where the compiler + // allocates stack space for every case branch when no optimizations are + // enabled. https://github.com/apple/swift-protobuf/issues/1034 + switch fieldNumber { + case 1: try { try decoder.decodeSingularBytesField(value: &self.appMetadata) }() + default: break + } + } + } + + func traverse(visitor: inout V) throws { + if !self.appMetadata.isEmpty { + try visitor.visitSingularBytesField(value: self.appMetadata, fieldNumber: 1) + } + try unknownFields.traverse(visitor: &visitor) + } + + static func ==(lhs: Arrow_Flight_Protocol_PutResult, rhs: Arrow_Flight_Protocol_PutResult) -> Bool { + if lhs.appMetadata != rhs.appMetadata {return false} + if lhs.unknownFields != rhs.unknownFields {return false} + return true + } +} diff --git a/Sources/SparkConnect/FlightData.swift b/Sources/SparkConnect/FlightData.swift new file mode 100644 index 0000000..ee13311 --- /dev/null +++ b/Sources/SparkConnect/FlightData.swift @@ -0,0 +1,49 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +import Foundation + +public class FlightData { + let flightData: Arrow_Flight_Protocol_FlightData + public var flightDescriptor: FlightDescriptor? { + return flightData.hasFlightDescriptor ? FlightDescriptor(flightData.flightDescriptor) : nil + } + + public var dataHeader: Data { flightData.dataHeader } + + public var dataBody: Data { flightData.dataBody } + + init(_ flightData: Arrow_Flight_Protocol_FlightData) { + self.flightData = flightData + } + + public init(_ dataHeader: Data, dataBody: Data, flightDescriptor: FlightDescriptor? = nil) { + if flightDescriptor != nil { + self.flightData = Arrow_Flight_Protocol_FlightData.with { + $0.dataHeader = dataHeader + $0.dataBody = dataBody + $0.flightDescriptor = flightDescriptor!.toProtocol() + } + } else { + self.flightData = Arrow_Flight_Protocol_FlightData.with { + $0.dataBody = dataBody + } + } + } + + func toProtocol() -> Arrow_Flight_Protocol_FlightData { self.flightData } +} diff --git a/Sources/SparkConnect/FlightDescriptor.swift b/Sources/SparkConnect/FlightDescriptor.swift new file mode 100644 index 0000000..41639c9 --- /dev/null +++ b/Sources/SparkConnect/FlightDescriptor.swift @@ -0,0 +1,56 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +import Foundation + +public class FlightDescriptor { + public enum DescriptorType { + case unknown + case path + case cmd + } + + public let type: FlightDescriptor.DescriptorType + public let cmd: Data + public let paths: [String] + + init(_ descriptor: Arrow_Flight_Protocol_FlightDescriptor) { + self.type = descriptor.type == .cmd ? .cmd : .path + self.cmd = descriptor.cmd + self.paths = descriptor.path + } + + public init(cmd: Data) { + self.type = .cmd + self.cmd = cmd + self.paths = [String]() + } + + public init(paths: [String]) { + self.type = .path + self.cmd = Data() + self.paths = paths + } + + func toProtocol() -> Arrow_Flight_Protocol_FlightDescriptor { + var descriptor = Arrow_Flight_Protocol_FlightDescriptor() + descriptor.type = self.type == .cmd ? .cmd : .path + descriptor.cmd = self.cmd + descriptor.path = self.paths + return descriptor + } +} diff --git a/Sources/SparkConnect/MemoryAllocator.swift b/Sources/SparkConnect/MemoryAllocator.swift new file mode 100644 index 0000000..63f91f5 --- /dev/null +++ b/Sources/SparkConnect/MemoryAllocator.swift @@ -0,0 +1,31 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +import Foundation + +public class MemoryAllocator { + let alignment: Int + init(_ alignment: Int) { + self.alignment = alignment + } + + func allocateArray(_ byteCount: Int) -> UnsafeMutableRawPointer { + return UnsafeMutableRawPointer.allocate( + byteCount: byteCount, + alignment: self.alignment) + } +} diff --git a/Sources/SparkConnect/Message_generated.swift b/Sources/SparkConnect/Message_generated.swift new file mode 100644 index 0000000..2bdf699 --- /dev/null +++ b/Sources/SparkConnect/Message_generated.swift @@ -0,0 +1,629 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +// automatically generated by the FlatBuffers compiler, do not modify +// swiftlint:disable all +// swiftformat:disable all + +import FlatBuffers + +public enum org_apache_arrow_flatbuf_CompressionType: Int8, Enum, Verifiable { + public typealias T = Int8 + public static var byteSize: Int { return MemoryLayout.size } + public var value: Int8 { return self.rawValue } + case lz4Frame = 0 + case zstd = 1 + + public static var max: org_apache_arrow_flatbuf_CompressionType { return .zstd } + public static var min: org_apache_arrow_flatbuf_CompressionType { return .lz4Frame } +} + +/// Provided for forward compatibility in case we need to support different +/// strategies for compressing the IPC message body (like whole-body +/// compression rather than buffer-level) in the future +public enum org_apache_arrow_flatbuf_BodyCompressionMethod: Int8, Enum, Verifiable { + public typealias T = Int8 + public static var byteSize: Int { return MemoryLayout.size } + public var value: Int8 { return self.rawValue } + /// Each constituent buffer is first compressed with the indicated + /// compressor, and then written with the uncompressed length in the first 8 + /// bytes as a 64-bit little-endian signed integer followed by the compressed + /// buffer bytes (and then padding as required by the protocol). The + /// uncompressed length may be set to -1 to indicate that the data that + /// follows is not compressed, which can be useful for cases where + /// compression does not yield appreciable savings. + case buffer = 0 + + public static var max: org_apache_arrow_flatbuf_BodyCompressionMethod { return .buffer } + public static var min: org_apache_arrow_flatbuf_BodyCompressionMethod { return .buffer } +} + +/// ---------------------------------------------------------------------- +/// The root Message type +/// This union enables us to easily send different message types without +/// redundant storage, and in the future we can easily add new message types. +/// +/// Arrow implementations do not need to implement all of the message types, +/// which may include experimental metadata types. For maximum compatibility, +/// it is best to send data using RecordBatch +public enum org_apache_arrow_flatbuf_MessageHeader: UInt8, UnionEnum { + public typealias T = UInt8 + + public init?(value: T) { + self.init(rawValue: value) + } + + public static var byteSize: Int { return MemoryLayout.size } + public var value: UInt8 { return self.rawValue } + case none_ = 0 + case schema = 1 + case dictionarybatch = 2 + case recordbatch = 3 + case tensor = 4 + case sparsetensor = 5 + + public static var max: org_apache_arrow_flatbuf_MessageHeader { return .sparsetensor } + public static var min: org_apache_arrow_flatbuf_MessageHeader { return .none_ } +} + +/// ---------------------------------------------------------------------- +/// Data structures for describing a table row batch (a collection of +/// equal-length Arrow arrays) +/// Metadata about a field at some level of a nested type tree (but not +/// its children). +/// +/// For example, a List with values `[[1, 2, 3], null, [4], [5, 6], null]` +/// would have {length: 5, null_count: 2} for its List node, and {length: 6, +/// null_count: 0} for its Int16 node, as separate FieldNode structs +public struct org_apache_arrow_flatbuf_FieldNode: NativeStruct, Verifiable, FlatbuffersInitializable +{ + + static func validateVersion() { FlatBuffersVersion_23_1_4() } + + /// The number of value slots in the Arrow array at this level of a nested + /// tree + private var _length: Int64 + /// The number of observed nulls. Fields with null_count == 0 may choose not + /// to write their physical validity bitmap out as a materialized buffer, + /// instead setting the length of the bitmap buffer to 0. + private var _nullCount: Int64 + + public init(_ bb: ByteBuffer, o: Int32) { + let _accessor = Struct(bb: bb, position: o) + _length = _accessor.readBuffer(of: Int64.self, at: 0) + _nullCount = _accessor.readBuffer(of: Int64.self, at: 8) + } + + public init(length: Int64, nullCount: Int64) { + _length = length + _nullCount = nullCount + } + + public init() { + _length = 0 + _nullCount = 0 + } + + /// The number of value slots in the Arrow array at this level of a nested + /// tree + public var length: Int64 { _length } + /// The number of observed nulls. Fields with null_count == 0 may choose not + /// to write their physical validity bitmap out as a materialized buffer, + /// instead setting the length of the bitmap buffer to 0. + public var nullCount: Int64 { _nullCount } + + public static func verify(_ verifier: inout Verifier, at position: Int, of type: T.Type) throws + where T: Verifiable { + try verifier.inBuffer(position: position, of: org_apache_arrow_flatbuf_FieldNode.self) + } +} + +/// ---------------------------------------------------------------------- +/// Data structures for describing a table row batch (a collection of +/// equal-length Arrow arrays) +/// Metadata about a field at some level of a nested type tree (but not +/// its children). +/// +/// For example, a List with values `[[1, 2, 3], null, [4], [5, 6], null]` +/// would have {length: 5, null_count: 2} for its List node, and {length: 6, +/// null_count: 0} for its Int16 node, as separate FieldNode structs +public struct org_apache_arrow_flatbuf_FieldNode_Mutable: FlatBufferObject { + + static func validateVersion() { FlatBuffersVersion_23_1_4() } + public var __buffer: ByteBuffer! { return _accessor.bb } + private var _accessor: Struct + + public init(_ bb: ByteBuffer, o: Int32) { _accessor = Struct(bb: bb, position: o) } + + public var length: Int64 { return _accessor.readBuffer(of: Int64.self, at: 0) } + public var nullCount: Int64 { return _accessor.readBuffer(of: Int64.self, at: 8) } +} + +/// Optional compression for the memory buffers constituting IPC message +/// bodies. Intended for use with RecordBatch but could be used for other +/// message types +public struct org_apache_arrow_flatbuf_BodyCompression: FlatBufferObject, Verifiable { + + static func validateVersion() { FlatBuffersVersion_23_1_4() } + public var __buffer: ByteBuffer! { return _accessor.bb } + private var _accessor: Table + + public static func getRootAsBodyCompression(bb: ByteBuffer) + -> org_apache_arrow_flatbuf_BodyCompression + { + return org_apache_arrow_flatbuf_BodyCompression( + Table( + bb: bb, position: Int32(bb.read(def: UOffset.self, position: bb.reader)) + Int32(bb.reader)) + ) + } + + private init(_ t: Table) { _accessor = t } + public init(_ bb: ByteBuffer, o: Int32) { _accessor = Table(bb: bb, position: o) } + + private enum VTOFFSET: VOffset { + case codec = 4 + case method = 6 + var v: Int32 { Int32(self.rawValue) } + var p: VOffset { self.rawValue } + } + + /// Compressor library. + /// For LZ4_FRAME, each compressed buffer must consist of a single frame. + public var codec: org_apache_arrow_flatbuf_CompressionType { + let o = _accessor.offset(VTOFFSET.codec.v) + return o == 0 + ? .lz4Frame + : org_apache_arrow_flatbuf_CompressionType( + rawValue: _accessor.readBuffer(of: Int8.self, at: o)) ?? .lz4Frame + } + /// Indicates the way the record batch body was compressed + public var method: org_apache_arrow_flatbuf_BodyCompressionMethod { + let o = _accessor.offset(VTOFFSET.method.v) + return o == 0 + ? .buffer + : org_apache_arrow_flatbuf_BodyCompressionMethod( + rawValue: _accessor.readBuffer(of: Int8.self, at: o)) ?? .buffer + } + public static func startBodyCompression(_ fbb: inout FlatBufferBuilder) -> UOffset { + fbb.startTable(with: 2) + } + public static func add( + codec: org_apache_arrow_flatbuf_CompressionType, _ fbb: inout FlatBufferBuilder + ) { fbb.add(element: codec.rawValue, def: 0, at: VTOFFSET.codec.p) } + public static func add( + method: org_apache_arrow_flatbuf_BodyCompressionMethod, _ fbb: inout FlatBufferBuilder + ) { fbb.add(element: method.rawValue, def: 0, at: VTOFFSET.method.p) } + public static func endBodyCompression(_ fbb: inout FlatBufferBuilder, start: UOffset) -> Offset { + let end = Offset(offset: fbb.endTable(at: start)) + return end + } + public static func createBodyCompression( + _ fbb: inout FlatBufferBuilder, + codec: org_apache_arrow_flatbuf_CompressionType = .lz4Frame, + method: org_apache_arrow_flatbuf_BodyCompressionMethod = .buffer + ) -> Offset { + let __start = org_apache_arrow_flatbuf_BodyCompression.startBodyCompression(&fbb) + org_apache_arrow_flatbuf_BodyCompression.add(codec: codec, &fbb) + org_apache_arrow_flatbuf_BodyCompression.add(method: method, &fbb) + return org_apache_arrow_flatbuf_BodyCompression.endBodyCompression(&fbb, start: __start) + } + + public static func verify(_ verifier: inout Verifier, at position: Int, of type: T.Type) throws + where T: Verifiable { + var _v = try verifier.visitTable(at: position) + try _v.visit( + field: VTOFFSET.codec.p, fieldName: "codec", required: false, + type: org_apache_arrow_flatbuf_CompressionType.self) + try _v.visit( + field: VTOFFSET.method.p, fieldName: "method", required: false, + type: org_apache_arrow_flatbuf_BodyCompressionMethod.self) + _v.finish() + } +} + +/// A data header describing the shared memory layout of a "record" or "row" +/// batch. Some systems call this a "row batch" internally and others a "record +/// batch". +public struct org_apache_arrow_flatbuf_RecordBatch: FlatBufferObject, Verifiable { + + static func validateVersion() { FlatBuffersVersion_23_1_4() } + public var __buffer: ByteBuffer! { return _accessor.bb } + private var _accessor: Table + + public static func getRootAsRecordBatch(bb: ByteBuffer) -> org_apache_arrow_flatbuf_RecordBatch { + return org_apache_arrow_flatbuf_RecordBatch( + Table( + bb: bb, position: Int32(bb.read(def: UOffset.self, position: bb.reader)) + Int32(bb.reader)) + ) + } + + private init(_ t: Table) { _accessor = t } + public init(_ bb: ByteBuffer, o: Int32) { _accessor = Table(bb: bb, position: o) } + + private enum VTOFFSET: VOffset { + case length = 4 + case nodes = 6 + case buffers = 8 + case compression = 10 + var v: Int32 { Int32(self.rawValue) } + var p: VOffset { self.rawValue } + } + + /// number of records / rows. The arrays in the batch should all have this + /// length + public var length: Int64 { + let o = _accessor.offset(VTOFFSET.length.v) + return o == 0 ? 0 : _accessor.readBuffer(of: Int64.self, at: o) + } + /// Nodes correspond to the pre-ordered flattened logical schema + public var hasNodes: Bool { + let o = _accessor.offset(VTOFFSET.nodes.v) + return o == 0 ? false : true + } + public var nodesCount: Int32 { + let o = _accessor.offset(VTOFFSET.nodes.v) + return o == 0 ? 0 : _accessor.vector(count: o) + } + public func nodes(at index: Int32) -> org_apache_arrow_flatbuf_FieldNode? { + let o = _accessor.offset(VTOFFSET.nodes.v) + return o == 0 + ? nil + : _accessor.directRead( + of: org_apache_arrow_flatbuf_FieldNode.self, offset: _accessor.vector(at: o) + index * 16) + } + public func mutableNodes(at index: Int32) -> org_apache_arrow_flatbuf_FieldNode_Mutable? { + let o = _accessor.offset(VTOFFSET.nodes.v) + return o == 0 + ? nil + : org_apache_arrow_flatbuf_FieldNode_Mutable( + _accessor.bb, o: _accessor.vector(at: o) + index * 16) + } + /// Buffers correspond to the pre-ordered flattened buffer tree + /// + /// The number of buffers appended to this list depends on the schema. For + /// example, most primitive arrays will have 2 buffers, 1 for the validity + /// bitmap and 1 for the values. For struct arrays, there will only be a + /// single buffer for the validity (nulls) bitmap + public var hasBuffers: Bool { + let o = _accessor.offset(VTOFFSET.buffers.v) + return o == 0 ? false : true + } + public var buffersCount: Int32 { + let o = _accessor.offset(VTOFFSET.buffers.v) + return o == 0 ? 0 : _accessor.vector(count: o) + } + public func buffers(at index: Int32) -> org_apache_arrow_flatbuf_Buffer? { + let o = _accessor.offset(VTOFFSET.buffers.v) + return o == 0 + ? nil + : _accessor.directRead( + of: org_apache_arrow_flatbuf_Buffer.self, offset: _accessor.vector(at: o) + index * 16) + } + public func mutableBuffers(at index: Int32) -> org_apache_arrow_flatbuf_Buffer_Mutable? { + let o = _accessor.offset(VTOFFSET.buffers.v) + return o == 0 + ? nil + : org_apache_arrow_flatbuf_Buffer_Mutable( + _accessor.bb, o: _accessor.vector(at: o) + index * 16) + } + /// Optional compression of the message body + public var compression: org_apache_arrow_flatbuf_BodyCompression? { + let o = _accessor.offset(VTOFFSET.compression.v) + return o == 0 + ? nil + : org_apache_arrow_flatbuf_BodyCompression( + _accessor.bb, o: _accessor.indirect(o + _accessor.postion)) + } + public static func startRecordBatch(_ fbb: inout FlatBufferBuilder) -> UOffset { + fbb.startTable(with: 4) + } + public static func add(length: Int64, _ fbb: inout FlatBufferBuilder) { + fbb.add(element: length, def: 0, at: VTOFFSET.length.p) + } + public static func addVectorOf(nodes: Offset, _ fbb: inout FlatBufferBuilder) { + fbb.add(offset: nodes, at: VTOFFSET.nodes.p) + } + public static func startVectorOfNodes(_ size: Int, in builder: inout FlatBufferBuilder) { + builder.startVector( + size * MemoryLayout.size, + elementSize: MemoryLayout.alignment) + } + public static func addVectorOf(buffers: Offset, _ fbb: inout FlatBufferBuilder) { + fbb.add(offset: buffers, at: VTOFFSET.buffers.p) + } + public static func startVectorOfBuffers(_ size: Int, in builder: inout FlatBufferBuilder) { + builder.startVector( + size * MemoryLayout.size, + elementSize: MemoryLayout.alignment) + } + public static func add(compression: Offset, _ fbb: inout FlatBufferBuilder) { + fbb.add(offset: compression, at: VTOFFSET.compression.p) + } + public static func endRecordBatch(_ fbb: inout FlatBufferBuilder, start: UOffset) -> Offset { + let end = Offset(offset: fbb.endTable(at: start)) + return end + } + public static func createRecordBatch( + _ fbb: inout FlatBufferBuilder, + length: Int64 = 0, + nodesVectorOffset nodes: Offset = Offset(), + buffersVectorOffset buffers: Offset = Offset(), + compressionOffset compression: Offset = Offset() + ) -> Offset { + let __start = org_apache_arrow_flatbuf_RecordBatch.startRecordBatch(&fbb) + org_apache_arrow_flatbuf_RecordBatch.add(length: length, &fbb) + org_apache_arrow_flatbuf_RecordBatch.addVectorOf(nodes: nodes, &fbb) + org_apache_arrow_flatbuf_RecordBatch.addVectorOf(buffers: buffers, &fbb) + org_apache_arrow_flatbuf_RecordBatch.add(compression: compression, &fbb) + return org_apache_arrow_flatbuf_RecordBatch.endRecordBatch(&fbb, start: __start) + } + + public static func verify(_ verifier: inout Verifier, at position: Int, of type: T.Type) throws + where T: Verifiable { + var _v = try verifier.visitTable(at: position) + try _v.visit(field: VTOFFSET.length.p, fieldName: "length", required: false, type: Int64.self) + try _v.visit( + field: VTOFFSET.nodes.p, fieldName: "nodes", required: false, + type: ForwardOffset< + Vector + >.self) + try _v.visit( + field: VTOFFSET.buffers.p, fieldName: "buffers", required: false, + type: ForwardOffset> + .self) + try _v.visit( + field: VTOFFSET.compression.p, fieldName: "compression", required: false, + type: ForwardOffset.self) + _v.finish() + } +} + +/// For sending dictionary encoding information. Any Field can be +/// dictionary-encoded, but in this case none of its children may be +/// dictionary-encoded. +/// There is one vector / column per dictionary, but that vector / column +/// may be spread across multiple dictionary batches by using the isDelta +/// flag +public struct org_apache_arrow_flatbuf_DictionaryBatch: FlatBufferObject, Verifiable { + + static func validateVersion() { FlatBuffersVersion_23_1_4() } + public var __buffer: ByteBuffer! { return _accessor.bb } + private var _accessor: Table + + public static func getRootAsDictionaryBatch(bb: ByteBuffer) + -> org_apache_arrow_flatbuf_DictionaryBatch + { + return org_apache_arrow_flatbuf_DictionaryBatch( + Table( + bb: bb, position: Int32(bb.read(def: UOffset.self, position: bb.reader)) + Int32(bb.reader)) + ) + } + + private init(_ t: Table) { _accessor = t } + public init(_ bb: ByteBuffer, o: Int32) { _accessor = Table(bb: bb, position: o) } + + private enum VTOFFSET: VOffset { + case id = 4 + case data = 6 + case isDelta = 8 + var v: Int32 { Int32(self.rawValue) } + var p: VOffset { self.rawValue } + } + + public var id: Int64 { + let o = _accessor.offset(VTOFFSET.id.v) + return o == 0 ? 0 : _accessor.readBuffer(of: Int64.self, at: o) + } + public var data: org_apache_arrow_flatbuf_RecordBatch? { + let o = _accessor.offset(VTOFFSET.data.v) + return o == 0 + ? nil + : org_apache_arrow_flatbuf_RecordBatch( + _accessor.bb, o: _accessor.indirect(o + _accessor.postion)) + } + /// If isDelta is true the values in the dictionary are to be appended to a + /// dictionary with the indicated id. If isDelta is false this dictionary + /// should replace the existing dictionary. + public var isDelta: Bool { + let o = _accessor.offset(VTOFFSET.isDelta.v) + return o == 0 ? false : 0 != _accessor.readBuffer(of: Byte.self, at: o) + } + public static func startDictionaryBatch(_ fbb: inout FlatBufferBuilder) -> UOffset { + fbb.startTable(with: 3) + } + public static func add(id: Int64, _ fbb: inout FlatBufferBuilder) { + fbb.add(element: id, def: 0, at: VTOFFSET.id.p) + } + public static func add(data: Offset, _ fbb: inout FlatBufferBuilder) { + fbb.add(offset: data, at: VTOFFSET.data.p) + } + public static func add(isDelta: Bool, _ fbb: inout FlatBufferBuilder) { + fbb.add( + element: isDelta, def: false, + at: VTOFFSET.isDelta.p) + } + public static func endDictionaryBatch(_ fbb: inout FlatBufferBuilder, start: UOffset) -> Offset { + let end = Offset(offset: fbb.endTable(at: start)) + return end + } + public static func createDictionaryBatch( + _ fbb: inout FlatBufferBuilder, + id: Int64 = 0, + dataOffset data: Offset = Offset(), + isDelta: Bool = false + ) -> Offset { + let __start = org_apache_arrow_flatbuf_DictionaryBatch.startDictionaryBatch(&fbb) + org_apache_arrow_flatbuf_DictionaryBatch.add(id: id, &fbb) + org_apache_arrow_flatbuf_DictionaryBatch.add(data: data, &fbb) + org_apache_arrow_flatbuf_DictionaryBatch.add(isDelta: isDelta, &fbb) + return org_apache_arrow_flatbuf_DictionaryBatch.endDictionaryBatch(&fbb, start: __start) + } + + public static func verify(_ verifier: inout Verifier, at position: Int, of type: T.Type) throws + where T: Verifiable { + var _v = try verifier.visitTable(at: position) + try _v.visit(field: VTOFFSET.id.p, fieldName: "id", required: false, type: Int64.self) + try _v.visit( + field: VTOFFSET.data.p, fieldName: "data", required: false, + type: ForwardOffset.self) + try _v.visit(field: VTOFFSET.isDelta.p, fieldName: "isDelta", required: false, type: Bool.self) + _v.finish() + } +} + +public struct org_apache_arrow_flatbuf_Message: FlatBufferObject, Verifiable { + + static func validateVersion() { FlatBuffersVersion_23_1_4() } + public var __buffer: ByteBuffer! { return _accessor.bb } + private var _accessor: Table + + public static func getRootAsMessage(bb: ByteBuffer) -> org_apache_arrow_flatbuf_Message { + return org_apache_arrow_flatbuf_Message( + Table( + bb: bb, position: Int32(bb.read(def: UOffset.self, position: bb.reader)) + Int32(bb.reader)) + ) + } + + private init(_ t: Table) { _accessor = t } + public init(_ bb: ByteBuffer, o: Int32) { _accessor = Table(bb: bb, position: o) } + + private enum VTOFFSET: VOffset { + case version = 4 + case headerType = 6 + case header = 8 + case bodyLength = 10 + case customMetadata = 12 + var v: Int32 { Int32(self.rawValue) } + var p: VOffset { self.rawValue } + } + + public var version: org_apache_arrow_flatbuf_MetadataVersion { + let o = _accessor.offset(VTOFFSET.version.v) + return o == 0 + ? .v1 + : org_apache_arrow_flatbuf_MetadataVersion( + rawValue: _accessor.readBuffer(of: Int16.self, at: o)) ?? .v1 + } + public var headerType: org_apache_arrow_flatbuf_MessageHeader { + let o = _accessor.offset(VTOFFSET.headerType.v) + return o == 0 + ? .none_ + : org_apache_arrow_flatbuf_MessageHeader( + rawValue: _accessor.readBuffer(of: UInt8.self, at: o)) ?? .none_ + } + public func header(type: T.Type) -> T? { + let o = _accessor.offset(VTOFFSET.header.v) + return o == 0 ? nil : _accessor.union(o) + } + public var bodyLength: Int64 { + let o = _accessor.offset(VTOFFSET.bodyLength.v) + return o == 0 ? 0 : _accessor.readBuffer(of: Int64.self, at: o) + } + public var hasCustomMetadata: Bool { + let o = _accessor.offset(VTOFFSET.customMetadata.v) + return o == 0 ? false : true + } + public var customMetadataCount: Int32 { + let o = _accessor.offset(VTOFFSET.customMetadata.v) + return o == 0 ? 0 : _accessor.vector(count: o) + } + public func customMetadata(at index: Int32) -> org_apache_arrow_flatbuf_KeyValue? { + let o = _accessor.offset(VTOFFSET.customMetadata.v) + return o == 0 + ? nil + : org_apache_arrow_flatbuf_KeyValue( + _accessor.bb, o: _accessor.indirect(_accessor.vector(at: o) + index * 4)) + } + public static func startMessage(_ fbb: inout FlatBufferBuilder) -> UOffset { + fbb.startTable(with: 5) + } + public static func add( + version: org_apache_arrow_flatbuf_MetadataVersion, _ fbb: inout FlatBufferBuilder + ) { fbb.add(element: version.rawValue, def: 0, at: VTOFFSET.version.p) } + public static func add( + headerType: org_apache_arrow_flatbuf_MessageHeader, _ fbb: inout FlatBufferBuilder + ) { fbb.add(element: headerType.rawValue, def: 0, at: VTOFFSET.headerType.p) } + public static func add(header: Offset, _ fbb: inout FlatBufferBuilder) { + fbb.add(offset: header, at: VTOFFSET.header.p) + } + public static func add(bodyLength: Int64, _ fbb: inout FlatBufferBuilder) { + fbb.add(element: bodyLength, def: 0, at: VTOFFSET.bodyLength.p) + } + public static func addVectorOf(customMetadata: Offset, _ fbb: inout FlatBufferBuilder) { + fbb.add(offset: customMetadata, at: VTOFFSET.customMetadata.p) + } + public static func endMessage(_ fbb: inout FlatBufferBuilder, start: UOffset) -> Offset { + let end = Offset(offset: fbb.endTable(at: start)) + return end + } + public static func createMessage( + _ fbb: inout FlatBufferBuilder, + version: org_apache_arrow_flatbuf_MetadataVersion = .v1, + headerType: org_apache_arrow_flatbuf_MessageHeader = .none_, + headerOffset header: Offset = Offset(), + bodyLength: Int64 = 0, + customMetadataVectorOffset customMetadata: Offset = Offset() + ) -> Offset { + let __start = org_apache_arrow_flatbuf_Message.startMessage(&fbb) + org_apache_arrow_flatbuf_Message.add(version: version, &fbb) + org_apache_arrow_flatbuf_Message.add(headerType: headerType, &fbb) + org_apache_arrow_flatbuf_Message.add(header: header, &fbb) + org_apache_arrow_flatbuf_Message.add(bodyLength: bodyLength, &fbb) + org_apache_arrow_flatbuf_Message.addVectorOf(customMetadata: customMetadata, &fbb) + return org_apache_arrow_flatbuf_Message.endMessage(&fbb, start: __start) + } + + public static func verify(_ verifier: inout Verifier, at position: Int, of type: T.Type) throws + where T: Verifiable { + var _v = try verifier.visitTable(at: position) + try _v.visit( + field: VTOFFSET.version.p, fieldName: "version", required: false, + type: org_apache_arrow_flatbuf_MetadataVersion.self) + try _v.visit( + unionKey: VTOFFSET.headerType.p, unionField: VTOFFSET.header.p, unionKeyName: "headerType", + fieldName: "header", required: false, + completion: { (verifier, key: org_apache_arrow_flatbuf_MessageHeader, pos) in + switch key { + case .none_: + break // NOTE - SWIFT doesnt support none + case .schema: + try ForwardOffset.verify( + &verifier, at: pos, of: org_apache_arrow_flatbuf_Schema.self) + case .dictionarybatch: + try ForwardOffset.verify( + &verifier, at: pos, of: org_apache_arrow_flatbuf_DictionaryBatch.self) + case .recordbatch: + try ForwardOffset.verify( + &verifier, at: pos, of: org_apache_arrow_flatbuf_RecordBatch.self) + case .tensor: + try ForwardOffset.verify( + &verifier, at: pos, of: org_apache_arrow_flatbuf_Tensor.self) + case .sparsetensor: + try ForwardOffset.verify( + &verifier, at: pos, of: org_apache_arrow_flatbuf_SparseTensor.self) + } + }) + try _v.visit( + field: VTOFFSET.bodyLength.p, fieldName: "bodyLength", required: false, type: Int64.self) + try _v.visit( + field: VTOFFSET.customMetadata.p, fieldName: "customMetadata", required: false, + type: ForwardOffset< + Vector, org_apache_arrow_flatbuf_KeyValue> + >.self) + _v.finish() + } +} diff --git a/Sources/SparkConnect/ProtoUtil.swift b/Sources/SparkConnect/ProtoUtil.swift new file mode 100644 index 0000000..ace45a9 --- /dev/null +++ b/Sources/SparkConnect/ProtoUtil.swift @@ -0,0 +1,72 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +import Foundation + +func fromProto( // swiftlint:disable:this cyclomatic_complexity + field: org_apache_arrow_flatbuf_Field +) -> ArrowField { + let type = field.typeType + var arrowType = ArrowType(ArrowType.ArrowUnknown) + switch type { + case .int: + let intType = field.type(type: org_apache_arrow_flatbuf_Int.self)! + let bitWidth = intType.bitWidth + if bitWidth == 8 { + arrowType = ArrowType(intType.isSigned ? ArrowType.ArrowInt8 : ArrowType.ArrowUInt8) + } else if bitWidth == 16 { + arrowType = ArrowType(intType.isSigned ? ArrowType.ArrowInt16 : ArrowType.ArrowUInt16) + } else if bitWidth == 32 { + arrowType = ArrowType(intType.isSigned ? ArrowType.ArrowInt32 : ArrowType.ArrowUInt32) + } else if bitWidth == 64 { + arrowType = ArrowType(intType.isSigned ? ArrowType.ArrowInt64 : ArrowType.ArrowUInt64) + } + case .bool: + arrowType = ArrowType(ArrowType.ArrowBool) + case .floatingpoint: + let floatType = field.type(type: org_apache_arrow_flatbuf_FloatingPoint.self)! + if floatType.precision == .single { + arrowType = ArrowType(ArrowType.ArrowFloat) + } else if floatType.precision == .double { + arrowType = ArrowType(ArrowType.ArrowDouble) + } + case .utf8: + arrowType = ArrowType(ArrowType.ArrowString) + case .binary: + arrowType = ArrowType(ArrowType.ArrowBinary) + case .date: + let dateType = field.type(type: org_apache_arrow_flatbuf_Date.self)! + if dateType.unit == .day { + arrowType = ArrowType(ArrowType.ArrowDate32) + } else { + arrowType = ArrowType(ArrowType.ArrowDate64) + } + case .time: + let timeType = field.type(type: org_apache_arrow_flatbuf_Time.self)! + if timeType.unit == .second || timeType.unit == .millisecond { + let arrowUnit: ArrowTime32Unit = timeType.unit == .second ? .seconds : .milliseconds + arrowType = ArrowTypeTime32(arrowUnit) + } else { + let arrowUnit: ArrowTime64Unit = timeType.unit == .microsecond ? .microseconds : .nanoseconds + arrowType = ArrowTypeTime64(arrowUnit) + } + default: + arrowType = ArrowType(ArrowType.ArrowUnknown) + } + + return ArrowField(field.name ?? "", type: arrowType, isNullable: field.nullable) +} diff --git a/Sources/SparkConnect/Schema_generated.swift b/Sources/SparkConnect/Schema_generated.swift new file mode 100644 index 0000000..1889866 --- /dev/null +++ b/Sources/SparkConnect/Schema_generated.swift @@ -0,0 +1,2113 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +// automatically generated by the FlatBuffers compiler, do not modify +// swiftlint:disable all +// swiftformat:disable all + +import FlatBuffers + +public enum org_apache_arrow_flatbuf_MetadataVersion: Int16, Enum, Verifiable { + public typealias T = Int16 + public static var byteSize: Int { return MemoryLayout.size } + public var value: Int16 { return self.rawValue } + /// 0.1.0 (October 2016). + case v1 = 0 + /// 0.2.0 (February 2017). Non-backwards compatible with V1. + case v2 = 1 + /// 0.3.0 -> 0.7.1 (May - December 2017). Non-backwards compatible with V2. + case v3 = 2 + /// >= 0.8.0 (December 2017). Non-backwards compatible with V3. + case v4 = 3 + /// >= 1.0.0 (July 2020. Backwards compatible with V4 (V5 readers can read V4 + /// metadata and IPC messages). Implementations are recommended to provide a + /// V4 compatibility mode with V5 format changes disabled. + /// + /// Incompatible changes between V4 and V5: + /// - Union buffer layout has changed. In V5, Unions don't have a validity + /// bitmap buffer. + case v5 = 4 + + public static var max: org_apache_arrow_flatbuf_MetadataVersion { return .v5 } + public static var min: org_apache_arrow_flatbuf_MetadataVersion { return .v1 } +} + +/// Represents Arrow Features that might not have full support +/// within implementations. This is intended to be used in +/// two scenarios: +/// 1. A mechanism for readers of Arrow Streams +/// and files to understand that the stream or file makes +/// use of a feature that isn't supported or unknown to +/// the implementation (and therefore can meet the Arrow +/// forward compatibility guarantees). +/// 2. A means of negotiating between a client and server +/// what features a stream is allowed to use. The enums +/// values here are intented to represent higher level +/// features, additional details maybe negotiated +/// with key-value pairs specific to the protocol. +/// +/// Enums added to this list should be assigned power-of-two values +/// to facilitate exchanging and comparing bitmaps for supported +/// features. +public enum org_apache_arrow_flatbuf_Feature: Int64, Enum, Verifiable { + public typealias T = Int64 + public static var byteSize: Int { return MemoryLayout.size } + public var value: Int64 { return self.rawValue } + /// Needed to make flatbuffers happy. + case unused = 0 + /// The stream makes use of multiple full dictionaries with the + /// same ID and assumes clients implement dictionary replacement + /// correctly. + case dictionaryReplacement = 1 + /// The stream makes use of compressed bodies as described + /// in Message.fbs. + case compressedBody = 2 + + public static var max: org_apache_arrow_flatbuf_Feature { return .compressedBody } + public static var min: org_apache_arrow_flatbuf_Feature { return .unused } +} + +public enum org_apache_arrow_flatbuf_UnionMode: Int16, Enum, Verifiable { + public typealias T = Int16 + public static var byteSize: Int { return MemoryLayout.size } + public var value: Int16 { return self.rawValue } + case sparse = 0 + case dense = 1 + + public static var max: org_apache_arrow_flatbuf_UnionMode { return .dense } + public static var min: org_apache_arrow_flatbuf_UnionMode { return .sparse } +} + +public enum org_apache_arrow_flatbuf_Precision: Int16, Enum, Verifiable { + public typealias T = Int16 + public static var byteSize: Int { return MemoryLayout.size } + public var value: Int16 { return self.rawValue } + case half = 0 + case single = 1 + case double = 2 + + public static var max: org_apache_arrow_flatbuf_Precision { return .double } + public static var min: org_apache_arrow_flatbuf_Precision { return .half } +} + +public enum org_apache_arrow_flatbuf_DateUnit: Int16, Enum, Verifiable { + public typealias T = Int16 + public static var byteSize: Int { return MemoryLayout.size } + public var value: Int16 { return self.rawValue } + case day = 0 + case millisecond = 1 + + public static var max: org_apache_arrow_flatbuf_DateUnit { return .millisecond } + public static var min: org_apache_arrow_flatbuf_DateUnit { return .day } +} + +public enum org_apache_arrow_flatbuf_TimeUnit: Int16, Enum, Verifiable { + public typealias T = Int16 + public static var byteSize: Int { return MemoryLayout.size } + public var value: Int16 { return self.rawValue } + case second = 0 + case millisecond = 1 + case microsecond = 2 + case nanosecond = 3 + + public static var max: org_apache_arrow_flatbuf_TimeUnit { return .nanosecond } + public static var min: org_apache_arrow_flatbuf_TimeUnit { return .second } +} + +public enum org_apache_arrow_flatbuf_IntervalUnit: Int16, Enum, Verifiable { + public typealias T = Int16 + public static var byteSize: Int { return MemoryLayout.size } + public var value: Int16 { return self.rawValue } + case yearMonth = 0 + case dayTime = 1 + case monthDayNano = 2 + + public static var max: org_apache_arrow_flatbuf_IntervalUnit { return .monthDayNano } + public static var min: org_apache_arrow_flatbuf_IntervalUnit { return .yearMonth } +} + +/// ---------------------------------------------------------------------- +/// Top-level Type value, enabling extensible type-specific metadata. We can +/// add new logical types to Type without breaking backwards compatibility +public enum org_apache_arrow_flatbuf_Type_: UInt8, UnionEnum { + public typealias T = UInt8 + + public init?(value: T) { + self.init(rawValue: value) + } + + public static var byteSize: Int { return MemoryLayout.size } + public var value: UInt8 { return self.rawValue } + case none_ = 0 + case null = 1 + case int = 2 + case floatingpoint = 3 + case binary = 4 + case utf8 = 5 + case bool = 6 + case decimal = 7 + case date = 8 + case time = 9 + case timestamp = 10 + case interval = 11 + case list = 12 + case struct_ = 13 + case union = 14 + case fixedsizebinary = 15 + case fixedsizelist = 16 + case map = 17 + case duration = 18 + case largebinary = 19 + case largeutf8 = 20 + case largelist = 21 + case runendencoded = 22 + + public static var max: org_apache_arrow_flatbuf_Type_ { return .runendencoded } + public static var min: org_apache_arrow_flatbuf_Type_ { return .none_ } +} + +/// ---------------------------------------------------------------------- +/// Dictionary encoding metadata +/// Maintained for forwards compatibility, in the future +/// Dictionaries might be explicit maps between integers and values +/// allowing for non-contiguous index values +public enum org_apache_arrow_flatbuf_DictionaryKind: Int16, Enum, Verifiable { + public typealias T = Int16 + public static var byteSize: Int { return MemoryLayout.size } + public var value: Int16 { return self.rawValue } + case densearray = 0 + + public static var max: org_apache_arrow_flatbuf_DictionaryKind { return .densearray } + public static var min: org_apache_arrow_flatbuf_DictionaryKind { return .densearray } +} + +/// ---------------------------------------------------------------------- +/// Endianness of the platform producing the data +public enum org_apache_arrow_flatbuf_Endianness: Int16, Enum, Verifiable { + public typealias T = Int16 + public static var byteSize: Int { return MemoryLayout.size } + public var value: Int16 { return self.rawValue } + case little = 0 + case big = 1 + + public static var max: org_apache_arrow_flatbuf_Endianness { return .big } + public static var min: org_apache_arrow_flatbuf_Endianness { return .little } +} + +/// ---------------------------------------------------------------------- +/// A Buffer represents a single contiguous memory segment +public struct org_apache_arrow_flatbuf_Buffer: NativeStruct, Verifiable, FlatbuffersInitializable { + + static func validateVersion() { FlatBuffersVersion_23_1_4() } + + /// The relative offset into the shared memory page where the bytes for this + /// buffer starts + private var _offset: Int64 + /// The absolute length (in bytes) of the memory buffer. The memory is found + /// from offset (inclusive) to offset + length (non-inclusive). When building + /// messages using the encapsulated IPC message, padding bytes may be written + /// after a buffer, but such padding bytes do not need to be accounted for in + /// the size here. + private var _length: Int64 + + public init(_ bb: ByteBuffer, o: Int32) { + let _accessor = Struct(bb: bb, position: o) + _offset = _accessor.readBuffer(of: Int64.self, at: 0) + _length = _accessor.readBuffer(of: Int64.self, at: 8) + } + + public init(offset: Int64, length: Int64) { + _offset = offset + _length = length + } + + public init() { + _offset = 0 + _length = 0 + } + + /// The relative offset into the shared memory page where the bytes for this + /// buffer starts + public var offset: Int64 { _offset } + /// The absolute length (in bytes) of the memory buffer. The memory is found + /// from offset (inclusive) to offset + length (non-inclusive). When building + /// messages using the encapsulated IPC message, padding bytes may be written + /// after a buffer, but such padding bytes do not need to be accounted for in + /// the size here. + public var length: Int64 { _length } + + public static func verify(_ verifier: inout Verifier, at position: Int, of type: T.Type) throws + where T: Verifiable { + try verifier.inBuffer(position: position, of: org_apache_arrow_flatbuf_Buffer.self) + } +} + +/// ---------------------------------------------------------------------- +/// A Buffer represents a single contiguous memory segment +public struct org_apache_arrow_flatbuf_Buffer_Mutable: FlatBufferObject { + + static func validateVersion() { FlatBuffersVersion_23_1_4() } + public var __buffer: ByteBuffer! { return _accessor.bb } + private var _accessor: Struct + + public init(_ bb: ByteBuffer, o: Int32) { _accessor = Struct(bb: bb, position: o) } + + public var offset: Int64 { return _accessor.readBuffer(of: Int64.self, at: 0) } + public var length: Int64 { return _accessor.readBuffer(of: Int64.self, at: 8) } +} + +/// These are stored in the flatbuffer in the Type union below +public struct org_apache_arrow_flatbuf_Null: FlatBufferObject, Verifiable { + + static func validateVersion() { FlatBuffersVersion_23_1_4() } + public var __buffer: ByteBuffer! { return _accessor.bb } + private var _accessor: Table + + public static func getRootAsNull(bb: ByteBuffer) -> org_apache_arrow_flatbuf_Null { + return org_apache_arrow_flatbuf_Null( + Table( + bb: bb, position: Int32(bb.read(def: UOffset.self, position: bb.reader)) + Int32(bb.reader)) + ) + } + + private init(_ t: Table) { _accessor = t } + public init(_ bb: ByteBuffer, o: Int32) { _accessor = Table(bb: bb, position: o) } + + public static func startNull(_ fbb: inout FlatBufferBuilder) -> UOffset { + fbb.startTable(with: 0) + } + public static func endNull(_ fbb: inout FlatBufferBuilder, start: UOffset) -> Offset { + let end = Offset(offset: fbb.endTable(at: start)) + return end + } + + public static func verify(_ verifier: inout Verifier, at position: Int, of type: T.Type) throws + where T: Verifiable { + var _v = try verifier.visitTable(at: position) + _v.finish() + } +} + +/// A Struct_ in the flatbuffer metadata is the same as an Arrow Struct +/// (according to the physical memory layout). We used Struct_ here as +/// Struct is a reserved word in Flatbuffers +public struct org_apache_arrow_flatbuf_Struct_: FlatBufferObject, Verifiable { + + static func validateVersion() { FlatBuffersVersion_23_1_4() } + public var __buffer: ByteBuffer! { return _accessor.bb } + private var _accessor: Table + + public static func getRootAsStruct_(bb: ByteBuffer) -> org_apache_arrow_flatbuf_Struct_ { + return org_apache_arrow_flatbuf_Struct_( + Table( + bb: bb, position: Int32(bb.read(def: UOffset.self, position: bb.reader)) + Int32(bb.reader)) + ) + } + + private init(_ t: Table) { _accessor = t } + public init(_ bb: ByteBuffer, o: Int32) { _accessor = Table(bb: bb, position: o) } + + public static func startStruct_(_ fbb: inout FlatBufferBuilder) -> UOffset { + fbb.startTable(with: 0) + } + public static func endStruct_(_ fbb: inout FlatBufferBuilder, start: UOffset) -> Offset { + let end = Offset(offset: fbb.endTable(at: start)) + return end + } + + public static func verify(_ verifier: inout Verifier, at position: Int, of type: T.Type) throws + where T: Verifiable { + var _v = try verifier.visitTable(at: position) + _v.finish() + } +} + +public struct org_apache_arrow_flatbuf_List: FlatBufferObject, Verifiable { + + static func validateVersion() { FlatBuffersVersion_23_1_4() } + public var __buffer: ByteBuffer! { return _accessor.bb } + private var _accessor: Table + + public static func getRootAsList(bb: ByteBuffer) -> org_apache_arrow_flatbuf_List { + return org_apache_arrow_flatbuf_List( + Table( + bb: bb, position: Int32(bb.read(def: UOffset.self, position: bb.reader)) + Int32(bb.reader)) + ) + } + + private init(_ t: Table) { _accessor = t } + public init(_ bb: ByteBuffer, o: Int32) { _accessor = Table(bb: bb, position: o) } + + public static func startList(_ fbb: inout FlatBufferBuilder) -> UOffset { + fbb.startTable(with: 0) + } + public static func endList(_ fbb: inout FlatBufferBuilder, start: UOffset) -> Offset { + let end = Offset(offset: fbb.endTable(at: start)) + return end + } + + public static func verify(_ verifier: inout Verifier, at position: Int, of type: T.Type) throws + where T: Verifiable { + var _v = try verifier.visitTable(at: position) + _v.finish() + } +} + +/// Same as List, but with 64-bit offsets, allowing to represent +/// extremely large data values. +public struct org_apache_arrow_flatbuf_LargeList: FlatBufferObject, Verifiable { + + static func validateVersion() { FlatBuffersVersion_23_1_4() } + public var __buffer: ByteBuffer! { return _accessor.bb } + private var _accessor: Table + + public static func getRootAsLargeList(bb: ByteBuffer) -> org_apache_arrow_flatbuf_LargeList { + return org_apache_arrow_flatbuf_LargeList( + Table( + bb: bb, position: Int32(bb.read(def: UOffset.self, position: bb.reader)) + Int32(bb.reader)) + ) + } + + private init(_ t: Table) { _accessor = t } + public init(_ bb: ByteBuffer, o: Int32) { _accessor = Table(bb: bb, position: o) } + + public static func startLargeList(_ fbb: inout FlatBufferBuilder) -> UOffset { + fbb.startTable(with: 0) + } + public static func endLargeList(_ fbb: inout FlatBufferBuilder, start: UOffset) -> Offset { + let end = Offset(offset: fbb.endTable(at: start)) + return end + } + + public static func verify(_ verifier: inout Verifier, at position: Int, of type: T.Type) throws + where T: Verifiable { + var _v = try verifier.visitTable(at: position) + _v.finish() + } +} + +public struct org_apache_arrow_flatbuf_FixedSizeList: FlatBufferObject, Verifiable { + + static func validateVersion() { FlatBuffersVersion_23_1_4() } + public var __buffer: ByteBuffer! { return _accessor.bb } + private var _accessor: Table + + public static func getRootAsFixedSizeList(bb: ByteBuffer) + -> org_apache_arrow_flatbuf_FixedSizeList + { + return org_apache_arrow_flatbuf_FixedSizeList( + Table( + bb: bb, position: Int32(bb.read(def: UOffset.self, position: bb.reader)) + Int32(bb.reader)) + ) + } + + private init(_ t: Table) { _accessor = t } + public init(_ bb: ByteBuffer, o: Int32) { _accessor = Table(bb: bb, position: o) } + + private enum VTOFFSET: VOffset { + case listSize = 4 + var v: Int32 { Int32(self.rawValue) } + var p: VOffset { self.rawValue } + } + + /// Number of list items per value + public var listSize: Int32 { + let o = _accessor.offset(VTOFFSET.listSize.v) + return o == 0 ? 0 : _accessor.readBuffer(of: Int32.self, at: o) + } + public static func startFixedSizeList(_ fbb: inout FlatBufferBuilder) -> UOffset { + fbb.startTable(with: 1) + } + public static func add(listSize: Int32, _ fbb: inout FlatBufferBuilder) { + fbb.add(element: listSize, def: 0, at: VTOFFSET.listSize.p) + } + public static func endFixedSizeList(_ fbb: inout FlatBufferBuilder, start: UOffset) -> Offset { + let end = Offset(offset: fbb.endTable(at: start)) + return end + } + public static func createFixedSizeList( + _ fbb: inout FlatBufferBuilder, + listSize: Int32 = 0 + ) -> Offset { + let __start = org_apache_arrow_flatbuf_FixedSizeList.startFixedSizeList(&fbb) + org_apache_arrow_flatbuf_FixedSizeList.add(listSize: listSize, &fbb) + return org_apache_arrow_flatbuf_FixedSizeList.endFixedSizeList(&fbb, start: __start) + } + + public static func verify(_ verifier: inout Verifier, at position: Int, of type: T.Type) throws + where T: Verifiable { + var _v = try verifier.visitTable(at: position) + try _v.visit( + field: VTOFFSET.listSize.p, fieldName: "listSize", required: false, type: Int32.self) + _v.finish() + } +} + +/// A Map is a logical nested type that is represented as +/// +/// List> +/// +/// In this layout, the keys and values are each respectively contiguous. We do +/// not constrain the key and value types, so the application is responsible +/// for ensuring that the keys are hashable and unique. Whether the keys are sorted +/// may be set in the metadata for this field. +/// +/// In a field with Map type, the field has a child Struct field, which then +/// has two children: key type and the second the value type. The names of the +/// child fields may be respectively "entries", "key", and "value", but this is +/// not enforced. +/// +/// Map +/// ```text +/// - child[0] entries: Struct +/// - child[0] key: K +/// - child[1] value: V +/// ``` +/// Neither the "entries" field nor the "key" field may be nullable. +/// +/// The metadata is structured so that Arrow systems without special handling +/// for Map can make Map an alias for List. The "layout" attribute for the Map +/// field must have the same contents as a List. +public struct org_apache_arrow_flatbuf_Map: FlatBufferObject, Verifiable { + + static func validateVersion() { FlatBuffersVersion_23_1_4() } + public var __buffer: ByteBuffer! { return _accessor.bb } + private var _accessor: Table + + public static func getRootAsMap(bb: ByteBuffer) -> org_apache_arrow_flatbuf_Map { + return org_apache_arrow_flatbuf_Map( + Table( + bb: bb, position: Int32(bb.read(def: UOffset.self, position: bb.reader)) + Int32(bb.reader)) + ) + } + + private init(_ t: Table) { _accessor = t } + public init(_ bb: ByteBuffer, o: Int32) { _accessor = Table(bb: bb, position: o) } + + private enum VTOFFSET: VOffset { + case keysSorted = 4 + var v: Int32 { Int32(self.rawValue) } + var p: VOffset { self.rawValue } + } + + /// Set to true if the keys within each value are sorted + public var keysSorted: Bool { + let o = _accessor.offset(VTOFFSET.keysSorted.v) + return o == 0 ? false : 0 != _accessor.readBuffer(of: Byte.self, at: o) + } + public static func startMap(_ fbb: inout FlatBufferBuilder) -> UOffset { fbb.startTable(with: 1) } + public static func add(keysSorted: Bool, _ fbb: inout FlatBufferBuilder) { + fbb.add( + element: keysSorted, def: false, + at: VTOFFSET.keysSorted.p) + } + public static func endMap(_ fbb: inout FlatBufferBuilder, start: UOffset) -> Offset { + let end = Offset(offset: fbb.endTable(at: start)) + return end + } + public static func createMap( + _ fbb: inout FlatBufferBuilder, + keysSorted: Bool = false + ) -> Offset { + let __start = org_apache_arrow_flatbuf_Map.startMap(&fbb) + org_apache_arrow_flatbuf_Map.add(keysSorted: keysSorted, &fbb) + return org_apache_arrow_flatbuf_Map.endMap(&fbb, start: __start) + } + + public static func verify(_ verifier: inout Verifier, at position: Int, of type: T.Type) throws + where T: Verifiable { + var _v = try verifier.visitTable(at: position) + try _v.visit( + field: VTOFFSET.keysSorted.p, fieldName: "keysSorted", required: false, type: Bool.self) + _v.finish() + } +} + +/// A union is a complex type with children in Field +/// By default ids in the type vector refer to the offsets in the children +/// optionally typeIds provides an indirection between the child offset and the type id +/// for each child `typeIds[offset]` is the id used in the type vector +public struct org_apache_arrow_flatbuf_Union: FlatBufferObject, Verifiable { + + static func validateVersion() { FlatBuffersVersion_23_1_4() } + public var __buffer: ByteBuffer! { return _accessor.bb } + private var _accessor: Table + + public static func getRootAsUnion(bb: ByteBuffer) -> org_apache_arrow_flatbuf_Union { + return org_apache_arrow_flatbuf_Union( + Table( + bb: bb, position: Int32(bb.read(def: UOffset.self, position: bb.reader)) + Int32(bb.reader)) + ) + } + + private init(_ t: Table) { _accessor = t } + public init(_ bb: ByteBuffer, o: Int32) { _accessor = Table(bb: bb, position: o) } + + private enum VTOFFSET: VOffset { + case mode = 4 + case typeIds = 6 + var v: Int32 { Int32(self.rawValue) } + var p: VOffset { self.rawValue } + } + + public var mode: org_apache_arrow_flatbuf_UnionMode { + let o = _accessor.offset(VTOFFSET.mode.v) + return o == 0 + ? .sparse + : org_apache_arrow_flatbuf_UnionMode(rawValue: _accessor.readBuffer(of: Int16.self, at: o)) + ?? .sparse + } + public var hasTypeIds: Bool { + let o = _accessor.offset(VTOFFSET.typeIds.v) + return o == 0 ? false : true + } + public var typeIdsCount: Int32 { + let o = _accessor.offset(VTOFFSET.typeIds.v) + return o == 0 ? 0 : _accessor.vector(count: o) + } + public func typeIds(at index: Int32) -> Int32 { + let o = _accessor.offset(VTOFFSET.typeIds.v) + return o == 0 + ? 0 : _accessor.directRead(of: Int32.self, offset: _accessor.vector(at: o) + index * 4) + } + public var typeIds: [Int32] { return _accessor.getVector(at: VTOFFSET.typeIds.v) ?? [] } + public static func startUnion(_ fbb: inout FlatBufferBuilder) -> UOffset { + fbb.startTable(with: 2) + } + public static func add(mode: org_apache_arrow_flatbuf_UnionMode, _ fbb: inout FlatBufferBuilder) { + fbb.add(element: mode.rawValue, def: 0, at: VTOFFSET.mode.p) + } + public static func addVectorOf(typeIds: Offset, _ fbb: inout FlatBufferBuilder) { + fbb.add(offset: typeIds, at: VTOFFSET.typeIds.p) + } + public static func endUnion(_ fbb: inout FlatBufferBuilder, start: UOffset) -> Offset { + let end = Offset(offset: fbb.endTable(at: start)) + return end + } + public static func createUnion( + _ fbb: inout FlatBufferBuilder, + mode: org_apache_arrow_flatbuf_UnionMode = .sparse, + typeIdsVectorOffset typeIds: Offset = Offset() + ) -> Offset { + let __start = org_apache_arrow_flatbuf_Union.startUnion(&fbb) + org_apache_arrow_flatbuf_Union.add(mode: mode, &fbb) + org_apache_arrow_flatbuf_Union.addVectorOf(typeIds: typeIds, &fbb) + return org_apache_arrow_flatbuf_Union.endUnion(&fbb, start: __start) + } + + public static func verify(_ verifier: inout Verifier, at position: Int, of type: T.Type) throws + where T: Verifiable { + var _v = try verifier.visitTable(at: position) + try _v.visit( + field: VTOFFSET.mode.p, fieldName: "mode", required: false, + type: org_apache_arrow_flatbuf_UnionMode.self) + try _v.visit( + field: VTOFFSET.typeIds.p, fieldName: "typeIds", required: false, + type: ForwardOffset>.self) + _v.finish() + } +} + +public struct org_apache_arrow_flatbuf_Int: FlatBufferObject, Verifiable { + + static func validateVersion() { FlatBuffersVersion_23_1_4() } + public var __buffer: ByteBuffer! { return _accessor.bb } + private var _accessor: Table + + public static func getRootAsInt(bb: ByteBuffer) -> org_apache_arrow_flatbuf_Int { + return org_apache_arrow_flatbuf_Int( + Table( + bb: bb, position: Int32(bb.read(def: UOffset.self, position: bb.reader)) + Int32(bb.reader)) + ) + } + + private init(_ t: Table) { _accessor = t } + public init(_ bb: ByteBuffer, o: Int32) { _accessor = Table(bb: bb, position: o) } + + private enum VTOFFSET: VOffset { + case bitWidth = 4 + case isSigned = 6 + var v: Int32 { Int32(self.rawValue) } + var p: VOffset { self.rawValue } + } + + public var bitWidth: Int32 { + let o = _accessor.offset(VTOFFSET.bitWidth.v) + return o == 0 ? 0 : _accessor.readBuffer(of: Int32.self, at: o) + } + public var isSigned: Bool { + let o = _accessor.offset(VTOFFSET.isSigned.v) + return o == 0 ? false : 0 != _accessor.readBuffer(of: Byte.self, at: o) + } + public static func startInt(_ fbb: inout FlatBufferBuilder) -> UOffset { fbb.startTable(with: 2) } + public static func add(bitWidth: Int32, _ fbb: inout FlatBufferBuilder) { + fbb.add(element: bitWidth, def: 0, at: VTOFFSET.bitWidth.p) + } + public static func add(isSigned: Bool, _ fbb: inout FlatBufferBuilder) { + fbb.add( + element: isSigned, def: false, + at: VTOFFSET.isSigned.p) + } + public static func endInt(_ fbb: inout FlatBufferBuilder, start: UOffset) -> Offset { + let end = Offset(offset: fbb.endTable(at: start)) + return end + } + public static func createInt( + _ fbb: inout FlatBufferBuilder, + bitWidth: Int32 = 0, + isSigned: Bool = false + ) -> Offset { + let __start = org_apache_arrow_flatbuf_Int.startInt(&fbb) + org_apache_arrow_flatbuf_Int.add(bitWidth: bitWidth, &fbb) + org_apache_arrow_flatbuf_Int.add(isSigned: isSigned, &fbb) + return org_apache_arrow_flatbuf_Int.endInt(&fbb, start: __start) + } + + public static func verify(_ verifier: inout Verifier, at position: Int, of type: T.Type) throws + where T: Verifiable { + var _v = try verifier.visitTable(at: position) + try _v.visit( + field: VTOFFSET.bitWidth.p, fieldName: "bitWidth", required: false, type: Int32.self) + try _v.visit( + field: VTOFFSET.isSigned.p, fieldName: "isSigned", required: false, type: Bool.self) + _v.finish() + } +} + +public struct org_apache_arrow_flatbuf_FloatingPoint: FlatBufferObject, Verifiable { + + static func validateVersion() { FlatBuffersVersion_23_1_4() } + public var __buffer: ByteBuffer! { return _accessor.bb } + private var _accessor: Table + + public static func getRootAsFloatingPoint(bb: ByteBuffer) + -> org_apache_arrow_flatbuf_FloatingPoint + { + return org_apache_arrow_flatbuf_FloatingPoint( + Table( + bb: bb, position: Int32(bb.read(def: UOffset.self, position: bb.reader)) + Int32(bb.reader)) + ) + } + + private init(_ t: Table) { _accessor = t } + public init(_ bb: ByteBuffer, o: Int32) { _accessor = Table(bb: bb, position: o) } + + private enum VTOFFSET: VOffset { + case precision = 4 + var v: Int32 { Int32(self.rawValue) } + var p: VOffset { self.rawValue } + } + + public var precision: org_apache_arrow_flatbuf_Precision { + let o = _accessor.offset(VTOFFSET.precision.v) + return o == 0 + ? .half + : org_apache_arrow_flatbuf_Precision(rawValue: _accessor.readBuffer(of: Int16.self, at: o)) + ?? .half + } + public static func startFloatingPoint(_ fbb: inout FlatBufferBuilder) -> UOffset { + fbb.startTable(with: 1) + } + public static func add( + precision: org_apache_arrow_flatbuf_Precision, _ fbb: inout FlatBufferBuilder + ) { fbb.add(element: precision.rawValue, def: 0, at: VTOFFSET.precision.p) } + public static func endFloatingPoint(_ fbb: inout FlatBufferBuilder, start: UOffset) -> Offset { + let end = Offset(offset: fbb.endTable(at: start)) + return end + } + public static func createFloatingPoint( + _ fbb: inout FlatBufferBuilder, + precision: org_apache_arrow_flatbuf_Precision = .half + ) -> Offset { + let __start = org_apache_arrow_flatbuf_FloatingPoint.startFloatingPoint(&fbb) + org_apache_arrow_flatbuf_FloatingPoint.add(precision: precision, &fbb) + return org_apache_arrow_flatbuf_FloatingPoint.endFloatingPoint(&fbb, start: __start) + } + + public static func verify(_ verifier: inout Verifier, at position: Int, of type: T.Type) throws + where T: Verifiable { + var _v = try verifier.visitTable(at: position) + try _v.visit( + field: VTOFFSET.precision.p, fieldName: "precision", required: false, + type: org_apache_arrow_flatbuf_Precision.self) + _v.finish() + } +} + +/// Unicode with UTF-8 encoding +public struct org_apache_arrow_flatbuf_Utf8: FlatBufferObject, Verifiable { + + static func validateVersion() { FlatBuffersVersion_23_1_4() } + public var __buffer: ByteBuffer! { return _accessor.bb } + private var _accessor: Table + + public static func getRootAsUtf8(bb: ByteBuffer) -> org_apache_arrow_flatbuf_Utf8 { + return org_apache_arrow_flatbuf_Utf8( + Table( + bb: bb, position: Int32(bb.read(def: UOffset.self, position: bb.reader)) + Int32(bb.reader)) + ) + } + + private init(_ t: Table) { _accessor = t } + public init(_ bb: ByteBuffer, o: Int32) { _accessor = Table(bb: bb, position: o) } + + public static func startUtf8(_ fbb: inout FlatBufferBuilder) -> UOffset { + fbb.startTable(with: 0) + } + public static func endUtf8(_ fbb: inout FlatBufferBuilder, start: UOffset) -> Offset { + let end = Offset(offset: fbb.endTable(at: start)) + return end + } + + public static func verify(_ verifier: inout Verifier, at position: Int, of type: T.Type) throws + where T: Verifiable { + var _v = try verifier.visitTable(at: position) + _v.finish() + } +} + +/// Opaque binary data +public struct org_apache_arrow_flatbuf_Binary: FlatBufferObject, Verifiable { + + static func validateVersion() { FlatBuffersVersion_23_1_4() } + public var __buffer: ByteBuffer! { return _accessor.bb } + private var _accessor: Table + + public static func getRootAsBinary(bb: ByteBuffer) -> org_apache_arrow_flatbuf_Binary { + return org_apache_arrow_flatbuf_Binary( + Table( + bb: bb, position: Int32(bb.read(def: UOffset.self, position: bb.reader)) + Int32(bb.reader)) + ) + } + + private init(_ t: Table) { _accessor = t } + public init(_ bb: ByteBuffer, o: Int32) { _accessor = Table(bb: bb, position: o) } + + public static func startBinary(_ fbb: inout FlatBufferBuilder) -> UOffset { + fbb.startTable(with: 0) + } + public static func endBinary(_ fbb: inout FlatBufferBuilder, start: UOffset) -> Offset { + let end = Offset(offset: fbb.endTable(at: start)) + return end + } + + public static func verify(_ verifier: inout Verifier, at position: Int, of type: T.Type) throws + where T: Verifiable { + var _v = try verifier.visitTable(at: position) + _v.finish() + } +} + +/// Same as Utf8, but with 64-bit offsets, allowing to represent +/// extremely large data values. +public struct org_apache_arrow_flatbuf_LargeUtf8: FlatBufferObject, Verifiable { + + static func validateVersion() { FlatBuffersVersion_23_1_4() } + public var __buffer: ByteBuffer! { return _accessor.bb } + private var _accessor: Table + + public static func getRootAsLargeUtf8(bb: ByteBuffer) -> org_apache_arrow_flatbuf_LargeUtf8 { + return org_apache_arrow_flatbuf_LargeUtf8( + Table( + bb: bb, position: Int32(bb.read(def: UOffset.self, position: bb.reader)) + Int32(bb.reader)) + ) + } + + private init(_ t: Table) { _accessor = t } + public init(_ bb: ByteBuffer, o: Int32) { _accessor = Table(bb: bb, position: o) } + + public static func startLargeUtf8(_ fbb: inout FlatBufferBuilder) -> UOffset { + fbb.startTable(with: 0) + } + public static func endLargeUtf8(_ fbb: inout FlatBufferBuilder, start: UOffset) -> Offset { + let end = Offset(offset: fbb.endTable(at: start)) + return end + } + + public static func verify(_ verifier: inout Verifier, at position: Int, of type: T.Type) throws + where T: Verifiable { + var _v = try verifier.visitTable(at: position) + _v.finish() + } +} + +/// Same as Binary, but with 64-bit offsets, allowing to represent +/// extremely large data values. +public struct org_apache_arrow_flatbuf_LargeBinary: FlatBufferObject, Verifiable { + + static func validateVersion() { FlatBuffersVersion_23_1_4() } + public var __buffer: ByteBuffer! { return _accessor.bb } + private var _accessor: Table + + public static func getRootAsLargeBinary(bb: ByteBuffer) -> org_apache_arrow_flatbuf_LargeBinary { + return org_apache_arrow_flatbuf_LargeBinary( + Table( + bb: bb, position: Int32(bb.read(def: UOffset.self, position: bb.reader)) + Int32(bb.reader)) + ) + } + + private init(_ t: Table) { _accessor = t } + public init(_ bb: ByteBuffer, o: Int32) { _accessor = Table(bb: bb, position: o) } + + public static func startLargeBinary(_ fbb: inout FlatBufferBuilder) -> UOffset { + fbb.startTable(with: 0) + } + public static func endLargeBinary(_ fbb: inout FlatBufferBuilder, start: UOffset) -> Offset { + let end = Offset(offset: fbb.endTable(at: start)) + return end + } + + public static func verify(_ verifier: inout Verifier, at position: Int, of type: T.Type) throws + where T: Verifiable { + var _v = try verifier.visitTable(at: position) + _v.finish() + } +} + +public struct org_apache_arrow_flatbuf_FixedSizeBinary: FlatBufferObject, Verifiable { + + static func validateVersion() { FlatBuffersVersion_23_1_4() } + public var __buffer: ByteBuffer! { return _accessor.bb } + private var _accessor: Table + + public static func getRootAsFixedSizeBinary(bb: ByteBuffer) + -> org_apache_arrow_flatbuf_FixedSizeBinary + { + return org_apache_arrow_flatbuf_FixedSizeBinary( + Table( + bb: bb, position: Int32(bb.read(def: UOffset.self, position: bb.reader)) + Int32(bb.reader)) + ) + } + + private init(_ t: Table) { _accessor = t } + public init(_ bb: ByteBuffer, o: Int32) { _accessor = Table(bb: bb, position: o) } + + private enum VTOFFSET: VOffset { + case byteWidth = 4 + var v: Int32 { Int32(self.rawValue) } + var p: VOffset { self.rawValue } + } + + /// Number of bytes per value + public var byteWidth: Int32 { + let o = _accessor.offset(VTOFFSET.byteWidth.v) + return o == 0 ? 0 : _accessor.readBuffer(of: Int32.self, at: o) + } + public static func startFixedSizeBinary(_ fbb: inout FlatBufferBuilder) -> UOffset { + fbb.startTable(with: 1) + } + public static func add(byteWidth: Int32, _ fbb: inout FlatBufferBuilder) { + fbb.add(element: byteWidth, def: 0, at: VTOFFSET.byteWidth.p) + } + public static func endFixedSizeBinary(_ fbb: inout FlatBufferBuilder, start: UOffset) -> Offset { + let end = Offset(offset: fbb.endTable(at: start)) + return end + } + public static func createFixedSizeBinary( + _ fbb: inout FlatBufferBuilder, + byteWidth: Int32 = 0 + ) -> Offset { + let __start = org_apache_arrow_flatbuf_FixedSizeBinary.startFixedSizeBinary(&fbb) + org_apache_arrow_flatbuf_FixedSizeBinary.add(byteWidth: byteWidth, &fbb) + return org_apache_arrow_flatbuf_FixedSizeBinary.endFixedSizeBinary(&fbb, start: __start) + } + + public static func verify(_ verifier: inout Verifier, at position: Int, of type: T.Type) throws + where T: Verifiable { + var _v = try verifier.visitTable(at: position) + try _v.visit( + field: VTOFFSET.byteWidth.p, fieldName: "byteWidth", required: false, type: Int32.self) + _v.finish() + } +} + +public struct org_apache_arrow_flatbuf_Bool: FlatBufferObject, Verifiable { + + static func validateVersion() { FlatBuffersVersion_23_1_4() } + public var __buffer: ByteBuffer! { return _accessor.bb } + private var _accessor: Table + + public static func getRootAsBool(bb: ByteBuffer) -> org_apache_arrow_flatbuf_Bool { + return org_apache_arrow_flatbuf_Bool( + Table( + bb: bb, position: Int32(bb.read(def: UOffset.self, position: bb.reader)) + Int32(bb.reader)) + ) + } + + private init(_ t: Table) { _accessor = t } + public init(_ bb: ByteBuffer, o: Int32) { _accessor = Table(bb: bb, position: o) } + + public static func startBool(_ fbb: inout FlatBufferBuilder) -> UOffset { + fbb.startTable(with: 0) + } + public static func endBool(_ fbb: inout FlatBufferBuilder, start: UOffset) -> Offset { + let end = Offset(offset: fbb.endTable(at: start)) + return end + } + + public static func verify(_ verifier: inout Verifier, at position: Int, of type: T.Type) throws + where T: Verifiable { + var _v = try verifier.visitTable(at: position) + _v.finish() + } +} + +/// Contains two child arrays, run_ends and values. +/// The run_ends child array must be a 16/32/64-bit integer array +/// which encodes the indices at which the run with the value in +/// each corresponding index in the values child array ends. +/// Like list/struct types, the value array can be of any type. +public struct org_apache_arrow_flatbuf_RunEndEncoded: FlatBufferObject, Verifiable { + + static func validateVersion() { FlatBuffersVersion_23_1_4() } + public var __buffer: ByteBuffer! { return _accessor.bb } + private var _accessor: Table + + public static func getRootAsRunEndEncoded(bb: ByteBuffer) + -> org_apache_arrow_flatbuf_RunEndEncoded + { + return org_apache_arrow_flatbuf_RunEndEncoded( + Table( + bb: bb, position: Int32(bb.read(def: UOffset.self, position: bb.reader)) + Int32(bb.reader)) + ) + } + + private init(_ t: Table) { _accessor = t } + public init(_ bb: ByteBuffer, o: Int32) { _accessor = Table(bb: bb, position: o) } + + public static func startRunEndEncoded(_ fbb: inout FlatBufferBuilder) -> UOffset { + fbb.startTable(with: 0) + } + public static func endRunEndEncoded(_ fbb: inout FlatBufferBuilder, start: UOffset) -> Offset { + let end = Offset(offset: fbb.endTable(at: start)) + return end + } + + public static func verify(_ verifier: inout Verifier, at position: Int, of type: T.Type) throws + where T: Verifiable { + var _v = try verifier.visitTable(at: position) + _v.finish() + } +} + +/// Exact decimal value represented as an integer value in two's +/// complement. Currently only 128-bit (16-byte) and 256-bit (32-byte) integers +/// are used. The representation uses the endianness indicated +/// in the Schema. +public struct org_apache_arrow_flatbuf_Decimal: FlatBufferObject, Verifiable { + + static func validateVersion() { FlatBuffersVersion_23_1_4() } + public var __buffer: ByteBuffer! { return _accessor.bb } + private var _accessor: Table + + public static func getRootAsDecimal(bb: ByteBuffer) -> org_apache_arrow_flatbuf_Decimal { + return org_apache_arrow_flatbuf_Decimal( + Table( + bb: bb, position: Int32(bb.read(def: UOffset.self, position: bb.reader)) + Int32(bb.reader)) + ) + } + + private init(_ t: Table) { _accessor = t } + public init(_ bb: ByteBuffer, o: Int32) { _accessor = Table(bb: bb, position: o) } + + private enum VTOFFSET: VOffset { + case precision = 4 + case scale = 6 + case bitWidth = 8 + var v: Int32 { Int32(self.rawValue) } + var p: VOffset { self.rawValue } + } + + /// Total number of decimal digits + public var precision: Int32 { + let o = _accessor.offset(VTOFFSET.precision.v) + return o == 0 ? 0 : _accessor.readBuffer(of: Int32.self, at: o) + } + /// Number of digits after the decimal point "." + public var scale: Int32 { + let o = _accessor.offset(VTOFFSET.scale.v) + return o == 0 ? 0 : _accessor.readBuffer(of: Int32.self, at: o) + } + /// Number of bits per value. The only accepted widths are 128 and 256. + /// We use bitWidth for consistency with Int::bitWidth. + public var bitWidth: Int32 { + let o = _accessor.offset(VTOFFSET.bitWidth.v) + return o == 0 ? 128 : _accessor.readBuffer(of: Int32.self, at: o) + } + public static func startDecimal(_ fbb: inout FlatBufferBuilder) -> UOffset { + fbb.startTable(with: 3) + } + public static func add(precision: Int32, _ fbb: inout FlatBufferBuilder) { + fbb.add(element: precision, def: 0, at: VTOFFSET.precision.p) + } + public static func add(scale: Int32, _ fbb: inout FlatBufferBuilder) { + fbb.add(element: scale, def: 0, at: VTOFFSET.scale.p) + } + public static func add(bitWidth: Int32, _ fbb: inout FlatBufferBuilder) { + fbb.add(element: bitWidth, def: 128, at: VTOFFSET.bitWidth.p) + } + public static func endDecimal(_ fbb: inout FlatBufferBuilder, start: UOffset) -> Offset { + let end = Offset(offset: fbb.endTable(at: start)) + return end + } + public static func createDecimal( + _ fbb: inout FlatBufferBuilder, + precision: Int32 = 0, + scale: Int32 = 0, + bitWidth: Int32 = 128 + ) -> Offset { + let __start = org_apache_arrow_flatbuf_Decimal.startDecimal(&fbb) + org_apache_arrow_flatbuf_Decimal.add(precision: precision, &fbb) + org_apache_arrow_flatbuf_Decimal.add(scale: scale, &fbb) + org_apache_arrow_flatbuf_Decimal.add(bitWidth: bitWidth, &fbb) + return org_apache_arrow_flatbuf_Decimal.endDecimal(&fbb, start: __start) + } + + public static func verify(_ verifier: inout Verifier, at position: Int, of type: T.Type) throws + where T: Verifiable { + var _v = try verifier.visitTable(at: position) + try _v.visit( + field: VTOFFSET.precision.p, fieldName: "precision", required: false, type: Int32.self) + try _v.visit(field: VTOFFSET.scale.p, fieldName: "scale", required: false, type: Int32.self) + try _v.visit( + field: VTOFFSET.bitWidth.p, fieldName: "bitWidth", required: false, type: Int32.self) + _v.finish() + } +} + +/// Date is either a 32-bit or 64-bit signed integer type representing an +/// elapsed time since UNIX epoch (1970-01-01), stored in either of two units: +/// +/// * Milliseconds (64 bits) indicating UNIX time elapsed since the epoch (no +/// leap seconds), where the values are evenly divisible by 86400000 +/// * Days (32 bits) since the UNIX epoch +public struct org_apache_arrow_flatbuf_Date: FlatBufferObject, Verifiable { + + static func validateVersion() { FlatBuffersVersion_23_1_4() } + public var __buffer: ByteBuffer! { return _accessor.bb } + private var _accessor: Table + + public static func getRootAsDate(bb: ByteBuffer) -> org_apache_arrow_flatbuf_Date { + return org_apache_arrow_flatbuf_Date( + Table( + bb: bb, position: Int32(bb.read(def: UOffset.self, position: bb.reader)) + Int32(bb.reader)) + ) + } + + private init(_ t: Table) { _accessor = t } + public init(_ bb: ByteBuffer, o: Int32) { _accessor = Table(bb: bb, position: o) } + + private enum VTOFFSET: VOffset { + case unit = 4 + var v: Int32 { Int32(self.rawValue) } + var p: VOffset { self.rawValue } + } + + public var unit: org_apache_arrow_flatbuf_DateUnit { + let o = _accessor.offset(VTOFFSET.unit.v) + return o == 0 + ? .millisecond + : org_apache_arrow_flatbuf_DateUnit(rawValue: _accessor.readBuffer(of: Int16.self, at: o)) + ?? .millisecond + } + public static func startDate(_ fbb: inout FlatBufferBuilder) -> UOffset { + fbb.startTable(with: 1) + } + public static func add(unit: org_apache_arrow_flatbuf_DateUnit, _ fbb: inout FlatBufferBuilder) { + fbb.add(element: unit.rawValue, def: 1, at: VTOFFSET.unit.p) + } + public static func endDate(_ fbb: inout FlatBufferBuilder, start: UOffset) -> Offset { + let end = Offset(offset: fbb.endTable(at: start)) + return end + } + public static func createDate( + _ fbb: inout FlatBufferBuilder, + unit: org_apache_arrow_flatbuf_DateUnit = .millisecond + ) -> Offset { + let __start = org_apache_arrow_flatbuf_Date.startDate(&fbb) + org_apache_arrow_flatbuf_Date.add(unit: unit, &fbb) + return org_apache_arrow_flatbuf_Date.endDate(&fbb, start: __start) + } + + public static func verify(_ verifier: inout Verifier, at position: Int, of type: T.Type) throws + where T: Verifiable { + var _v = try verifier.visitTable(at: position) + try _v.visit( + field: VTOFFSET.unit.p, fieldName: "unit", required: false, + type: org_apache_arrow_flatbuf_DateUnit.self) + _v.finish() + } +} + +/// Time is either a 32-bit or 64-bit signed integer type representing an +/// elapsed time since midnight, stored in either of four units: seconds, +/// milliseconds, microseconds or nanoseconds. +/// +/// The integer `bitWidth` depends on the `unit` and must be one of the following: +/// * SECOND and MILLISECOND: 32 bits +/// * MICROSECOND and NANOSECOND: 64 bits +/// +/// The allowed values are between 0 (inclusive) and 86400 (=24*60*60) seconds +/// (exclusive), adjusted for the time unit (for example, up to 86400000 +/// exclusive for the MILLISECOND unit). +/// This definition doesn't allow for leap seconds. Time values from +/// measurements with leap seconds will need to be corrected when ingesting +/// into Arrow (for example by replacing the value 86400 with 86399). +public struct org_apache_arrow_flatbuf_Time: FlatBufferObject, Verifiable { + + static func validateVersion() { FlatBuffersVersion_23_1_4() } + public var __buffer: ByteBuffer! { return _accessor.bb } + private var _accessor: Table + + public static func getRootAsTime(bb: ByteBuffer) -> org_apache_arrow_flatbuf_Time { + return org_apache_arrow_flatbuf_Time( + Table( + bb: bb, position: Int32(bb.read(def: UOffset.self, position: bb.reader)) + Int32(bb.reader)) + ) + } + + private init(_ t: Table) { _accessor = t } + public init(_ bb: ByteBuffer, o: Int32) { _accessor = Table(bb: bb, position: o) } + + private enum VTOFFSET: VOffset { + case unit = 4 + case bitWidth = 6 + var v: Int32 { Int32(self.rawValue) } + var p: VOffset { self.rawValue } + } + + public var unit: org_apache_arrow_flatbuf_TimeUnit { + let o = _accessor.offset(VTOFFSET.unit.v) + return o == 0 + ? .millisecond + : org_apache_arrow_flatbuf_TimeUnit(rawValue: _accessor.readBuffer(of: Int16.self, at: o)) + ?? .millisecond + } + public var bitWidth: Int32 { + let o = _accessor.offset(VTOFFSET.bitWidth.v) + return o == 0 ? 32 : _accessor.readBuffer(of: Int32.self, at: o) + } + public static func startTime(_ fbb: inout FlatBufferBuilder) -> UOffset { + fbb.startTable(with: 2) + } + public static func add(unit: org_apache_arrow_flatbuf_TimeUnit, _ fbb: inout FlatBufferBuilder) { + fbb.add(element: unit.rawValue, def: 1, at: VTOFFSET.unit.p) + } + public static func add(bitWidth: Int32, _ fbb: inout FlatBufferBuilder) { + fbb.add(element: bitWidth, def: 32, at: VTOFFSET.bitWidth.p) + } + public static func endTime(_ fbb: inout FlatBufferBuilder, start: UOffset) -> Offset { + let end = Offset(offset: fbb.endTable(at: start)) + return end + } + public static func createTime( + _ fbb: inout FlatBufferBuilder, + unit: org_apache_arrow_flatbuf_TimeUnit = .millisecond, + bitWidth: Int32 = 32 + ) -> Offset { + let __start = org_apache_arrow_flatbuf_Time.startTime(&fbb) + org_apache_arrow_flatbuf_Time.add(unit: unit, &fbb) + org_apache_arrow_flatbuf_Time.add(bitWidth: bitWidth, &fbb) + return org_apache_arrow_flatbuf_Time.endTime(&fbb, start: __start) + } + + public static func verify(_ verifier: inout Verifier, at position: Int, of type: T.Type) throws + where T: Verifiable { + var _v = try verifier.visitTable(at: position) + try _v.visit( + field: VTOFFSET.unit.p, fieldName: "unit", required: false, + type: org_apache_arrow_flatbuf_TimeUnit.self) + try _v.visit( + field: VTOFFSET.bitWidth.p, fieldName: "bitWidth", required: false, type: Int32.self) + _v.finish() + } +} + +/// Timestamp is a 64-bit signed integer representing an elapsed time since a +/// fixed epoch, stored in either of four units: seconds, milliseconds, +/// microseconds or nanoseconds, and is optionally annotated with a timezone. +/// +/// Timestamp values do not include any leap seconds (in other words, all +/// days are considered 86400 seconds long). +/// +/// Timestamps with a non-empty timezone +/// ------------------------------------ +/// +/// If a Timestamp column has a non-empty timezone value, its epoch is +/// 1970-01-01 00:00:00 (January 1st 1970, midnight) in the *UTC* timezone +/// (the Unix epoch), regardless of the Timestamp's own timezone. +/// +/// Therefore, timestamp values with a non-empty timezone correspond to +/// physical points in time together with some additional information about +/// how the data was obtained and/or how to display it (the timezone). +/// +/// For example, the timestamp value 0 with the timezone string "Europe/Paris" +/// corresponds to "January 1st 1970, 00h00" in the UTC timezone, but the +/// application may prefer to display it as "January 1st 1970, 01h00" in +/// the Europe/Paris timezone (which is the same physical point in time). +/// +/// One consequence is that timestamp values with a non-empty timezone +/// can be compared and ordered directly, since they all share the same +/// well-known point of reference (the Unix epoch). +/// +/// Timestamps with an unset / empty timezone +/// ----------------------------------------- +/// +/// If a Timestamp column has no timezone value, its epoch is +/// 1970-01-01 00:00:00 (January 1st 1970, midnight) in an *unknown* timezone. +/// +/// Therefore, timestamp values without a timezone cannot be meaningfully +/// interpreted as physical points in time, but only as calendar / clock +/// indications ("wall clock time") in an unspecified timezone. +/// +/// For example, the timestamp value 0 with an empty timezone string +/// corresponds to "January 1st 1970, 00h00" in an unknown timezone: there +/// is not enough information to interpret it as a well-defined physical +/// point in time. +/// +/// One consequence is that timestamp values without a timezone cannot +/// be reliably compared or ordered, since they may have different points of +/// reference. In particular, it is *not* possible to interpret an unset +/// or empty timezone as the same as "UTC". +/// +/// Conversion between timezones +/// ---------------------------- +/// +/// If a Timestamp column has a non-empty timezone, changing the timezone +/// to a different non-empty value is a metadata-only operation: +/// the timestamp values need not change as their point of reference remains +/// the same (the Unix epoch). +/// +/// However, if a Timestamp column has no timezone value, changing it to a +/// non-empty value requires to think about the desired semantics. +/// One possibility is to assume that the original timestamp values are +/// relative to the epoch of the timezone being set; timestamp values should +/// then adjusted to the Unix epoch (for example, changing the timezone from +/// empty to "Europe/Paris" would require converting the timestamp values +/// from "Europe/Paris" to "UTC", which seems counter-intuitive but is +/// nevertheless correct). +/// +/// Guidelines for encoding data from external libraries +/// ---------------------------------------------------- +/// +/// Date & time libraries often have multiple different data types for temporal +/// data. In order to ease interoperability between different implementations the +/// Arrow project has some recommendations for encoding these types into a Timestamp +/// column. +/// +/// An "instant" represents a physical point in time that has no relevant timezone +/// (for example, astronomical data). To encode an instant, use a Timestamp with +/// the timezone string set to "UTC", and make sure the Timestamp values +/// are relative to the UTC epoch (January 1st 1970, midnight). +/// +/// A "zoned date-time" represents a physical point in time annotated with an +/// informative timezone (for example, the timezone in which the data was +/// recorded). To encode a zoned date-time, use a Timestamp with the timezone +/// string set to the name of the timezone, and make sure the Timestamp values +/// are relative to the UTC epoch (January 1st 1970, midnight). +/// +/// (There is some ambiguity between an instant and a zoned date-time with the +/// UTC timezone. Both of these are stored the same in Arrow. Typically, +/// this distinction does not matter. If it does, then an application should +/// use custom metadata or an extension type to distinguish between the two cases.) +/// +/// An "offset date-time" represents a physical point in time combined with an +/// explicit offset from UTC. To encode an offset date-time, use a Timestamp +/// with the timezone string set to the numeric timezone offset string +/// (e.g. "+03:00"), and make sure the Timestamp values are relative to +/// the UTC epoch (January 1st 1970, midnight). +/// +/// A "naive date-time" (also called "local date-time" in some libraries) +/// represents a wall clock time combined with a calendar date, but with +/// no indication of how to map this information to a physical point in time. +/// Naive date-times must be handled with care because of this missing +/// information, and also because daylight saving time (DST) may make +/// some values ambiguous or nonexistent. A naive date-time may be +/// stored as a struct with Date and Time fields. However, it may also be +/// encoded into a Timestamp column with an empty timezone. The timestamp +/// values should be computed "as if" the timezone of the date-time values +/// was UTC; for example, the naive date-time "January 1st 1970, 00h00" would +/// be encoded as timestamp value 0. +public struct org_apache_arrow_flatbuf_Timestamp: FlatBufferObject, Verifiable { + + static func validateVersion() { FlatBuffersVersion_23_1_4() } + public var __buffer: ByteBuffer! { return _accessor.bb } + private var _accessor: Table + + public static func getRootAsTimestamp(bb: ByteBuffer) -> org_apache_arrow_flatbuf_Timestamp { + return org_apache_arrow_flatbuf_Timestamp( + Table( + bb: bb, position: Int32(bb.read(def: UOffset.self, position: bb.reader)) + Int32(bb.reader)) + ) + } + + private init(_ t: Table) { _accessor = t } + public init(_ bb: ByteBuffer, o: Int32) { _accessor = Table(bb: bb, position: o) } + + private enum VTOFFSET: VOffset { + case unit = 4 + case timezone = 6 + var v: Int32 { Int32(self.rawValue) } + var p: VOffset { self.rawValue } + } + + public var unit: org_apache_arrow_flatbuf_TimeUnit { + let o = _accessor.offset(VTOFFSET.unit.v) + return o == 0 + ? .second + : org_apache_arrow_flatbuf_TimeUnit(rawValue: _accessor.readBuffer(of: Int16.self, at: o)) + ?? .second + } + /// The timezone is an optional string indicating the name of a timezone, + /// one of: + /// + /// * As used in the Olson timezone database (the "tz database" or + /// "tzdata"), such as "America/New_York". + /// * An absolute timezone offset of the form "+XX:XX" or "-XX:XX", + /// such as "+07:30". + /// + /// Whether a timezone string is present indicates different semantics about + /// the data (see above). + public var timezone: String? { + let o = _accessor.offset(VTOFFSET.timezone.v) + return o == 0 ? nil : _accessor.string(at: o) + } + public var timezoneSegmentArray: [UInt8]? { return _accessor.getVector(at: VTOFFSET.timezone.v) } + public static func startTimestamp(_ fbb: inout FlatBufferBuilder) -> UOffset { + fbb.startTable(with: 2) + } + public static func add(unit: org_apache_arrow_flatbuf_TimeUnit, _ fbb: inout FlatBufferBuilder) { + fbb.add(element: unit.rawValue, def: 0, at: VTOFFSET.unit.p) + } + public static func add(timezone: Offset, _ fbb: inout FlatBufferBuilder) { + fbb.add(offset: timezone, at: VTOFFSET.timezone.p) + } + public static func endTimestamp(_ fbb: inout FlatBufferBuilder, start: UOffset) -> Offset { + let end = Offset(offset: fbb.endTable(at: start)) + return end + } + public static func createTimestamp( + _ fbb: inout FlatBufferBuilder, + unit: org_apache_arrow_flatbuf_TimeUnit = .second, + timezoneOffset timezone: Offset = Offset() + ) -> Offset { + let __start = org_apache_arrow_flatbuf_Timestamp.startTimestamp(&fbb) + org_apache_arrow_flatbuf_Timestamp.add(unit: unit, &fbb) + org_apache_arrow_flatbuf_Timestamp.add(timezone: timezone, &fbb) + return org_apache_arrow_flatbuf_Timestamp.endTimestamp(&fbb, start: __start) + } + + public static func verify(_ verifier: inout Verifier, at position: Int, of type: T.Type) throws + where T: Verifiable { + var _v = try verifier.visitTable(at: position) + try _v.visit( + field: VTOFFSET.unit.p, fieldName: "unit", required: false, + type: org_apache_arrow_flatbuf_TimeUnit.self) + try _v.visit( + field: VTOFFSET.timezone.p, fieldName: "timezone", required: false, + type: ForwardOffset.self) + _v.finish() + } +} + +public struct org_apache_arrow_flatbuf_Interval: FlatBufferObject, Verifiable { + + static func validateVersion() { FlatBuffersVersion_23_1_4() } + public var __buffer: ByteBuffer! { return _accessor.bb } + private var _accessor: Table + + public static func getRootAsInterval(bb: ByteBuffer) -> org_apache_arrow_flatbuf_Interval { + return org_apache_arrow_flatbuf_Interval( + Table( + bb: bb, position: Int32(bb.read(def: UOffset.self, position: bb.reader)) + Int32(bb.reader)) + ) + } + + private init(_ t: Table) { _accessor = t } + public init(_ bb: ByteBuffer, o: Int32) { _accessor = Table(bb: bb, position: o) } + + private enum VTOFFSET: VOffset { + case unit = 4 + var v: Int32 { Int32(self.rawValue) } + var p: VOffset { self.rawValue } + } + + public var unit: org_apache_arrow_flatbuf_IntervalUnit { + let o = _accessor.offset(VTOFFSET.unit.v) + return o == 0 + ? .yearMonth + : org_apache_arrow_flatbuf_IntervalUnit(rawValue: _accessor.readBuffer(of: Int16.self, at: o)) + ?? .yearMonth + } + public static func startInterval(_ fbb: inout FlatBufferBuilder) -> UOffset { + fbb.startTable(with: 1) + } + public static func add( + unit: org_apache_arrow_flatbuf_IntervalUnit, _ fbb: inout FlatBufferBuilder + ) { fbb.add(element: unit.rawValue, def: 0, at: VTOFFSET.unit.p) } + public static func endInterval(_ fbb: inout FlatBufferBuilder, start: UOffset) -> Offset { + let end = Offset(offset: fbb.endTable(at: start)) + return end + } + public static func createInterval( + _ fbb: inout FlatBufferBuilder, + unit: org_apache_arrow_flatbuf_IntervalUnit = .yearMonth + ) -> Offset { + let __start = org_apache_arrow_flatbuf_Interval.startInterval(&fbb) + org_apache_arrow_flatbuf_Interval.add(unit: unit, &fbb) + return org_apache_arrow_flatbuf_Interval.endInterval(&fbb, start: __start) + } + + public static func verify(_ verifier: inout Verifier, at position: Int, of type: T.Type) throws + where T: Verifiable { + var _v = try verifier.visitTable(at: position) + try _v.visit( + field: VTOFFSET.unit.p, fieldName: "unit", required: false, + type: org_apache_arrow_flatbuf_IntervalUnit.self) + _v.finish() + } +} + +public struct org_apache_arrow_flatbuf_Duration: FlatBufferObject, Verifiable { + + static func validateVersion() { FlatBuffersVersion_23_1_4() } + public var __buffer: ByteBuffer! { return _accessor.bb } + private var _accessor: Table + + public static func getRootAsDuration(bb: ByteBuffer) -> org_apache_arrow_flatbuf_Duration { + return org_apache_arrow_flatbuf_Duration( + Table( + bb: bb, position: Int32(bb.read(def: UOffset.self, position: bb.reader)) + Int32(bb.reader)) + ) + } + + private init(_ t: Table) { _accessor = t } + public init(_ bb: ByteBuffer, o: Int32) { _accessor = Table(bb: bb, position: o) } + + private enum VTOFFSET: VOffset { + case unit = 4 + var v: Int32 { Int32(self.rawValue) } + var p: VOffset { self.rawValue } + } + + public var unit: org_apache_arrow_flatbuf_TimeUnit { + let o = _accessor.offset(VTOFFSET.unit.v) + return o == 0 + ? .millisecond + : org_apache_arrow_flatbuf_TimeUnit(rawValue: _accessor.readBuffer(of: Int16.self, at: o)) + ?? .millisecond + } + public static func startDuration(_ fbb: inout FlatBufferBuilder) -> UOffset { + fbb.startTable(with: 1) + } + public static func add(unit: org_apache_arrow_flatbuf_TimeUnit, _ fbb: inout FlatBufferBuilder) { + fbb.add(element: unit.rawValue, def: 1, at: VTOFFSET.unit.p) + } + public static func endDuration(_ fbb: inout FlatBufferBuilder, start: UOffset) -> Offset { + let end = Offset(offset: fbb.endTable(at: start)) + return end + } + public static func createDuration( + _ fbb: inout FlatBufferBuilder, + unit: org_apache_arrow_flatbuf_TimeUnit = .millisecond + ) -> Offset { + let __start = org_apache_arrow_flatbuf_Duration.startDuration(&fbb) + org_apache_arrow_flatbuf_Duration.add(unit: unit, &fbb) + return org_apache_arrow_flatbuf_Duration.endDuration(&fbb, start: __start) + } + + public static func verify(_ verifier: inout Verifier, at position: Int, of type: T.Type) throws + where T: Verifiable { + var _v = try verifier.visitTable(at: position) + try _v.visit( + field: VTOFFSET.unit.p, fieldName: "unit", required: false, + type: org_apache_arrow_flatbuf_TimeUnit.self) + _v.finish() + } +} + +/// ---------------------------------------------------------------------- +/// user defined key value pairs to add custom metadata to arrow +/// key namespacing is the responsibility of the user +public struct org_apache_arrow_flatbuf_KeyValue: FlatBufferObject, Verifiable { + + static func validateVersion() { FlatBuffersVersion_23_1_4() } + public var __buffer: ByteBuffer! { return _accessor.bb } + private var _accessor: Table + + public static func getRootAsKeyValue(bb: ByteBuffer) -> org_apache_arrow_flatbuf_KeyValue { + return org_apache_arrow_flatbuf_KeyValue( + Table( + bb: bb, position: Int32(bb.read(def: UOffset.self, position: bb.reader)) + Int32(bb.reader)) + ) + } + + private init(_ t: Table) { _accessor = t } + public init(_ bb: ByteBuffer, o: Int32) { _accessor = Table(bb: bb, position: o) } + + private enum VTOFFSET: VOffset { + case key = 4 + case value = 6 + var v: Int32 { Int32(self.rawValue) } + var p: VOffset { self.rawValue } + } + + public var key: String? { + let o = _accessor.offset(VTOFFSET.key.v) + return o == 0 ? nil : _accessor.string(at: o) + } + public var keySegmentArray: [UInt8]? { return _accessor.getVector(at: VTOFFSET.key.v) } + public var value: String? { + let o = _accessor.offset(VTOFFSET.value.v) + return o == 0 ? nil : _accessor.string(at: o) + } + public var valueSegmentArray: [UInt8]? { return _accessor.getVector(at: VTOFFSET.value.v) } + public static func startKeyValue(_ fbb: inout FlatBufferBuilder) -> UOffset { + fbb.startTable(with: 2) + } + public static func add(key: Offset, _ fbb: inout FlatBufferBuilder) { + fbb.add(offset: key, at: VTOFFSET.key.p) + } + public static func add(value: Offset, _ fbb: inout FlatBufferBuilder) { + fbb.add(offset: value, at: VTOFFSET.value.p) + } + public static func endKeyValue(_ fbb: inout FlatBufferBuilder, start: UOffset) -> Offset { + let end = Offset(offset: fbb.endTable(at: start)) + return end + } + public static func createKeyValue( + _ fbb: inout FlatBufferBuilder, + keyOffset key: Offset = Offset(), + valueOffset value: Offset = Offset() + ) -> Offset { + let __start = org_apache_arrow_flatbuf_KeyValue.startKeyValue(&fbb) + org_apache_arrow_flatbuf_KeyValue.add(key: key, &fbb) + org_apache_arrow_flatbuf_KeyValue.add(value: value, &fbb) + return org_apache_arrow_flatbuf_KeyValue.endKeyValue(&fbb, start: __start) + } + + public static func verify(_ verifier: inout Verifier, at position: Int, of type: T.Type) throws + where T: Verifiable { + var _v = try verifier.visitTable(at: position) + try _v.visit( + field: VTOFFSET.key.p, fieldName: "key", required: false, type: ForwardOffset.self) + try _v.visit( + field: VTOFFSET.value.p, fieldName: "value", required: false, type: ForwardOffset.self + ) + _v.finish() + } +} + +public struct org_apache_arrow_flatbuf_DictionaryEncoding: FlatBufferObject, Verifiable { + + static func validateVersion() { FlatBuffersVersion_23_1_4() } + public var __buffer: ByteBuffer! { return _accessor.bb } + private var _accessor: Table + + public static func getRootAsDictionaryEncoding(bb: ByteBuffer) + -> org_apache_arrow_flatbuf_DictionaryEncoding + { + return org_apache_arrow_flatbuf_DictionaryEncoding( + Table( + bb: bb, position: Int32(bb.read(def: UOffset.self, position: bb.reader)) + Int32(bb.reader)) + ) + } + + private init(_ t: Table) { _accessor = t } + public init(_ bb: ByteBuffer, o: Int32) { _accessor = Table(bb: bb, position: o) } + + private enum VTOFFSET: VOffset { + case id = 4 + case indexType = 6 + case isOrdered = 8 + case dictionaryKind = 10 + var v: Int32 { Int32(self.rawValue) } + var p: VOffset { self.rawValue } + } + + /// The known dictionary id in the application where this data is used. In + /// the file or streaming formats, the dictionary ids are found in the + /// DictionaryBatch messages + public var id: Int64 { + let o = _accessor.offset(VTOFFSET.id.v) + return o == 0 ? 0 : _accessor.readBuffer(of: Int64.self, at: o) + } + /// The dictionary indices are constrained to be non-negative integers. If + /// this field is null, the indices must be signed int32. To maximize + /// cross-language compatibility and performance, implementations are + /// recommended to prefer signed integer types over unsigned integer types + /// and to avoid uint64 indices unless they are required by an application. + public var indexType: org_apache_arrow_flatbuf_Int? { + let o = _accessor.offset(VTOFFSET.indexType.v) + return o == 0 + ? nil + : org_apache_arrow_flatbuf_Int(_accessor.bb, o: _accessor.indirect(o + _accessor.postion)) + } + /// By default, dictionaries are not ordered, or the order does not have + /// semantic meaning. In some statistical, applications, dictionary-encoding + /// is used to represent ordered categorical data, and we provide a way to + /// preserve that metadata here + public var isOrdered: Bool { + let o = _accessor.offset(VTOFFSET.isOrdered.v) + return o == 0 ? false : 0 != _accessor.readBuffer(of: Byte.self, at: o) + } + public var dictionaryKind: org_apache_arrow_flatbuf_DictionaryKind { + let o = _accessor.offset(VTOFFSET.dictionaryKind.v) + return o == 0 + ? .densearray + : org_apache_arrow_flatbuf_DictionaryKind( + rawValue: _accessor.readBuffer(of: Int16.self, at: o)) ?? .densearray + } + public static func startDictionaryEncoding(_ fbb: inout FlatBufferBuilder) -> UOffset { + fbb.startTable(with: 4) + } + public static func add(id: Int64, _ fbb: inout FlatBufferBuilder) { + fbb.add(element: id, def: 0, at: VTOFFSET.id.p) + } + public static func add(indexType: Offset, _ fbb: inout FlatBufferBuilder) { + fbb.add(offset: indexType, at: VTOFFSET.indexType.p) + } + public static func add(isOrdered: Bool, _ fbb: inout FlatBufferBuilder) { + fbb.add( + element: isOrdered, def: false, + at: VTOFFSET.isOrdered.p) + } + public static func add( + dictionaryKind: org_apache_arrow_flatbuf_DictionaryKind, _ fbb: inout FlatBufferBuilder + ) { fbb.add(element: dictionaryKind.rawValue, def: 0, at: VTOFFSET.dictionaryKind.p) } + public static func endDictionaryEncoding(_ fbb: inout FlatBufferBuilder, start: UOffset) -> Offset + { + let end = Offset(offset: fbb.endTable(at: start)) + return end + } + public static func createDictionaryEncoding( + _ fbb: inout FlatBufferBuilder, + id: Int64 = 0, + indexTypeOffset indexType: Offset = Offset(), + isOrdered: Bool = false, + dictionaryKind: org_apache_arrow_flatbuf_DictionaryKind = .densearray + ) -> Offset { + let __start = org_apache_arrow_flatbuf_DictionaryEncoding.startDictionaryEncoding(&fbb) + org_apache_arrow_flatbuf_DictionaryEncoding.add(id: id, &fbb) + org_apache_arrow_flatbuf_DictionaryEncoding.add(indexType: indexType, &fbb) + org_apache_arrow_flatbuf_DictionaryEncoding.add(isOrdered: isOrdered, &fbb) + org_apache_arrow_flatbuf_DictionaryEncoding.add(dictionaryKind: dictionaryKind, &fbb) + return org_apache_arrow_flatbuf_DictionaryEncoding.endDictionaryEncoding(&fbb, start: __start) + } + + public static func verify(_ verifier: inout Verifier, at position: Int, of type: T.Type) throws + where T: Verifiable { + var _v = try verifier.visitTable(at: position) + try _v.visit(field: VTOFFSET.id.p, fieldName: "id", required: false, type: Int64.self) + try _v.visit( + field: VTOFFSET.indexType.p, fieldName: "indexType", required: false, + type: ForwardOffset.self) + try _v.visit( + field: VTOFFSET.isOrdered.p, fieldName: "isOrdered", required: false, type: Bool.self) + try _v.visit( + field: VTOFFSET.dictionaryKind.p, fieldName: "dictionaryKind", required: false, + type: org_apache_arrow_flatbuf_DictionaryKind.self) + _v.finish() + } +} + +/// ---------------------------------------------------------------------- +/// A field represents a named column in a record / row batch or child of a +/// nested type. +public struct org_apache_arrow_flatbuf_Field: FlatBufferObject, Verifiable { + + static func validateVersion() { FlatBuffersVersion_23_1_4() } + public var __buffer: ByteBuffer! { return _accessor.bb } + private var _accessor: Table + + public static func getRootAsField(bb: ByteBuffer) -> org_apache_arrow_flatbuf_Field { + return org_apache_arrow_flatbuf_Field( + Table( + bb: bb, position: Int32(bb.read(def: UOffset.self, position: bb.reader)) + Int32(bb.reader)) + ) + } + + private init(_ t: Table) { _accessor = t } + public init(_ bb: ByteBuffer, o: Int32) { _accessor = Table(bb: bb, position: o) } + + private enum VTOFFSET: VOffset { + case name = 4 + case nullable = 6 + case typeType = 8 + case type = 10 + case dictionary = 12 + case children = 14 + case customMetadata = 16 + var v: Int32 { Int32(self.rawValue) } + var p: VOffset { self.rawValue } + } + + /// Name is not required, in i.e. a List + public var name: String? { + let o = _accessor.offset(VTOFFSET.name.v) + return o == 0 ? nil : _accessor.string(at: o) + } + public var nameSegmentArray: [UInt8]? { return _accessor.getVector(at: VTOFFSET.name.v) } + /// Whether or not this field can contain nulls. Should be true in general. + public var nullable: Bool { + let o = _accessor.offset(VTOFFSET.nullable.v) + return o == 0 ? false : 0 != _accessor.readBuffer(of: Byte.self, at: o) + } + public var typeType: org_apache_arrow_flatbuf_Type_ { + let o = _accessor.offset(VTOFFSET.typeType.v) + return o == 0 + ? .none_ + : org_apache_arrow_flatbuf_Type_(rawValue: _accessor.readBuffer(of: UInt8.self, at: o)) + ?? .none_ + } + /// This is the type of the decoded value if the field is dictionary encoded. + public func type(type: T.Type) -> T? { + let o = _accessor.offset(VTOFFSET.type.v) + return o == 0 ? nil : _accessor.union(o) + } + /// Present only if the field is dictionary encoded. + public var dictionary: org_apache_arrow_flatbuf_DictionaryEncoding? { + let o = _accessor.offset(VTOFFSET.dictionary.v) + return o == 0 + ? nil + : org_apache_arrow_flatbuf_DictionaryEncoding( + _accessor.bb, o: _accessor.indirect(o + _accessor.postion)) + } + /// children apply only to nested data types like Struct, List and Union. For + /// primitive types children will have length 0. + public var hasChildren: Bool { + let o = _accessor.offset(VTOFFSET.children.v) + return o == 0 ? false : true + } + public var childrenCount: Int32 { + let o = _accessor.offset(VTOFFSET.children.v) + return o == 0 ? 0 : _accessor.vector(count: o) + } + public func children(at index: Int32) -> org_apache_arrow_flatbuf_Field? { + let o = _accessor.offset(VTOFFSET.children.v) + return o == 0 + ? nil + : org_apache_arrow_flatbuf_Field( + _accessor.bb, o: _accessor.indirect(_accessor.vector(at: o) + index * 4)) + } + /// User-defined metadata + public var hasCustomMetadata: Bool { + let o = _accessor.offset(VTOFFSET.customMetadata.v) + return o == 0 ? false : true + } + public var customMetadataCount: Int32 { + let o = _accessor.offset(VTOFFSET.customMetadata.v) + return o == 0 ? 0 : _accessor.vector(count: o) + } + public func customMetadata(at index: Int32) -> org_apache_arrow_flatbuf_KeyValue? { + let o = _accessor.offset(VTOFFSET.customMetadata.v) + return o == 0 + ? nil + : org_apache_arrow_flatbuf_KeyValue( + _accessor.bb, o: _accessor.indirect(_accessor.vector(at: o) + index * 4)) + } + public static func startField(_ fbb: inout FlatBufferBuilder) -> UOffset { + fbb.startTable(with: 7) + } + public static func add(name: Offset, _ fbb: inout FlatBufferBuilder) { + fbb.add(offset: name, at: VTOFFSET.name.p) + } + public static func add(nullable: Bool, _ fbb: inout FlatBufferBuilder) { + fbb.add( + element: nullable, def: false, + at: VTOFFSET.nullable.p) + } + public static func add(typeType: org_apache_arrow_flatbuf_Type_, _ fbb: inout FlatBufferBuilder) { + fbb.add(element: typeType.rawValue, def: 0, at: VTOFFSET.typeType.p) + } + public static func add(type: Offset, _ fbb: inout FlatBufferBuilder) { + fbb.add(offset: type, at: VTOFFSET.type.p) + } + public static func add(dictionary: Offset, _ fbb: inout FlatBufferBuilder) { + fbb.add(offset: dictionary, at: VTOFFSET.dictionary.p) + } + public static func addVectorOf(children: Offset, _ fbb: inout FlatBufferBuilder) { + fbb.add(offset: children, at: VTOFFSET.children.p) + } + public static func addVectorOf(customMetadata: Offset, _ fbb: inout FlatBufferBuilder) { + fbb.add(offset: customMetadata, at: VTOFFSET.customMetadata.p) + } + public static func endField(_ fbb: inout FlatBufferBuilder, start: UOffset) -> Offset { + let end = Offset(offset: fbb.endTable(at: start)) + return end + } + public static func createField( + _ fbb: inout FlatBufferBuilder, + nameOffset name: Offset = Offset(), + nullable: Bool = false, + typeType: org_apache_arrow_flatbuf_Type_ = .none_, + typeOffset type: Offset = Offset(), + dictionaryOffset dictionary: Offset = Offset(), + childrenVectorOffset children: Offset = Offset(), + customMetadataVectorOffset customMetadata: Offset = Offset() + ) -> Offset { + let __start = org_apache_arrow_flatbuf_Field.startField(&fbb) + org_apache_arrow_flatbuf_Field.add(name: name, &fbb) + org_apache_arrow_flatbuf_Field.add(nullable: nullable, &fbb) + org_apache_arrow_flatbuf_Field.add(typeType: typeType, &fbb) + org_apache_arrow_flatbuf_Field.add(type: type, &fbb) + org_apache_arrow_flatbuf_Field.add(dictionary: dictionary, &fbb) + org_apache_arrow_flatbuf_Field.addVectorOf(children: children, &fbb) + org_apache_arrow_flatbuf_Field.addVectorOf(customMetadata: customMetadata, &fbb) + return org_apache_arrow_flatbuf_Field.endField(&fbb, start: __start) + } + + public static func verify(_ verifier: inout Verifier, at position: Int, of type: T.Type) throws + where T: Verifiable { + var _v = try verifier.visitTable(at: position) + try _v.visit( + field: VTOFFSET.name.p, fieldName: "name", required: false, type: ForwardOffset.self) + try _v.visit( + field: VTOFFSET.nullable.p, fieldName: "nullable", required: false, type: Bool.self) + try _v.visit( + unionKey: VTOFFSET.typeType.p, unionField: VTOFFSET.type.p, unionKeyName: "typeType", + fieldName: "type", required: false, + completion: { (verifier, key: org_apache_arrow_flatbuf_Type_, pos) in + switch key { + case .none_: + break // NOTE - SWIFT doesnt support none + case .null: + try ForwardOffset.verify( + &verifier, at: pos, of: org_apache_arrow_flatbuf_Null.self) + case .int: + try ForwardOffset.verify( + &verifier, at: pos, of: org_apache_arrow_flatbuf_Int.self) + case .floatingpoint: + try ForwardOffset.verify( + &verifier, at: pos, of: org_apache_arrow_flatbuf_FloatingPoint.self) + case .binary: + try ForwardOffset.verify( + &verifier, at: pos, of: org_apache_arrow_flatbuf_Binary.self) + case .utf8: + try ForwardOffset.verify( + &verifier, at: pos, of: org_apache_arrow_flatbuf_Utf8.self) + case .bool: + try ForwardOffset.verify( + &verifier, at: pos, of: org_apache_arrow_flatbuf_Bool.self) + case .decimal: + try ForwardOffset.verify( + &verifier, at: pos, of: org_apache_arrow_flatbuf_Decimal.self) + case .date: + try ForwardOffset.verify( + &verifier, at: pos, of: org_apache_arrow_flatbuf_Date.self) + case .time: + try ForwardOffset.verify( + &verifier, at: pos, of: org_apache_arrow_flatbuf_Time.self) + case .timestamp: + try ForwardOffset.verify( + &verifier, at: pos, of: org_apache_arrow_flatbuf_Timestamp.self) + case .interval: + try ForwardOffset.verify( + &verifier, at: pos, of: org_apache_arrow_flatbuf_Interval.self) + case .list: + try ForwardOffset.verify( + &verifier, at: pos, of: org_apache_arrow_flatbuf_List.self) + case .struct_: + try ForwardOffset.verify( + &verifier, at: pos, of: org_apache_arrow_flatbuf_Struct_.self) + case .union: + try ForwardOffset.verify( + &verifier, at: pos, of: org_apache_arrow_flatbuf_Union.self) + case .fixedsizebinary: + try ForwardOffset.verify( + &verifier, at: pos, of: org_apache_arrow_flatbuf_FixedSizeBinary.self) + case .fixedsizelist: + try ForwardOffset.verify( + &verifier, at: pos, of: org_apache_arrow_flatbuf_FixedSizeList.self) + case .map: + try ForwardOffset.verify( + &verifier, at: pos, of: org_apache_arrow_flatbuf_Map.self) + case .duration: + try ForwardOffset.verify( + &verifier, at: pos, of: org_apache_arrow_flatbuf_Duration.self) + case .largebinary: + try ForwardOffset.verify( + &verifier, at: pos, of: org_apache_arrow_flatbuf_LargeBinary.self) + case .largeutf8: + try ForwardOffset.verify( + &verifier, at: pos, of: org_apache_arrow_flatbuf_LargeUtf8.self) + case .largelist: + try ForwardOffset.verify( + &verifier, at: pos, of: org_apache_arrow_flatbuf_LargeList.self) + case .runendencoded: + try ForwardOffset.verify( + &verifier, at: pos, of: org_apache_arrow_flatbuf_RunEndEncoded.self) + } + }) + try _v.visit( + field: VTOFFSET.dictionary.p, fieldName: "dictionary", required: false, + type: ForwardOffset.self) + try _v.visit( + field: VTOFFSET.children.p, fieldName: "children", required: false, + type: ForwardOffset< + Vector, org_apache_arrow_flatbuf_Field> + >.self) + try _v.visit( + field: VTOFFSET.customMetadata.p, fieldName: "customMetadata", required: false, + type: ForwardOffset< + Vector, org_apache_arrow_flatbuf_KeyValue> + >.self) + _v.finish() + } +} + +/// ---------------------------------------------------------------------- +/// A Schema describes the columns in a row batch +public struct org_apache_arrow_flatbuf_Schema: FlatBufferObject, Verifiable { + + static func validateVersion() { FlatBuffersVersion_23_1_4() } + public var __buffer: ByteBuffer! { return _accessor.bb } + private var _accessor: Table + + public static func getRootAsSchema(bb: ByteBuffer) -> org_apache_arrow_flatbuf_Schema { + return org_apache_arrow_flatbuf_Schema( + Table( + bb: bb, position: Int32(bb.read(def: UOffset.self, position: bb.reader)) + Int32(bb.reader)) + ) + } + + private init(_ t: Table) { _accessor = t } + public init(_ bb: ByteBuffer, o: Int32) { _accessor = Table(bb: bb, position: o) } + + private enum VTOFFSET: VOffset { + case endianness = 4 + case fields = 6 + case customMetadata = 8 + case features = 10 + var v: Int32 { Int32(self.rawValue) } + var p: VOffset { self.rawValue } + } + + /// endianness of the buffer + /// it is Little Endian by default + /// if endianness doesn't match the underlying system then the vectors need to be converted + public var endianness: org_apache_arrow_flatbuf_Endianness { + let o = _accessor.offset(VTOFFSET.endianness.v) + return o == 0 + ? .little + : org_apache_arrow_flatbuf_Endianness(rawValue: _accessor.readBuffer(of: Int16.self, at: o)) + ?? .little + } + public var hasFields: Bool { + let o = _accessor.offset(VTOFFSET.fields.v) + return o == 0 ? false : true + } + public var fieldsCount: Int32 { + let o = _accessor.offset(VTOFFSET.fields.v) + return o == 0 ? 0 : _accessor.vector(count: o) + } + public func fields(at index: Int32) -> org_apache_arrow_flatbuf_Field? { + let o = _accessor.offset(VTOFFSET.fields.v) + return o == 0 + ? nil + : org_apache_arrow_flatbuf_Field( + _accessor.bb, o: _accessor.indirect(_accessor.vector(at: o) + index * 4)) + } + public var hasCustomMetadata: Bool { + let o = _accessor.offset(VTOFFSET.customMetadata.v) + return o == 0 ? false : true + } + public var customMetadataCount: Int32 { + let o = _accessor.offset(VTOFFSET.customMetadata.v) + return o == 0 ? 0 : _accessor.vector(count: o) + } + public func customMetadata(at index: Int32) -> org_apache_arrow_flatbuf_KeyValue? { + let o = _accessor.offset(VTOFFSET.customMetadata.v) + return o == 0 + ? nil + : org_apache_arrow_flatbuf_KeyValue( + _accessor.bb, o: _accessor.indirect(_accessor.vector(at: o) + index * 4)) + } + /// Features used in the stream/file. + public var hasFeatures: Bool { + let o = _accessor.offset(VTOFFSET.features.v) + return o == 0 ? false : true + } + public var featuresCount: Int32 { + let o = _accessor.offset(VTOFFSET.features.v) + return o == 0 ? 0 : _accessor.vector(count: o) + } + public func features(at index: Int32) -> org_apache_arrow_flatbuf_Feature? { + let o = _accessor.offset(VTOFFSET.features.v) + return o == 0 + ? org_apache_arrow_flatbuf_Feature.unused + : org_apache_arrow_flatbuf_Feature( + rawValue: _accessor.directRead(of: Int64.self, offset: _accessor.vector(at: o) + index * 8)) + } + public static func startSchema(_ fbb: inout FlatBufferBuilder) -> UOffset { + fbb.startTable(with: 4) + } + public static func add( + endianness: org_apache_arrow_flatbuf_Endianness, _ fbb: inout FlatBufferBuilder + ) { fbb.add(element: endianness.rawValue, def: 0, at: VTOFFSET.endianness.p) } + public static func addVectorOf(fields: Offset, _ fbb: inout FlatBufferBuilder) { + fbb.add(offset: fields, at: VTOFFSET.fields.p) + } + public static func addVectorOf(customMetadata: Offset, _ fbb: inout FlatBufferBuilder) { + fbb.add(offset: customMetadata, at: VTOFFSET.customMetadata.p) + } + public static func addVectorOf(features: Offset, _ fbb: inout FlatBufferBuilder) { + fbb.add(offset: features, at: VTOFFSET.features.p) + } + public static func endSchema(_ fbb: inout FlatBufferBuilder, start: UOffset) -> Offset { + let end = Offset(offset: fbb.endTable(at: start)) + return end + } + public static func createSchema( + _ fbb: inout FlatBufferBuilder, + endianness: org_apache_arrow_flatbuf_Endianness = .little, + fieldsVectorOffset fields: Offset = Offset(), + customMetadataVectorOffset customMetadata: Offset = Offset(), + featuresVectorOffset features: Offset = Offset() + ) -> Offset { + let __start = org_apache_arrow_flatbuf_Schema.startSchema(&fbb) + org_apache_arrow_flatbuf_Schema.add(endianness: endianness, &fbb) + org_apache_arrow_flatbuf_Schema.addVectorOf(fields: fields, &fbb) + org_apache_arrow_flatbuf_Schema.addVectorOf(customMetadata: customMetadata, &fbb) + org_apache_arrow_flatbuf_Schema.addVectorOf(features: features, &fbb) + return org_apache_arrow_flatbuf_Schema.endSchema(&fbb, start: __start) + } + + public static func verify(_ verifier: inout Verifier, at position: Int, of type: T.Type) throws + where T: Verifiable { + var _v = try verifier.visitTable(at: position) + try _v.visit( + field: VTOFFSET.endianness.p, fieldName: "endianness", required: false, + type: org_apache_arrow_flatbuf_Endianness.self) + try _v.visit( + field: VTOFFSET.fields.p, fieldName: "fields", required: false, + type: ForwardOffset< + Vector, org_apache_arrow_flatbuf_Field> + >.self) + try _v.visit( + field: VTOFFSET.customMetadata.p, fieldName: "customMetadata", required: false, + type: ForwardOffset< + Vector, org_apache_arrow_flatbuf_KeyValue> + >.self) + try _v.visit( + field: VTOFFSET.features.p, fieldName: "features", required: false, + type: ForwardOffset< + Vector + >.self) + _v.finish() + } +} diff --git a/Sources/SparkConnect/SparseTensor_generated.swift b/Sources/SparkConnect/SparseTensor_generated.swift new file mode 100644 index 0000000..3e99d51 --- /dev/null +++ b/Sources/SparkConnect/SparseTensor_generated.swift @@ -0,0 +1,863 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +// automatically generated by the FlatBuffers compiler, do not modify +// swiftlint:disable all +// swiftformat:disable all + +import FlatBuffers + +public enum org_apache_arrow_flatbuf_SparseMatrixCompressedAxis: Int16, Enum, Verifiable { + public typealias T = Int16 + public static var byteSize: Int { return MemoryLayout.size } + public var value: Int16 { return self.rawValue } + case row = 0 + case column = 1 + + public static var max: org_apache_arrow_flatbuf_SparseMatrixCompressedAxis { return .column } + public static var min: org_apache_arrow_flatbuf_SparseMatrixCompressedAxis { return .row } +} + +public enum org_apache_arrow_flatbuf_SparseTensorIndex: UInt8, UnionEnum { + public typealias T = UInt8 + + public init?(value: T) { + self.init(rawValue: value) + } + + public static var byteSize: Int { return MemoryLayout.size } + public var value: UInt8 { return self.rawValue } + case none_ = 0 + case sparsetensorindexcoo = 1 + case sparsematrixindexcsx = 2 + case sparsetensorindexcsf = 3 + + public static var max: org_apache_arrow_flatbuf_SparseTensorIndex { return .sparsetensorindexcsf } + public static var min: org_apache_arrow_flatbuf_SparseTensorIndex { return .none_ } +} + +/// ---------------------------------------------------------------------- +/// EXPERIMENTAL: Data structures for sparse tensors +/// Coordinate (COO) format of sparse tensor index. +/// +/// COO's index list are represented as a NxM matrix, +/// where N is the number of non-zero values, +/// and M is the number of dimensions of a sparse tensor. +/// +/// indicesBuffer stores the location and size of the data of this indices +/// matrix. The value type and the stride of the indices matrix is +/// specified in indicesType and indicesStrides fields. +/// +/// For example, let X be a 2x3x4x5 tensor, and it has the following +/// 6 non-zero values: +/// ```text +/// X[0, 1, 2, 0] := 1 +/// X[1, 1, 2, 3] := 2 +/// X[0, 2, 1, 0] := 3 +/// X[0, 1, 3, 0] := 4 +/// X[0, 1, 2, 1] := 5 +/// X[1, 2, 0, 4] := 6 +/// ``` +/// In COO format, the index matrix of X is the following 4x6 matrix: +/// ```text +/// [[0, 0, 0, 0, 1, 1], +/// [1, 1, 1, 2, 1, 2], +/// [2, 2, 3, 1, 2, 0], +/// [0, 1, 0, 0, 3, 4]] +/// ``` +/// When isCanonical is true, the indices is sorted in lexicographical order +/// (row-major order), and it does not have duplicated entries. Otherwise, +/// the indices may not be sorted, or may have duplicated entries. +public struct org_apache_arrow_flatbuf_SparseTensorIndexCOO: FlatBufferObject, Verifiable { + + static func validateVersion() { FlatBuffersVersion_23_1_4() } + public var __buffer: ByteBuffer! { return _accessor.bb } + private var _accessor: Table + + public static func getRootAsSparseTensorIndexCOO(bb: ByteBuffer) + -> org_apache_arrow_flatbuf_SparseTensorIndexCOO + { + return org_apache_arrow_flatbuf_SparseTensorIndexCOO( + Table( + bb: bb, position: Int32(bb.read(def: UOffset.self, position: bb.reader)) + Int32(bb.reader)) + ) + } + + private init(_ t: Table) { _accessor = t } + public init(_ bb: ByteBuffer, o: Int32) { _accessor = Table(bb: bb, position: o) } + + private enum VTOFFSET: VOffset { + case indicesType = 4 + case indicesStrides = 6 + case indicesBuffer = 8 + case isCanonical = 10 + var v: Int32 { Int32(self.rawValue) } + var p: VOffset { self.rawValue } + } + + /// The type of values in indicesBuffer + public var indicesType: org_apache_arrow_flatbuf_Int! { + let o = _accessor.offset(VTOFFSET.indicesType.v) + return org_apache_arrow_flatbuf_Int(_accessor.bb, o: _accessor.indirect(o + _accessor.postion)) + } + /// Non-negative byte offsets to advance one value cell along each dimension + /// If omitted, default to row-major order (C-like). + public var hasIndicesStrides: Bool { + let o = _accessor.offset(VTOFFSET.indicesStrides.v) + return o == 0 ? false : true + } + public var indicesStridesCount: Int32 { + let o = _accessor.offset(VTOFFSET.indicesStrides.v) + return o == 0 ? 0 : _accessor.vector(count: o) + } + public func indicesStrides(at index: Int32) -> Int64 { + let o = _accessor.offset(VTOFFSET.indicesStrides.v) + return o == 0 + ? 0 : _accessor.directRead(of: Int64.self, offset: _accessor.vector(at: o) + index * 8) + } + public var indicesStrides: [Int64] { + return _accessor.getVector(at: VTOFFSET.indicesStrides.v) ?? [] + } + /// The location and size of the indices matrix's data + public var indicesBuffer: org_apache_arrow_flatbuf_Buffer! { + let o = _accessor.offset(VTOFFSET.indicesBuffer.v) + return _accessor.readBuffer(of: org_apache_arrow_flatbuf_Buffer.self, at: o) + } + public var mutableIndicesBuffer: org_apache_arrow_flatbuf_Buffer_Mutable! { + let o = _accessor.offset(VTOFFSET.indicesBuffer.v) + return org_apache_arrow_flatbuf_Buffer_Mutable(_accessor.bb, o: o + _accessor.postion) + } + /// This flag is true if and only if the indices matrix is sorted in + /// row-major order, and does not have duplicated entries. + /// This sort order is the same as of Tensorflow's SparseTensor, + /// but it is inverse order of SciPy's canonical coo_matrix + /// (SciPy employs column-major order for its coo_matrix). + public var isCanonical: Bool { + let o = _accessor.offset(VTOFFSET.isCanonical.v) + return o == 0 ? false : 0 != _accessor.readBuffer(of: Byte.self, at: o) + } + public static func startSparseTensorIndexCOO(_ fbb: inout FlatBufferBuilder) -> UOffset { + fbb.startTable(with: 4) + } + public static func add(indicesType: Offset, _ fbb: inout FlatBufferBuilder) { + fbb.add(offset: indicesType, at: VTOFFSET.indicesType.p) + } + public static func addVectorOf(indicesStrides: Offset, _ fbb: inout FlatBufferBuilder) { + fbb.add(offset: indicesStrides, at: VTOFFSET.indicesStrides.p) + } + public static func add( + indicesBuffer: org_apache_arrow_flatbuf_Buffer?, _ fbb: inout FlatBufferBuilder + ) { + guard let indicesBuffer = indicesBuffer else { return } + fbb.create(struct: indicesBuffer, position: VTOFFSET.indicesBuffer.p) + } + public static func add(isCanonical: Bool, _ fbb: inout FlatBufferBuilder) { + fbb.add( + element: isCanonical, def: false, + at: VTOFFSET.isCanonical.p) + } + public static func endSparseTensorIndexCOO(_ fbb: inout FlatBufferBuilder, start: UOffset) + -> Offset + { + let end = Offset(offset: fbb.endTable(at: start)) + fbb.require(table: end, fields: [4, 8]) + return end + } + public static func createSparseTensorIndexCOO( + _ fbb: inout FlatBufferBuilder, + indicesTypeOffset indicesType: Offset, + indicesStridesVectorOffset indicesStrides: Offset = Offset(), + indicesBuffer: org_apache_arrow_flatbuf_Buffer, + isCanonical: Bool = false + ) -> Offset { + let __start = org_apache_arrow_flatbuf_SparseTensorIndexCOO.startSparseTensorIndexCOO(&fbb) + org_apache_arrow_flatbuf_SparseTensorIndexCOO.add(indicesType: indicesType, &fbb) + org_apache_arrow_flatbuf_SparseTensorIndexCOO.addVectorOf(indicesStrides: indicesStrides, &fbb) + org_apache_arrow_flatbuf_SparseTensorIndexCOO.add(indicesBuffer: indicesBuffer, &fbb) + org_apache_arrow_flatbuf_SparseTensorIndexCOO.add(isCanonical: isCanonical, &fbb) + return org_apache_arrow_flatbuf_SparseTensorIndexCOO.endSparseTensorIndexCOO( + &fbb, start: __start) + } + + public static func verify(_ verifier: inout Verifier, at position: Int, of type: T.Type) throws + where T: Verifiable { + var _v = try verifier.visitTable(at: position) + try _v.visit( + field: VTOFFSET.indicesType.p, fieldName: "indicesType", required: true, + type: ForwardOffset.self) + try _v.visit( + field: VTOFFSET.indicesStrides.p, fieldName: "indicesStrides", required: false, + type: ForwardOffset>.self) + try _v.visit( + field: VTOFFSET.indicesBuffer.p, fieldName: "indicesBuffer", required: true, + type: org_apache_arrow_flatbuf_Buffer.self) + try _v.visit( + field: VTOFFSET.isCanonical.p, fieldName: "isCanonical", required: false, type: Bool.self) + _v.finish() + } +} + +/// Compressed Sparse format, that is matrix-specific. +public struct org_apache_arrow_flatbuf_SparseMatrixIndexCSX: FlatBufferObject, Verifiable { + + static func validateVersion() { FlatBuffersVersion_23_1_4() } + public var __buffer: ByteBuffer! { return _accessor.bb } + private var _accessor: Table + + public static func getRootAsSparseMatrixIndexCSX(bb: ByteBuffer) + -> org_apache_arrow_flatbuf_SparseMatrixIndexCSX + { + return org_apache_arrow_flatbuf_SparseMatrixIndexCSX( + Table( + bb: bb, position: Int32(bb.read(def: UOffset.self, position: bb.reader)) + Int32(bb.reader)) + ) + } + + private init(_ t: Table) { _accessor = t } + public init(_ bb: ByteBuffer, o: Int32) { _accessor = Table(bb: bb, position: o) } + + private enum VTOFFSET: VOffset { + case compressedAxis = 4 + case indptrType = 6 + case indptrBuffer = 8 + case indicesType = 10 + case indicesBuffer = 12 + var v: Int32 { Int32(self.rawValue) } + var p: VOffset { self.rawValue } + } + + /// Which axis, row or column, is compressed + public var compressedAxis: org_apache_arrow_flatbuf_SparseMatrixCompressedAxis { + let o = _accessor.offset(VTOFFSET.compressedAxis.v) + return o == 0 + ? .row + : org_apache_arrow_flatbuf_SparseMatrixCompressedAxis( + rawValue: _accessor.readBuffer(of: Int16.self, at: o)) ?? .row + } + /// The type of values in indptrBuffer + public var indptrType: org_apache_arrow_flatbuf_Int! { + let o = _accessor.offset(VTOFFSET.indptrType.v) + return org_apache_arrow_flatbuf_Int(_accessor.bb, o: _accessor.indirect(o + _accessor.postion)) + } + /// indptrBuffer stores the location and size of indptr array that + /// represents the range of the rows. + /// The i-th row spans from `indptr[i]` to `indptr[i+1]` in the data. + /// The length of this array is 1 + (the number of rows), and the type + /// of index value is long. + /// + /// For example, let X be the following 6x4 matrix: + /// ```text + /// X := [[0, 1, 2, 0], + /// [0, 0, 3, 0], + /// [0, 4, 0, 5], + /// [0, 0, 0, 0], + /// [6, 0, 7, 8], + /// [0, 9, 0, 0]]. + /// ``` + /// The array of non-zero values in X is: + /// ```text + /// values(X) = [1, 2, 3, 4, 5, 6, 7, 8, 9]. + /// ``` + /// And the indptr of X is: + /// ```text + /// indptr(X) = [0, 2, 3, 5, 5, 8, 10]. + /// ``` + public var indptrBuffer: org_apache_arrow_flatbuf_Buffer! { + let o = _accessor.offset(VTOFFSET.indptrBuffer.v) + return _accessor.readBuffer(of: org_apache_arrow_flatbuf_Buffer.self, at: o) + } + public var mutableIndptrBuffer: org_apache_arrow_flatbuf_Buffer_Mutable! { + let o = _accessor.offset(VTOFFSET.indptrBuffer.v) + return org_apache_arrow_flatbuf_Buffer_Mutable(_accessor.bb, o: o + _accessor.postion) + } + /// The type of values in indicesBuffer + public var indicesType: org_apache_arrow_flatbuf_Int! { + let o = _accessor.offset(VTOFFSET.indicesType.v) + return org_apache_arrow_flatbuf_Int(_accessor.bb, o: _accessor.indirect(o + _accessor.postion)) + } + /// indicesBuffer stores the location and size of the array that + /// contains the column indices of the corresponding non-zero values. + /// The type of index value is long. + /// + /// For example, the indices of the above X is: + /// ```text + /// indices(X) = [1, 2, 2, 1, 3, 0, 2, 3, 1]. + /// ``` + /// Note that the indices are sorted in lexicographical order for each row. + public var indicesBuffer: org_apache_arrow_flatbuf_Buffer! { + let o = _accessor.offset(VTOFFSET.indicesBuffer.v) + return _accessor.readBuffer(of: org_apache_arrow_flatbuf_Buffer.self, at: o) + } + public var mutableIndicesBuffer: org_apache_arrow_flatbuf_Buffer_Mutable! { + let o = _accessor.offset(VTOFFSET.indicesBuffer.v) + return org_apache_arrow_flatbuf_Buffer_Mutable(_accessor.bb, o: o + _accessor.postion) + } + public static func startSparseMatrixIndexCSX(_ fbb: inout FlatBufferBuilder) -> UOffset { + fbb.startTable(with: 5) + } + public static func add( + compressedAxis: org_apache_arrow_flatbuf_SparseMatrixCompressedAxis, + _ fbb: inout FlatBufferBuilder + ) { fbb.add(element: compressedAxis.rawValue, def: 0, at: VTOFFSET.compressedAxis.p) } + public static func add(indptrType: Offset, _ fbb: inout FlatBufferBuilder) { + fbb.add(offset: indptrType, at: VTOFFSET.indptrType.p) + } + public static func add( + indptrBuffer: org_apache_arrow_flatbuf_Buffer?, _ fbb: inout FlatBufferBuilder + ) { + guard let indptrBuffer = indptrBuffer else { return } + fbb.create(struct: indptrBuffer, position: VTOFFSET.indptrBuffer.p) + } + public static func add(indicesType: Offset, _ fbb: inout FlatBufferBuilder) { + fbb.add(offset: indicesType, at: VTOFFSET.indicesType.p) + } + public static func add( + indicesBuffer: org_apache_arrow_flatbuf_Buffer?, _ fbb: inout FlatBufferBuilder + ) { + guard let indicesBuffer = indicesBuffer else { return } + fbb.create(struct: indicesBuffer, position: VTOFFSET.indicesBuffer.p) + } + public static func endSparseMatrixIndexCSX(_ fbb: inout FlatBufferBuilder, start: UOffset) + -> Offset + { + let end = Offset(offset: fbb.endTable(at: start)) + fbb.require(table: end, fields: [6, 8, 10, 12]) + return end + } + public static func createSparseMatrixIndexCSX( + _ fbb: inout FlatBufferBuilder, + compressedAxis: org_apache_arrow_flatbuf_SparseMatrixCompressedAxis = .row, + indptrTypeOffset indptrType: Offset, + indptrBuffer: org_apache_arrow_flatbuf_Buffer, + indicesTypeOffset indicesType: Offset, + indicesBuffer: org_apache_arrow_flatbuf_Buffer + ) -> Offset { + let __start = org_apache_arrow_flatbuf_SparseMatrixIndexCSX.startSparseMatrixIndexCSX(&fbb) + org_apache_arrow_flatbuf_SparseMatrixIndexCSX.add(compressedAxis: compressedAxis, &fbb) + org_apache_arrow_flatbuf_SparseMatrixIndexCSX.add(indptrType: indptrType, &fbb) + org_apache_arrow_flatbuf_SparseMatrixIndexCSX.add(indptrBuffer: indptrBuffer, &fbb) + org_apache_arrow_flatbuf_SparseMatrixIndexCSX.add(indicesType: indicesType, &fbb) + org_apache_arrow_flatbuf_SparseMatrixIndexCSX.add(indicesBuffer: indicesBuffer, &fbb) + return org_apache_arrow_flatbuf_SparseMatrixIndexCSX.endSparseMatrixIndexCSX( + &fbb, start: __start) + } + + public static func verify(_ verifier: inout Verifier, at position: Int, of type: T.Type) throws + where T: Verifiable { + var _v = try verifier.visitTable(at: position) + try _v.visit( + field: VTOFFSET.compressedAxis.p, fieldName: "compressedAxis", required: false, + type: org_apache_arrow_flatbuf_SparseMatrixCompressedAxis.self) + try _v.visit( + field: VTOFFSET.indptrType.p, fieldName: "indptrType", required: true, + type: ForwardOffset.self) + try _v.visit( + field: VTOFFSET.indptrBuffer.p, fieldName: "indptrBuffer", required: true, + type: org_apache_arrow_flatbuf_Buffer.self) + try _v.visit( + field: VTOFFSET.indicesType.p, fieldName: "indicesType", required: true, + type: ForwardOffset.self) + try _v.visit( + field: VTOFFSET.indicesBuffer.p, fieldName: "indicesBuffer", required: true, + type: org_apache_arrow_flatbuf_Buffer.self) + _v.finish() + } +} + +/// Compressed Sparse Fiber (CSF) sparse tensor index. +public struct org_apache_arrow_flatbuf_SparseTensorIndexCSF: FlatBufferObject, Verifiable { + + static func validateVersion() { FlatBuffersVersion_23_1_4() } + public var __buffer: ByteBuffer! { return _accessor.bb } + private var _accessor: Table + + public static func getRootAsSparseTensorIndexCSF(bb: ByteBuffer) + -> org_apache_arrow_flatbuf_SparseTensorIndexCSF + { + return org_apache_arrow_flatbuf_SparseTensorIndexCSF( + Table( + bb: bb, position: Int32(bb.read(def: UOffset.self, position: bb.reader)) + Int32(bb.reader)) + ) + } + + private init(_ t: Table) { _accessor = t } + public init(_ bb: ByteBuffer, o: Int32) { _accessor = Table(bb: bb, position: o) } + + private enum VTOFFSET: VOffset { + case indptrType = 4 + case indptrBuffers = 6 + case indicesType = 8 + case indicesBuffers = 10 + case axisOrder = 12 + var v: Int32 { Int32(self.rawValue) } + var p: VOffset { self.rawValue } + } + + /// CSF is a generalization of compressed sparse row (CSR) index. + /// See [smith2017knl](http://shaden.io/pub-files/smith2017knl.pdf) + /// + /// CSF index recursively compresses each dimension of a tensor into a set + /// of prefix trees. Each path from a root to leaf forms one tensor + /// non-zero index. CSF is implemented with two arrays of buffers and one + /// arrays of integers. + /// + /// For example, let X be a 2x3x4x5 tensor and let it have the following + /// 8 non-zero values: + /// ```text + /// X[0, 0, 0, 1] := 1 + /// X[0, 0, 0, 2] := 2 + /// X[0, 1, 0, 0] := 3 + /// X[0, 1, 0, 2] := 4 + /// X[0, 1, 1, 0] := 5 + /// X[1, 1, 1, 0] := 6 + /// X[1, 1, 1, 1] := 7 + /// X[1, 1, 1, 2] := 8 + /// ``` + /// As a prefix tree this would be represented as: + /// ```text + /// 0 1 + /// / \ | + /// 0 1 1 + /// / / \ | + /// 0 0 1 1 + /// /| /| | /| | + /// 1 2 0 2 0 0 1 2 + /// ``` + /// The type of values in indptrBuffers + public var indptrType: org_apache_arrow_flatbuf_Int! { + let o = _accessor.offset(VTOFFSET.indptrType.v) + return org_apache_arrow_flatbuf_Int(_accessor.bb, o: _accessor.indirect(o + _accessor.postion)) + } + /// indptrBuffers stores the sparsity structure. + /// Each two consecutive dimensions in a tensor correspond to a buffer in + /// indptrBuffers. A pair of consecutive values at `indptrBuffers[dim][i]` + /// and `indptrBuffers[dim][i + 1]` signify a range of nodes in + /// `indicesBuffers[dim + 1]` who are children of `indicesBuffers[dim][i]` node. + /// + /// For example, the indptrBuffers for the above X is: + /// ```text + /// indptrBuffer(X) = [ + /// [0, 2, 3], + /// [0, 1, 3, 4], + /// [0, 2, 4, 5, 8] + /// ]. + /// ``` + public var hasIndptrBuffers: Bool { + let o = _accessor.offset(VTOFFSET.indptrBuffers.v) + return o == 0 ? false : true + } + public var indptrBuffersCount: Int32 { + let o = _accessor.offset(VTOFFSET.indptrBuffers.v) + return o == 0 ? 0 : _accessor.vector(count: o) + } + public func indptrBuffers(at index: Int32) -> org_apache_arrow_flatbuf_Buffer? { + let o = _accessor.offset(VTOFFSET.indptrBuffers.v) + return o == 0 + ? nil + : _accessor.directRead( + of: org_apache_arrow_flatbuf_Buffer.self, offset: _accessor.vector(at: o) + index * 16) + } + public func mutableIndptrBuffers(at index: Int32) -> org_apache_arrow_flatbuf_Buffer_Mutable? { + let o = _accessor.offset(VTOFFSET.indptrBuffers.v) + return o == 0 + ? nil + : org_apache_arrow_flatbuf_Buffer_Mutable( + _accessor.bb, o: _accessor.vector(at: o) + index * 16) + } + /// The type of values in indicesBuffers + public var indicesType: org_apache_arrow_flatbuf_Int! { + let o = _accessor.offset(VTOFFSET.indicesType.v) + return org_apache_arrow_flatbuf_Int(_accessor.bb, o: _accessor.indirect(o + _accessor.postion)) + } + /// indicesBuffers stores values of nodes. + /// Each tensor dimension corresponds to a buffer in indicesBuffers. + /// For example, the indicesBuffers for the above X is: + /// ```text + /// indicesBuffer(X) = [ + /// [0, 1], + /// [0, 1, 1], + /// [0, 0, 1, 1], + /// [1, 2, 0, 2, 0, 0, 1, 2] + /// ]. + /// ``` + public var hasIndicesBuffers: Bool { + let o = _accessor.offset(VTOFFSET.indicesBuffers.v) + return o == 0 ? false : true + } + public var indicesBuffersCount: Int32 { + let o = _accessor.offset(VTOFFSET.indicesBuffers.v) + return o == 0 ? 0 : _accessor.vector(count: o) + } + public func indicesBuffers(at index: Int32) -> org_apache_arrow_flatbuf_Buffer? { + let o = _accessor.offset(VTOFFSET.indicesBuffers.v) + return o == 0 + ? nil + : _accessor.directRead( + of: org_apache_arrow_flatbuf_Buffer.self, offset: _accessor.vector(at: o) + index * 16) + } + public func mutableIndicesBuffers(at index: Int32) -> org_apache_arrow_flatbuf_Buffer_Mutable? { + let o = _accessor.offset(VTOFFSET.indicesBuffers.v) + return o == 0 + ? nil + : org_apache_arrow_flatbuf_Buffer_Mutable( + _accessor.bb, o: _accessor.vector(at: o) + index * 16) + } + /// axisOrder stores the sequence in which dimensions were traversed to + /// produce the prefix tree. + /// For example, the axisOrder for the above X is: + /// ```text + /// axisOrder(X) = [0, 1, 2, 3]. + /// ``` + public var hasAxisOrder: Bool { + let o = _accessor.offset(VTOFFSET.axisOrder.v) + return o == 0 ? false : true + } + public var axisOrderCount: Int32 { + let o = _accessor.offset(VTOFFSET.axisOrder.v) + return o == 0 ? 0 : _accessor.vector(count: o) + } + public func axisOrder(at index: Int32) -> Int32 { + let o = _accessor.offset(VTOFFSET.axisOrder.v) + return o == 0 + ? 0 : _accessor.directRead(of: Int32.self, offset: _accessor.vector(at: o) + index * 4) + } + public var axisOrder: [Int32] { return _accessor.getVector(at: VTOFFSET.axisOrder.v) ?? [] } + public static func startSparseTensorIndexCSF(_ fbb: inout FlatBufferBuilder) -> UOffset { + fbb.startTable(with: 5) + } + public static func add(indptrType: Offset, _ fbb: inout FlatBufferBuilder) { + fbb.add(offset: indptrType, at: VTOFFSET.indptrType.p) + } + public static func addVectorOf(indptrBuffers: Offset, _ fbb: inout FlatBufferBuilder) { + fbb.add(offset: indptrBuffers, at: VTOFFSET.indptrBuffers.p) + } + public static func startVectorOfIndptrBuffers(_ size: Int, in builder: inout FlatBufferBuilder) { + builder.startVector( + size * MemoryLayout.size, + elementSize: MemoryLayout.alignment) + } + public static func add(indicesType: Offset, _ fbb: inout FlatBufferBuilder) { + fbb.add(offset: indicesType, at: VTOFFSET.indicesType.p) + } + public static func addVectorOf(indicesBuffers: Offset, _ fbb: inout FlatBufferBuilder) { + fbb.add(offset: indicesBuffers, at: VTOFFSET.indicesBuffers.p) + } + public static func startVectorOfIndicesBuffers(_ size: Int, in builder: inout FlatBufferBuilder) { + builder.startVector( + size * MemoryLayout.size, + elementSize: MemoryLayout.alignment) + } + public static func addVectorOf(axisOrder: Offset, _ fbb: inout FlatBufferBuilder) { + fbb.add(offset: axisOrder, at: VTOFFSET.axisOrder.p) + } + public static func endSparseTensorIndexCSF(_ fbb: inout FlatBufferBuilder, start: UOffset) + -> Offset + { + let end = Offset(offset: fbb.endTable(at: start)) + fbb.require(table: end, fields: [4, 6, 8, 10, 12]) + return end + } + public static func createSparseTensorIndexCSF( + _ fbb: inout FlatBufferBuilder, + indptrTypeOffset indptrType: Offset, + indptrBuffersVectorOffset indptrBuffers: Offset, + indicesTypeOffset indicesType: Offset, + indicesBuffersVectorOffset indicesBuffers: Offset, + axisOrderVectorOffset axisOrder: Offset + ) -> Offset { + let __start = org_apache_arrow_flatbuf_SparseTensorIndexCSF.startSparseTensorIndexCSF(&fbb) + org_apache_arrow_flatbuf_SparseTensorIndexCSF.add(indptrType: indptrType, &fbb) + org_apache_arrow_flatbuf_SparseTensorIndexCSF.addVectorOf(indptrBuffers: indptrBuffers, &fbb) + org_apache_arrow_flatbuf_SparseTensorIndexCSF.add(indicesType: indicesType, &fbb) + org_apache_arrow_flatbuf_SparseTensorIndexCSF.addVectorOf(indicesBuffers: indicesBuffers, &fbb) + org_apache_arrow_flatbuf_SparseTensorIndexCSF.addVectorOf(axisOrder: axisOrder, &fbb) + return org_apache_arrow_flatbuf_SparseTensorIndexCSF.endSparseTensorIndexCSF( + &fbb, start: __start) + } + + public static func verify(_ verifier: inout Verifier, at position: Int, of type: T.Type) throws + where T: Verifiable { + var _v = try verifier.visitTable(at: position) + try _v.visit( + field: VTOFFSET.indptrType.p, fieldName: "indptrType", required: true, + type: ForwardOffset.self) + try _v.visit( + field: VTOFFSET.indptrBuffers.p, fieldName: "indptrBuffers", required: true, + type: ForwardOffset> + .self) + try _v.visit( + field: VTOFFSET.indicesType.p, fieldName: "indicesType", required: true, + type: ForwardOffset.self) + try _v.visit( + field: VTOFFSET.indicesBuffers.p, fieldName: "indicesBuffers", required: true, + type: ForwardOffset> + .self) + try _v.visit( + field: VTOFFSET.axisOrder.p, fieldName: "axisOrder", required: true, + type: ForwardOffset>.self) + _v.finish() + } +} + +public struct org_apache_arrow_flatbuf_SparseTensor: FlatBufferObject, Verifiable { + + static func validateVersion() { FlatBuffersVersion_23_1_4() } + public var __buffer: ByteBuffer! { return _accessor.bb } + private var _accessor: Table + + public static func getRootAsSparseTensor(bb: ByteBuffer) -> org_apache_arrow_flatbuf_SparseTensor + { + return org_apache_arrow_flatbuf_SparseTensor( + Table( + bb: bb, position: Int32(bb.read(def: UOffset.self, position: bb.reader)) + Int32(bb.reader)) + ) + } + + private init(_ t: Table) { _accessor = t } + public init(_ bb: ByteBuffer, o: Int32) { _accessor = Table(bb: bb, position: o) } + + private enum VTOFFSET: VOffset { + case typeType = 4 + case type = 6 + case shape = 8 + case nonZeroLength = 10 + case sparseIndexType = 12 + case sparseIndex = 14 + case data = 16 + var v: Int32 { Int32(self.rawValue) } + var p: VOffset { self.rawValue } + } + + public var typeType: org_apache_arrow_flatbuf_Type_ { + let o = _accessor.offset(VTOFFSET.typeType.v) + return o == 0 + ? .none_ + : org_apache_arrow_flatbuf_Type_(rawValue: _accessor.readBuffer(of: UInt8.self, at: o)) + ?? .none_ + } + /// The type of data contained in a value cell. + /// Currently only fixed-width value types are supported, + /// no strings or nested types. + public func type(type: T.Type) -> T! { + let o = _accessor.offset(VTOFFSET.type.v) + return _accessor.union(o) + } + /// The dimensions of the tensor, optionally named. + public var hasShape: Bool { + let o = _accessor.offset(VTOFFSET.shape.v) + return o == 0 ? false : true + } + public var shapeCount: Int32 { + let o = _accessor.offset(VTOFFSET.shape.v) + return o == 0 ? 0 : _accessor.vector(count: o) + } + public func shape(at index: Int32) -> org_apache_arrow_flatbuf_TensorDim? { + let o = _accessor.offset(VTOFFSET.shape.v) + return o == 0 + ? nil + : org_apache_arrow_flatbuf_TensorDim( + _accessor.bb, o: _accessor.indirect(_accessor.vector(at: o) + index * 4)) + } + /// The number of non-zero values in a sparse tensor. + public var nonZeroLength: Int64 { + let o = _accessor.offset(VTOFFSET.nonZeroLength.v) + return o == 0 ? 0 : _accessor.readBuffer(of: Int64.self, at: o) + } + public var sparseIndexType: org_apache_arrow_flatbuf_SparseTensorIndex { + let o = _accessor.offset(VTOFFSET.sparseIndexType.v) + return o == 0 + ? .none_ + : org_apache_arrow_flatbuf_SparseTensorIndex( + rawValue: _accessor.readBuffer(of: UInt8.self, at: o)) ?? .none_ + } + /// Sparse tensor index + public func sparseIndex(type: T.Type) -> T! { + let o = _accessor.offset(VTOFFSET.sparseIndex.v) + return _accessor.union(o) + } + /// The location and size of the tensor's data + public var data: org_apache_arrow_flatbuf_Buffer! { + let o = _accessor.offset(VTOFFSET.data.v) + return _accessor.readBuffer(of: org_apache_arrow_flatbuf_Buffer.self, at: o) + } + public var mutableData: org_apache_arrow_flatbuf_Buffer_Mutable! { + let o = _accessor.offset(VTOFFSET.data.v) + return org_apache_arrow_flatbuf_Buffer_Mutable(_accessor.bb, o: o + _accessor.postion) + } + public static func startSparseTensor(_ fbb: inout FlatBufferBuilder) -> UOffset { + fbb.startTable(with: 7) + } + public static func add(typeType: org_apache_arrow_flatbuf_Type_, _ fbb: inout FlatBufferBuilder) { + fbb.add(element: typeType.rawValue, def: 0, at: VTOFFSET.typeType.p) + } + public static func add(type: Offset, _ fbb: inout FlatBufferBuilder) { + fbb.add(offset: type, at: VTOFFSET.type.p) + } + public static func addVectorOf(shape: Offset, _ fbb: inout FlatBufferBuilder) { + fbb.add(offset: shape, at: VTOFFSET.shape.p) + } + public static func add(nonZeroLength: Int64, _ fbb: inout FlatBufferBuilder) { + fbb.add(element: nonZeroLength, def: 0, at: VTOFFSET.nonZeroLength.p) + } + public static func add( + sparseIndexType: org_apache_arrow_flatbuf_SparseTensorIndex, _ fbb: inout FlatBufferBuilder + ) { fbb.add(element: sparseIndexType.rawValue, def: 0, at: VTOFFSET.sparseIndexType.p) } + public static func add(sparseIndex: Offset, _ fbb: inout FlatBufferBuilder) { + fbb.add(offset: sparseIndex, at: VTOFFSET.sparseIndex.p) + } + public static func add(data: org_apache_arrow_flatbuf_Buffer?, _ fbb: inout FlatBufferBuilder) { + guard let data = data else { return } + fbb.create(struct: data, position: VTOFFSET.data.p) + } + public static func endSparseTensor(_ fbb: inout FlatBufferBuilder, start: UOffset) -> Offset { + let end = Offset(offset: fbb.endTable(at: start)) + fbb.require(table: end, fields: [6, 8, 14, 16]) + return end + } + public static func createSparseTensor( + _ fbb: inout FlatBufferBuilder, + typeType: org_apache_arrow_flatbuf_Type_ = .none_, + typeOffset type: Offset, + shapeVectorOffset shape: Offset, + nonZeroLength: Int64 = 0, + sparseIndexType: org_apache_arrow_flatbuf_SparseTensorIndex = .none_, + sparseIndexOffset sparseIndex: Offset, + data: org_apache_arrow_flatbuf_Buffer + ) -> Offset { + let __start = org_apache_arrow_flatbuf_SparseTensor.startSparseTensor(&fbb) + org_apache_arrow_flatbuf_SparseTensor.add(typeType: typeType, &fbb) + org_apache_arrow_flatbuf_SparseTensor.add(type: type, &fbb) + org_apache_arrow_flatbuf_SparseTensor.addVectorOf(shape: shape, &fbb) + org_apache_arrow_flatbuf_SparseTensor.add(nonZeroLength: nonZeroLength, &fbb) + org_apache_arrow_flatbuf_SparseTensor.add(sparseIndexType: sparseIndexType, &fbb) + org_apache_arrow_flatbuf_SparseTensor.add(sparseIndex: sparseIndex, &fbb) + org_apache_arrow_flatbuf_SparseTensor.add(data: data, &fbb) + return org_apache_arrow_flatbuf_SparseTensor.endSparseTensor(&fbb, start: __start) + } + + public static func verify(_ verifier: inout Verifier, at position: Int, of type: T.Type) throws + where T: Verifiable { + var _v = try verifier.visitTable(at: position) + try _v.visit( + unionKey: VTOFFSET.typeType.p, unionField: VTOFFSET.type.p, unionKeyName: "typeType", + fieldName: "type", required: true, + completion: { (verifier, key: org_apache_arrow_flatbuf_Type_, pos) in + switch key { + case .none_: + break // NOTE - SWIFT doesnt support none + case .null: + try ForwardOffset.verify( + &verifier, at: pos, of: org_apache_arrow_flatbuf_Null.self) + case .int: + try ForwardOffset.verify( + &verifier, at: pos, of: org_apache_arrow_flatbuf_Int.self) + case .floatingpoint: + try ForwardOffset.verify( + &verifier, at: pos, of: org_apache_arrow_flatbuf_FloatingPoint.self) + case .binary: + try ForwardOffset.verify( + &verifier, at: pos, of: org_apache_arrow_flatbuf_Binary.self) + case .utf8: + try ForwardOffset.verify( + &verifier, at: pos, of: org_apache_arrow_flatbuf_Utf8.self) + case .bool: + try ForwardOffset.verify( + &verifier, at: pos, of: org_apache_arrow_flatbuf_Bool.self) + case .decimal: + try ForwardOffset.verify( + &verifier, at: pos, of: org_apache_arrow_flatbuf_Decimal.self) + case .date: + try ForwardOffset.verify( + &verifier, at: pos, of: org_apache_arrow_flatbuf_Date.self) + case .time: + try ForwardOffset.verify( + &verifier, at: pos, of: org_apache_arrow_flatbuf_Time.self) + case .timestamp: + try ForwardOffset.verify( + &verifier, at: pos, of: org_apache_arrow_flatbuf_Timestamp.self) + case .interval: + try ForwardOffset.verify( + &verifier, at: pos, of: org_apache_arrow_flatbuf_Interval.self) + case .list: + try ForwardOffset.verify( + &verifier, at: pos, of: org_apache_arrow_flatbuf_List.self) + case .struct_: + try ForwardOffset.verify( + &verifier, at: pos, of: org_apache_arrow_flatbuf_Struct_.self) + case .union: + try ForwardOffset.verify( + &verifier, at: pos, of: org_apache_arrow_flatbuf_Union.self) + case .fixedsizebinary: + try ForwardOffset.verify( + &verifier, at: pos, of: org_apache_arrow_flatbuf_FixedSizeBinary.self) + case .fixedsizelist: + try ForwardOffset.verify( + &verifier, at: pos, of: org_apache_arrow_flatbuf_FixedSizeList.self) + case .map: + try ForwardOffset.verify( + &verifier, at: pos, of: org_apache_arrow_flatbuf_Map.self) + case .duration: + try ForwardOffset.verify( + &verifier, at: pos, of: org_apache_arrow_flatbuf_Duration.self) + case .largebinary: + try ForwardOffset.verify( + &verifier, at: pos, of: org_apache_arrow_flatbuf_LargeBinary.self) + case .largeutf8: + try ForwardOffset.verify( + &verifier, at: pos, of: org_apache_arrow_flatbuf_LargeUtf8.self) + case .largelist: + try ForwardOffset.verify( + &verifier, at: pos, of: org_apache_arrow_flatbuf_LargeList.self) + case .runendencoded: + try ForwardOffset.verify( + &verifier, at: pos, of: org_apache_arrow_flatbuf_RunEndEncoded.self) + } + }) + try _v.visit( + field: VTOFFSET.shape.p, fieldName: "shape", required: true, + type: ForwardOffset< + Vector< + ForwardOffset, org_apache_arrow_flatbuf_TensorDim + > + >.self) + try _v.visit( + field: VTOFFSET.nonZeroLength.p, fieldName: "nonZeroLength", required: false, type: Int64.self + ) + try _v.visit( + unionKey: VTOFFSET.sparseIndexType.p, unionField: VTOFFSET.sparseIndex.p, + unionKeyName: "sparseIndexType", fieldName: "sparseIndex", required: true, + completion: { (verifier, key: org_apache_arrow_flatbuf_SparseTensorIndex, pos) in + switch key { + case .none_: + break // NOTE - SWIFT doesnt support none + case .sparsetensorindexcoo: + try ForwardOffset.verify( + &verifier, at: pos, of: org_apache_arrow_flatbuf_SparseTensorIndexCOO.self) + case .sparsematrixindexcsx: + try ForwardOffset.verify( + &verifier, at: pos, of: org_apache_arrow_flatbuf_SparseMatrixIndexCSX.self) + case .sparsetensorindexcsf: + try ForwardOffset.verify( + &verifier, at: pos, of: org_apache_arrow_flatbuf_SparseTensorIndexCSF.self) + } + }) + try _v.visit( + field: VTOFFSET.data.p, fieldName: "data", required: true, + type: org_apache_arrow_flatbuf_Buffer.self) + _v.finish() + } +} diff --git a/Sources/SparkConnect/Tensor_generated.swift b/Sources/SparkConnect/Tensor_generated.swift new file mode 100644 index 0000000..e33469a --- /dev/null +++ b/Sources/SparkConnect/Tensor_generated.swift @@ -0,0 +1,309 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +// automatically generated by the FlatBuffers compiler, do not modify +// swiftlint:disable all +// swiftformat:disable all + +import FlatBuffers + +/// ---------------------------------------------------------------------- +/// Data structures for dense tensors +/// Shape data for a single axis in a tensor +public struct org_apache_arrow_flatbuf_TensorDim: FlatBufferObject, Verifiable { + + static func validateVersion() { FlatBuffersVersion_23_1_4() } + public var __buffer: ByteBuffer! { return _accessor.bb } + private var _accessor: Table + + public static func getRootAsTensorDim(bb: ByteBuffer) -> org_apache_arrow_flatbuf_TensorDim { + return org_apache_arrow_flatbuf_TensorDim( + Table( + bb: bb, position: Int32(bb.read(def: UOffset.self, position: bb.reader)) + Int32(bb.reader)) + ) + } + + private init(_ t: Table) { _accessor = t } + public init(_ bb: ByteBuffer, o: Int32) { _accessor = Table(bb: bb, position: o) } + + private enum VTOFFSET: VOffset { + case size = 4 + case name = 6 + var v: Int32 { Int32(self.rawValue) } + var p: VOffset { self.rawValue } + } + + /// Length of dimension + public var size: Int64 { + let o = _accessor.offset(VTOFFSET.size.v) + return o == 0 ? 0 : _accessor.readBuffer(of: Int64.self, at: o) + } + /// Name of the dimension, optional + public var name: String? { + let o = _accessor.offset(VTOFFSET.name.v) + return o == 0 ? nil : _accessor.string(at: o) + } + public var nameSegmentArray: [UInt8]? { return _accessor.getVector(at: VTOFFSET.name.v) } + public static func startTensorDim(_ fbb: inout FlatBufferBuilder) -> UOffset { + fbb.startTable(with: 2) + } + public static func add(size: Int64, _ fbb: inout FlatBufferBuilder) { + fbb.add(element: size, def: 0, at: VTOFFSET.size.p) + } + public static func add(name: Offset, _ fbb: inout FlatBufferBuilder) { + fbb.add(offset: name, at: VTOFFSET.name.p) + } + public static func endTensorDim(_ fbb: inout FlatBufferBuilder, start: UOffset) -> Offset { + let end = Offset(offset: fbb.endTable(at: start)) + return end + } + public static func createTensorDim( + _ fbb: inout FlatBufferBuilder, + size: Int64 = 0, + nameOffset name: Offset = Offset() + ) -> Offset { + let __start = org_apache_arrow_flatbuf_TensorDim.startTensorDim(&fbb) + org_apache_arrow_flatbuf_TensorDim.add(size: size, &fbb) + org_apache_arrow_flatbuf_TensorDim.add(name: name, &fbb) + return org_apache_arrow_flatbuf_TensorDim.endTensorDim(&fbb, start: __start) + } + + public static func verify(_ verifier: inout Verifier, at position: Int, of type: T.Type) throws + where T: Verifiable { + var _v = try verifier.visitTable(at: position) + try _v.visit(field: VTOFFSET.size.p, fieldName: "size", required: false, type: Int64.self) + try _v.visit( + field: VTOFFSET.name.p, fieldName: "name", required: false, type: ForwardOffset.self) + _v.finish() + } +} + +public struct org_apache_arrow_flatbuf_Tensor: FlatBufferObject, Verifiable { + + static func validateVersion() { FlatBuffersVersion_23_1_4() } + public var __buffer: ByteBuffer! { return _accessor.bb } + private var _accessor: Table + + public static func getRootAsTensor(bb: ByteBuffer) -> org_apache_arrow_flatbuf_Tensor { + return org_apache_arrow_flatbuf_Tensor( + Table( + bb: bb, position: Int32(bb.read(def: UOffset.self, position: bb.reader)) + Int32(bb.reader)) + ) + } + + private init(_ t: Table) { _accessor = t } + public init(_ bb: ByteBuffer, o: Int32) { _accessor = Table(bb: bb, position: o) } + + private enum VTOFFSET: VOffset { + case typeType = 4 + case type = 6 + case shape = 8 + case strides = 10 + case data = 12 + var v: Int32 { Int32(self.rawValue) } + var p: VOffset { self.rawValue } + } + + public var typeType: org_apache_arrow_flatbuf_Type_ { + let o = _accessor.offset(VTOFFSET.typeType.v) + return o == 0 + ? .none_ + : org_apache_arrow_flatbuf_Type_(rawValue: _accessor.readBuffer(of: UInt8.self, at: o)) + ?? .none_ + } + /// The type of data contained in a value cell. Currently only fixed-width + /// value types are supported, no strings or nested types + public func type(type: T.Type) -> T! { + let o = _accessor.offset(VTOFFSET.type.v) + return _accessor.union(o) + } + /// The dimensions of the tensor, optionally named + public var hasShape: Bool { + let o = _accessor.offset(VTOFFSET.shape.v) + return o == 0 ? false : true + } + public var shapeCount: Int32 { + let o = _accessor.offset(VTOFFSET.shape.v) + return o == 0 ? 0 : _accessor.vector(count: o) + } + public func shape(at index: Int32) -> org_apache_arrow_flatbuf_TensorDim? { + let o = _accessor.offset(VTOFFSET.shape.v) + return o == 0 + ? nil + : org_apache_arrow_flatbuf_TensorDim( + _accessor.bb, o: _accessor.indirect(_accessor.vector(at: o) + index * 4)) + } + /// Non-negative byte offsets to advance one value cell along each dimension + /// If omitted, default to row-major order (C-like). + public var hasStrides: Bool { + let o = _accessor.offset(VTOFFSET.strides.v) + return o == 0 ? false : true + } + public var stridesCount: Int32 { + let o = _accessor.offset(VTOFFSET.strides.v) + return o == 0 ? 0 : _accessor.vector(count: o) + } + public func strides(at index: Int32) -> Int64 { + let o = _accessor.offset(VTOFFSET.strides.v) + return o == 0 + ? 0 : _accessor.directRead(of: Int64.self, offset: _accessor.vector(at: o) + index * 8) + } + public var strides: [Int64] { return _accessor.getVector(at: VTOFFSET.strides.v) ?? [] } + /// The location and size of the tensor's data + public var data: org_apache_arrow_flatbuf_Buffer! { + let o = _accessor.offset(VTOFFSET.data.v) + return _accessor.readBuffer(of: org_apache_arrow_flatbuf_Buffer.self, at: o) + } + public var mutableData: org_apache_arrow_flatbuf_Buffer_Mutable! { + let o = _accessor.offset(VTOFFSET.data.v) + return org_apache_arrow_flatbuf_Buffer_Mutable(_accessor.bb, o: o + _accessor.postion) + } + public static func startTensor(_ fbb: inout FlatBufferBuilder) -> UOffset { + fbb.startTable(with: 5) + } + public static func add(typeType: org_apache_arrow_flatbuf_Type_, _ fbb: inout FlatBufferBuilder) { + fbb.add(element: typeType.rawValue, def: 0, at: VTOFFSET.typeType.p) + } + public static func add(type: Offset, _ fbb: inout FlatBufferBuilder) { + fbb.add(offset: type, at: VTOFFSET.type.p) + } + public static func addVectorOf(shape: Offset, _ fbb: inout FlatBufferBuilder) { + fbb.add(offset: shape, at: VTOFFSET.shape.p) + } + public static func addVectorOf(strides: Offset, _ fbb: inout FlatBufferBuilder) { + fbb.add(offset: strides, at: VTOFFSET.strides.p) + } + public static func add(data: org_apache_arrow_flatbuf_Buffer?, _ fbb: inout FlatBufferBuilder) { + guard let data = data else { return } + fbb.create(struct: data, position: VTOFFSET.data.p) + } + public static func endTensor(_ fbb: inout FlatBufferBuilder, start: UOffset) -> Offset { + let end = Offset(offset: fbb.endTable(at: start)) + fbb.require(table: end, fields: [6, 8, 12]) + return end + } + public static func createTensor( + _ fbb: inout FlatBufferBuilder, + typeType: org_apache_arrow_flatbuf_Type_ = .none_, + typeOffset type: Offset, + shapeVectorOffset shape: Offset, + stridesVectorOffset strides: Offset = Offset(), + data: org_apache_arrow_flatbuf_Buffer + ) -> Offset { + let __start = org_apache_arrow_flatbuf_Tensor.startTensor(&fbb) + org_apache_arrow_flatbuf_Tensor.add(typeType: typeType, &fbb) + org_apache_arrow_flatbuf_Tensor.add(type: type, &fbb) + org_apache_arrow_flatbuf_Tensor.addVectorOf(shape: shape, &fbb) + org_apache_arrow_flatbuf_Tensor.addVectorOf(strides: strides, &fbb) + org_apache_arrow_flatbuf_Tensor.add(data: data, &fbb) + return org_apache_arrow_flatbuf_Tensor.endTensor(&fbb, start: __start) + } + + public static func verify(_ verifier: inout Verifier, at position: Int, of type: T.Type) throws + where T: Verifiable { + var _v = try verifier.visitTable(at: position) + try _v.visit( + unionKey: VTOFFSET.typeType.p, unionField: VTOFFSET.type.p, unionKeyName: "typeType", + fieldName: "type", required: true, + completion: { (verifier, key: org_apache_arrow_flatbuf_Type_, pos) in + switch key { + case .none_: + break // NOTE - SWIFT doesnt support none + case .null: + try ForwardOffset.verify( + &verifier, at: pos, of: org_apache_arrow_flatbuf_Null.self) + case .int: + try ForwardOffset.verify( + &verifier, at: pos, of: org_apache_arrow_flatbuf_Int.self) + case .floatingpoint: + try ForwardOffset.verify( + &verifier, at: pos, of: org_apache_arrow_flatbuf_FloatingPoint.self) + case .binary: + try ForwardOffset.verify( + &verifier, at: pos, of: org_apache_arrow_flatbuf_Binary.self) + case .utf8: + try ForwardOffset.verify( + &verifier, at: pos, of: org_apache_arrow_flatbuf_Utf8.self) + case .bool: + try ForwardOffset.verify( + &verifier, at: pos, of: org_apache_arrow_flatbuf_Bool.self) + case .decimal: + try ForwardOffset.verify( + &verifier, at: pos, of: org_apache_arrow_flatbuf_Decimal.self) + case .date: + try ForwardOffset.verify( + &verifier, at: pos, of: org_apache_arrow_flatbuf_Date.self) + case .time: + try ForwardOffset.verify( + &verifier, at: pos, of: org_apache_arrow_flatbuf_Time.self) + case .timestamp: + try ForwardOffset.verify( + &verifier, at: pos, of: org_apache_arrow_flatbuf_Timestamp.self) + case .interval: + try ForwardOffset.verify( + &verifier, at: pos, of: org_apache_arrow_flatbuf_Interval.self) + case .list: + try ForwardOffset.verify( + &verifier, at: pos, of: org_apache_arrow_flatbuf_List.self) + case .struct_: + try ForwardOffset.verify( + &verifier, at: pos, of: org_apache_arrow_flatbuf_Struct_.self) + case .union: + try ForwardOffset.verify( + &verifier, at: pos, of: org_apache_arrow_flatbuf_Union.self) + case .fixedsizebinary: + try ForwardOffset.verify( + &verifier, at: pos, of: org_apache_arrow_flatbuf_FixedSizeBinary.self) + case .fixedsizelist: + try ForwardOffset.verify( + &verifier, at: pos, of: org_apache_arrow_flatbuf_FixedSizeList.self) + case .map: + try ForwardOffset.verify( + &verifier, at: pos, of: org_apache_arrow_flatbuf_Map.self) + case .duration: + try ForwardOffset.verify( + &verifier, at: pos, of: org_apache_arrow_flatbuf_Duration.self) + case .largebinary: + try ForwardOffset.verify( + &verifier, at: pos, of: org_apache_arrow_flatbuf_LargeBinary.self) + case .largeutf8: + try ForwardOffset.verify( + &verifier, at: pos, of: org_apache_arrow_flatbuf_LargeUtf8.self) + case .largelist: + try ForwardOffset.verify( + &verifier, at: pos, of: org_apache_arrow_flatbuf_LargeList.self) + case .runendencoded: + try ForwardOffset.verify( + &verifier, at: pos, of: org_apache_arrow_flatbuf_RunEndEncoded.self) + } + }) + try _v.visit( + field: VTOFFSET.shape.p, fieldName: "shape", required: true, + type: ForwardOffset< + Vector< + ForwardOffset, org_apache_arrow_flatbuf_TensorDim + > + >.self) + try _v.visit( + field: VTOFFSET.strides.p, fieldName: "strides", required: false, + type: ForwardOffset>.self) + try _v.visit( + field: VTOFFSET.data.p, fieldName: "data", required: true, + type: org_apache_arrow_flatbuf_Buffer.self) + _v.finish() + } +}