Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
12 changes: 11 additions & 1 deletion Sources/SparkConnect/DataFrame.swift
Original file line number Diff line number Diff line change
Expand Up @@ -113,6 +113,8 @@ import Synchronization
/// - ``unpivot(_:_:_:_:)``
/// - ``melt(_:_:_:)``
/// - ``melt(_:_:_:_:)``
/// - ``transpose()``
/// - ``transpose(_:)``
///
/// ### Join Operations
/// - ``join(_:)``
Expand Down Expand Up @@ -166,6 +168,7 @@ import Synchronization
/// ### Write Operations
/// - ``write``
/// - ``writeTo(_:)``
/// - ``writeStream``
///
/// ### Sampling
/// - ``sample(_:_:_:)``
Expand Down Expand Up @@ -1381,7 +1384,7 @@ public actor DataFrame: Sendable {
/// Returns a ``DataFrameWriter`` that can be used to write non-streaming data.
public var write: DataFrameWriter {
get {
return DataFrameWriter(df: self)
DataFrameWriter(df: self)
}
}

Expand All @@ -1391,4 +1394,11 @@ public actor DataFrame: Sendable {
public func writeTo(_ table: String) -> DataFrameWriterV2 {
return DataFrameWriterV2(table, self)
}

/// Returns a ``DataStreamWriter`` that can be used to write streaming data.
public var writeStream: DataStreamWriter {
get {
DataStreamWriter(df: self)
}
}
}
221 changes: 221 additions & 0 deletions Sources/SparkConnect/DataStreamReader.swift
Original file line number Diff line number Diff line change
@@ -0,0 +1,221 @@
//
// 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

/// An actor to load a streaming `Dataset` from external storage systems
/// (e.g. file systems, key-value stores, etc). Use `SparkSession.readStream` to access this.
public actor DataStreamReader: Sendable {
var source: String = ""

var paths: [String] = []

var extraOptions: CaseInsensitiveDictionary = CaseInsensitiveDictionary([:])

var userSpecifiedSchemaDDL: String? = nil

let sparkSession: SparkSession

init(sparkSession: SparkSession) {
self.sparkSession = sparkSession
}

/// Specifies the input data source format.
/// - Parameter source: A string.
/// - Returns: A ``DataStreamReader``.
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
/// - Returns: A ``DataStreamReader``.
/// - Returns: A `DataStreamReader`.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Just following other methods in this PR.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Or maybe other methods are incorrect but this is correct?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Both are correct~ A single backquote is for codifying. A double-backquote is for codifying and linking.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yea, I guess so. Just checked if a consistent style could be better.

public func format(_ source: String) -> DataStreamReader {
self.source = source
return self
}

/// Specifies the input schema. Some data sources (e.g. JSON) can infer the input schema
/// automatically from data. By specifying the schema here, the underlying data source can skip
/// the schema inference step, and thus speed up data loading.
/// - Parameter schema: A DDL schema string.
/// - Returns: A `DataStreamReader`.
@discardableResult
public func schema(_ schema: String) async throws -> DataStreamReader {
// Validate by parsing.
do {
try await sparkSession.client.ddlParse(schema)
} catch {
throw SparkConnectError.InvalidTypeException
}
self.userSpecifiedSchemaDDL = schema
return self
}

/// Adds an input option for the underlying data source.
/// - Parameters:
/// - key: A key string.
/// - value: A value string.
/// - Returns: A `DataStreamReader`.
public func option(_ key: String, _ value: String) -> DataStreamReader {
self.extraOptions[key] = value
return self
}

/// Adds an input option for the underlying data source.
/// - Parameters:
/// - key: A key string.
/// - value: A `Bool` value.
/// - Returns: A `DataStreamReader`.
public func option(_ key: String, _ value: Bool) -> DataStreamReader {
self.extraOptions[key] = String(value)
return self
}

/// Adds an input option for the underlying data source.
/// - Parameters:
/// - key: A key string.
/// - value: A `Int64` value.
/// - Returns: A `DataStreamReader`.
public func option(_ key: String, _ value: Int64) -> DataStreamReader {
self.extraOptions[key] = String(value)
return self
}

/// Adds an input option for the underlying data source.
/// - Parameters:
/// - key: A key string.
/// - value: A `Double` value.
/// - Returns: A `DataStreamReader`.
public func option(_ key: String, _ value: Double) -> DataStreamReader {
self.extraOptions[key] = String(value)
return self
}

/// Adds input options for the underlying data source.
/// - Parameter options: A string-string dictionary.
/// - Returns: A `DataStreamReader`.
public func options(_ options: [String: String]) -> DataStreamReader {
for (key, value) in options {
self.extraOptions[key] = value
}
return self
}

/// Loads input data stream in as a `DataFrame`, for data streams that don't require a path
/// (e.g. external key-value stores).
/// - Returns: A `DataFrame`.
public func load() -> DataFrame {
return load([])
}

/// Loads input data stream in as a `DataFrame`, for data streams that require a path
/// (e.g. data backed by a local or distributed file system).
/// - Parameter path: A path string.
/// - Returns: A `DataFrame`.
public func load(_ path: String) -> DataFrame {
return load([path])
}

func load(_ paths: [String]) -> DataFrame {
self.paths = paths

var dataSource = DataSource()
dataSource.format = self.source
dataSource.paths = self.paths
dataSource.options = self.extraOptions.toStringDictionary()
if let userSpecifiedSchemaDDL = self.userSpecifiedSchemaDDL {
dataSource.schema = userSpecifiedSchemaDDL
}

var read = Read()
read.dataSource = dataSource
read.isStreaming = true

var relation = Relation()
relation.read = read

var plan = Plan()
plan.opType = .root(relation)

return DataFrame(spark: sparkSession, plan: plan)
}

/// Define a Streaming DataFrame on a Table. The DataSource corresponding to the table should
/// support streaming mode.
/// - Parameter tableName: The name of the table.
/// - Returns: A ``DataFrame``.
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
/// - Returns: A ``DataFrame``.
/// - Returns: A `DataFrame`.

public func table(_ tableName: String) -> DataFrame {
var namedTable = NamedTable()
namedTable.unparsedIdentifier = tableName
namedTable.options = self.extraOptions.toStringDictionary()

var read = Read()
read.namedTable = namedTable
read.isStreaming = true

var relation = Relation()
relation.read = read

var plan = Plan()
plan.opType = .root(relation)

return DataFrame(spark: sparkSession, plan: plan)
}

/// Loads a text file stream and returns the result as a `DataFrame`.
/// - Parameter path: A path string
/// - Returns: A `DataFrame`.
public func text(_ path: String) -> DataFrame {
self.source = "text"
return load(path)
}

/// Loads a CSV file stream and returns the result as a `DataFrame`.
/// - Parameter path: A path string
/// - Returns: A `DataFrame`.
public func csv(_ path: String) -> DataFrame {
self.source = "csv"
return load(path)
}

/// Loads a JSON file stream and returns the result as a `DataFrame`.
/// - Parameter path: A path string
/// - Returns: A `DataFrame`.
public func json(_ path: String) -> DataFrame {
self.source = "json"
return load(path)
}

/// Loads an XML file stream and returns the result as a `DataFrame`.
/// - Parameter path: A path string
/// - Returns: A `DataFrame`.
public func xml(_ path: String) -> DataFrame {
self.source = "xml"
return load(path)
}

/// Loads an ORC file stream and returns the result as a `DataFrame`.
/// - Parameter path: A path string
/// - Returns: A `DataFrame`.
public func orc(_ path: String) -> DataFrame {
self.source = "orc"
return load(path)
}

/// Loads a Parquet file stream and returns the result as a `DataFrame`.
/// - Parameter path: A path string
/// - Returns: A `DataFrame`.
public func parquet(_ path: String) -> DataFrame {
self.source = "parquet"
return load(path)
}
}
Loading
Loading