spark-connect 0.2.0

This diff represents the content of publicly available package versions that have been released to one of the supported registries. The information contained in this diff is provided for informational purposes only and reflects changes between package versions as they appear in their respective public registries.
Files changed (53) hide show
  1. checksums.yaml +7 -0
  2. data/CHANGELOG.md +82 -0
  3. data/LICENSE +202 -0
  4. data/NOTICE +16 -0
  5. data/README.md +166 -0
  6. data/lib/spark-connect.rb +5 -0
  7. data/lib/spark_connect/arrow.rb +115 -0
  8. data/lib/spark_connect/catalog.rb +190 -0
  9. data/lib/spark_connect/channel_builder.rb +134 -0
  10. data/lib/spark_connect/client.rb +264 -0
  11. data/lib/spark_connect/column.rb +379 -0
  12. data/lib/spark_connect/conf.rb +79 -0
  13. data/lib/spark_connect/data_frame.rb +828 -0
  14. data/lib/spark_connect/errors.rb +58 -0
  15. data/lib/spark_connect/functions.rb +903 -0
  16. data/lib/spark_connect/grouped_data.rb +101 -0
  17. data/lib/spark_connect/na_functions.rb +98 -0
  18. data/lib/spark_connect/observation.rb +61 -0
  19. data/lib/spark_connect/pipelines.rb +221 -0
  20. data/lib/spark_connect/plan.rb +39 -0
  21. data/lib/spark_connect/proto/spark/connect/base_pb.rb +118 -0
  22. data/lib/spark_connect/proto/spark/connect/base_services_pb.rb +82 -0
  23. data/lib/spark_connect/proto/spark/connect/catalog_pb.rb +46 -0
  24. data/lib/spark_connect/proto/spark/connect/commands_pb.rb +67 -0
  25. data/lib/spark_connect/proto/spark/connect/common_pb.rb +32 -0
  26. data/lib/spark_connect/proto/spark/connect/expressions_pb.rb +63 -0
  27. data/lib/spark_connect/proto/spark/connect/ml_common_pb.rb +22 -0
  28. data/lib/spark_connect/proto/spark/connect/ml_pb.rb +32 -0
  29. data/lib/spark_connect/proto/spark/connect/pipelines_pb.rb +45 -0
  30. data/lib/spark_connect/proto/spark/connect/relations_pb.rb +102 -0
  31. data/lib/spark_connect/proto/spark/connect/types_pb.rb +46 -0
  32. data/lib/spark_connect/proto.rb +32 -0
  33. data/lib/spark_connect/reader.rb +98 -0
  34. data/lib/spark_connect/row.rb +105 -0
  35. data/lib/spark_connect/session.rb +317 -0
  36. data/lib/spark_connect/stat_functions.rb +109 -0
  37. data/lib/spark_connect/streaming.rb +351 -0
  38. data/lib/spark_connect/types.rb +490 -0
  39. data/lib/spark_connect/version.rb +11 -0
  40. data/lib/spark_connect/window.rb +119 -0
  41. data/lib/spark_connect/writer.rb +208 -0
  42. data/lib/spark_connect.rb +58 -0
  43. data/proto/spark/connect/base.proto +1275 -0
  44. data/proto/spark/connect/catalog.proto +243 -0
  45. data/proto/spark/connect/commands.proto +553 -0
  46. data/proto/spark/connect/common.proto +179 -0
  47. data/proto/spark/connect/expressions.proto +557 -0
  48. data/proto/spark/connect/ml.proto +147 -0
  49. data/proto/spark/connect/ml_common.proto +64 -0
  50. data/proto/spark/connect/pipelines.proto +307 -0
  51. data/proto/spark/connect/relations.proto +1252 -0
  52. data/proto/spark/connect/types.proto +227 -0
  53. metadata +149 -0
@@ -0,0 +1,317 @@
1
+ # frozen_string_literal: true
2
+
3
+ require "securerandom"
4
+
5
+ module SparkConnect
6
+ # The entry point to programming Spark with the DataFrame API over Spark
7
+ # Connect. Create one with the {Builder}:
8
+ #
9
+ # @example
10
+ # spark = SparkConnect::SparkSession.builder
11
+ # .remote("sc://localhost:15002")
12
+ # .app_name("my-app")
13
+ # .get_or_create
14
+ #
15
+ # A session owns the underlying {SparkConnectClient}, a monotonic plan-id
16
+ # allocator (so each relation is uniquely identifiable to the server), and the
17
+ # {RuntimeConfig} and {Catalog} facades.
18
+ class SparkSession
19
+ Proto = SparkConnect::Proto
20
+
21
+ # @return [SparkConnectClient]
22
+ attr_reader :client
23
+
24
+ # @param client [SparkConnectClient]
25
+ def initialize(client)
26
+ @client = client
27
+ @plan_id = -1
28
+ @conf = RuntimeConfig.new(client)
29
+ end
30
+
31
+ class << self
32
+ # @return [Builder] a new session builder.
33
+ def builder
34
+ Builder.new
35
+ end
36
+
37
+ # The currently active session set by {#set_active} / {Builder#get_or_create}.
38
+ # @return [SparkSession, nil]
39
+ attr_accessor :active
40
+
41
+ # @api private
42
+ end
43
+
44
+ # Allocate the next unique plan id. Used by {PlanBuilder.relation}.
45
+ # @return [Integer]
46
+ def next_plan_id
47
+ @plan_id += 1
48
+ end
49
+
50
+ # @return [String] the client session id (UUID).
51
+ def session_id
52
+ @client.session_id
53
+ end
54
+
55
+ # Create a {DataFrame} with a single `id` column over the given integer range.
56
+ #
57
+ # @overload range(end_)
58
+ # @overload range(start, end_, step = 1, num_partitions = nil)
59
+ # @return [DataFrame]
60
+ def range(start, end_ = nil, step = 1, num_partitions = nil)
61
+ if end_.nil?
62
+ end_ = start
63
+ start = 0
64
+ end
65
+ r = Proto::Range.new(start: start, end: end_, step: step)
66
+ r.num_partitions = num_partitions if num_partitions
67
+ DataFrame.new(self, PlanBuilder.relation(self, range: r))
68
+ end
69
+
70
+ # Execute a SQL query and return a lazy {DataFrame} over its result.
71
+ #
72
+ # @param query [String]
73
+ # @param args [Hash{String=>Object}, Array<Object>, nil] named or positional
74
+ # parameters bound into the query.
75
+ # @return [DataFrame]
76
+ def sql(query, args = nil)
77
+ sql = Proto::SQL.new(query: query)
78
+ case args
79
+ when Hash
80
+ args.each { |k, v| sql.named_arguments[k.to_s] = Column.to_col(v).to_expr }
81
+ when Array
82
+ sql.pos_arguments += args.map { |v| Column.to_col(v).to_expr }
83
+ end
84
+ DataFrame.new(self, PlanBuilder.relation(self, sql: sql))
85
+ end
86
+
87
+ # Return a {DataFrame} reading the named table or view.
88
+ #
89
+ # @param name [String]
90
+ # @return [DataFrame]
91
+ def table(name)
92
+ read.table(name)
93
+ end
94
+
95
+ # @return [DataFrameReader] interface for loading external data.
96
+ def read
97
+ DataFrameReader.new(self)
98
+ end
99
+
100
+ # @return [DataStreamReader] interface for loading a streaming DataFrame.
101
+ def read_stream
102
+ DataStreamReader.new(self)
103
+ end
104
+ alias readStream read_stream
105
+
106
+ # @return [StreamingQueryManager] the manager for this session's streaming queries.
107
+ def streams
108
+ StreamingQueryManager.new(self)
109
+ end
110
+
111
+ # Create a new Spark Declarative Pipeline (a dataflow graph) in this session.
112
+ #
113
+ # @param default_catalog [String, nil]
114
+ # @param default_database [String, nil]
115
+ # @param sql_conf [Hash{String=>String}] SQL configs applied to all flows.
116
+ # @return [Pipeline]
117
+ def pipeline(default_catalog: nil, default_database: nil, sql_conf: {})
118
+ Pipeline.new(self, default_catalog: default_catalog, default_database: default_database, sql_conf: sql_conf)
119
+ end
120
+
121
+ # Build a {DataFrame} from local Ruby data.
122
+ #
123
+ # @param data [Array<Hash>, Array<Array>, Array<Row>]
124
+ # @param schema [Types::StructType, Array<String>, String, nil] an explicit
125
+ # schema, a list of column names, a DDL string, or `nil` to infer.
126
+ # @return [DataFrame]
127
+ def create_data_frame(data, schema = nil)
128
+ data = data.to_a
129
+ struct = resolve_schema(data, schema)
130
+ arrow_bytes = ArrowConverter.from_rows(data, struct)
131
+ local = Proto::LocalRelation.new(data: arrow_bytes, schema: struct.simple_string.sub(/\Astruct</, "").sub(/>\z/, ""))
132
+ DataFrame.new(self, PlanBuilder.relation(self, local_relation: local))
133
+ end
134
+ alias create_dataframe create_data_frame
135
+ alias createDataFrame create_data_frame
136
+
137
+ # @return [RuntimeConfig] runtime configuration facade.
138
+ attr_reader :conf
139
+
140
+ # @return [Catalog] the catalog facade (databases, tables, functions, cache).
141
+ def catalog
142
+ @catalog ||= Catalog.new(self)
143
+ end
144
+
145
+ # @return [String] the Spark version reported by the server.
146
+ def version
147
+ @client.analyze(spark_version: Proto::AnalyzePlanRequest::SparkVersion.new).spark_version.version
148
+ end
149
+
150
+ # Make this the active/default session.
151
+ # @return [self]
152
+ def set_active
153
+ SparkSession.active = self
154
+ self
155
+ end
156
+
157
+ # Start a brand-new session against the same endpoint (independent
158
+ # server-side session id, configuration, and temporary views).
159
+ # @return [SparkSession]
160
+ def new_session
161
+ SparkSession.new(SparkConnectClient.new(@client.channel_builder))
162
+ end
163
+
164
+ # Interrupt all operations running in this session.
165
+ # @return [Array<String>] the ids of the interrupted operations.
166
+ def interrupt_all
167
+ @client.interrupt(type: :all).interrupted_ids.to_a
168
+ end
169
+
170
+ # Interrupt all operations tagged with `tag` (see {#add_tag}).
171
+ # @return [Array<String>]
172
+ def interrupt_tag(tag)
173
+ @client.interrupt(type: :tag, value: tag.to_s).interrupted_ids.to_a
174
+ end
175
+
176
+ # Interrupt a single operation by id.
177
+ # @return [Array<String>]
178
+ def interrupt_operation(operation_id)
179
+ @client.interrupt(type: :operation_id, value: operation_id.to_s).interrupted_ids.to_a
180
+ end
181
+
182
+ # Add an operation tag applied to all subsequent executions in this session.
183
+ # @return [void]
184
+ def add_tag(tag)
185
+ @client.add_tag(tag)
186
+ end
187
+
188
+ # Remove a previously added operation tag. @return [void]
189
+ def remove_tag(tag)
190
+ @client.remove_tag(tag)
191
+ end
192
+
193
+ # @return [Array<String>] the currently active operation tags.
194
+ def get_tags
195
+ @client.tags.dup
196
+ end
197
+
198
+ # Remove all operation tags. @return [void]
199
+ def clear_tags
200
+ @client.clear_tags
201
+ end
202
+
203
+ # Release the server-side session and stop the client.
204
+ # @return [void]
205
+ def stop
206
+ @client.release_session
207
+ SparkSession.active = nil if SparkSession.active.equal?(self)
208
+ nil
209
+ end
210
+
211
+ # @api private
212
+ def create_data_frame_from_relation(relation)
213
+ DataFrame.new(self, relation)
214
+ end
215
+
216
+ private
217
+
218
+ def resolve_schema(data, schema)
219
+ case schema
220
+ when Types::StructType then schema
221
+ when String then parse_ddl_schema(schema)
222
+ when Array then infer_schema(data, names: schema.map(&:to_s))
223
+ when nil then infer_schema(data)
224
+ else
225
+ raise IllegalArgumentError, "Unsupported schema: #{schema.inspect}"
226
+ end
227
+ end
228
+
229
+ def parse_ddl_schema(ddl)
230
+ # Ask the server to parse the DDL into a concrete schema.
231
+ proto = @client.analyze(ddl_parse: Proto::AnalyzePlanRequest::DDLParse.new(ddl_string: ddl)).ddl_parse.parsed
232
+ Types.from_proto(proto)
233
+ end
234
+
235
+ def infer_schema(data, names: nil)
236
+ raise IllegalArgumentError, "Cannot infer schema from empty data; pass a schema" if data.empty?
237
+
238
+ first = data.first
239
+ case first
240
+ when Hash
241
+ keys = first.keys.map(&:to_s)
242
+ Types::StructType.new(keys.map.with_index do |k, i|
243
+ Types::StructField.new(names ? names[i] : k, column_type(data, k, i), nullable: true)
244
+ end)
245
+ when Row
246
+ Types::StructType.new(first.fields.map.with_index do |k, i|
247
+ Types::StructField.new(names ? names[i] : k, column_type(data, k, i), nullable: true)
248
+ end)
249
+ when Array
250
+ Types::StructType.new(first.each_index.map do |i|
251
+ Types::StructField.new(names ? names[i] : "_#{i + 1}", column_type(data, nil, i), nullable: true)
252
+ end)
253
+ else
254
+ raise IllegalArgumentError, "Cannot infer schema from rows of type #{first.class}"
255
+ end
256
+ end
257
+
258
+ def column_type(data, key, index)
259
+ sample = data.map { |row| ArrowConverter.extract_value(row, key, index) }.find { |v| !v.nil? }
260
+ Column.infer_type(sample)
261
+ end
262
+ end
263
+
264
+ # Fluent builder for {SparkSession}. Returned by {SparkSession.builder}.
265
+ class SparkSession
266
+ class Builder
267
+ def initialize
268
+ @options = {}
269
+ @remote = nil
270
+ end
271
+
272
+ # Set the connection string (`sc://...`).
273
+ # @return [self]
274
+ def remote(url)
275
+ @remote = url
276
+ self
277
+ end
278
+
279
+ # Set the application name.
280
+ # @return [self]
281
+ def app_name(name)
282
+ @options["spark.app.name"] = name
283
+ self
284
+ end
285
+
286
+ # Set an arbitrary configuration option to apply after connecting.
287
+ # @return [self]
288
+ def config(key, value)
289
+ @options[key.to_s] = value
290
+ self
291
+ end
292
+
293
+ # Build (or reuse the active) {SparkSession}.
294
+ # @return [SparkSession]
295
+ def get_or_create
296
+ existing = SparkSession.active
297
+ return existing if existing
298
+
299
+ session = create
300
+ SparkSession.active = session
301
+ session
302
+ end
303
+ alias getOrCreate get_or_create
304
+
305
+ # Always build a new {SparkSession}.
306
+ # @return [SparkSession]
307
+ def create
308
+ url = @remote || ENV["SPARK_REMOTE"] || "sc://localhost:15002"
309
+ client = SparkConnectClient.new(ChannelBuilder.new(url))
310
+ session = SparkSession.new(client)
311
+ @options.each { |k, v| session.conf.set(k, v) unless k == "spark.app.name" }
312
+ session
313
+ end
314
+ alias build create
315
+ end
316
+ end
317
+ end
@@ -0,0 +1,109 @@
1
+ # frozen_string_literal: true
2
+
3
+ module SparkConnect
4
+ # Statistical helpers, returned by {DataFrame#stat}. Mirrors PySpark's
5
+ # `DataFrame.stat` (`DataFrameStatFunctions`).
6
+ #
7
+ # @example
8
+ # df.stat.corr("x", "y")
9
+ # df.stat.approx_quantile("x", [0.25, 0.5, 0.75], 0.01)
10
+ # df.stat.crosstab("a", "b").show
11
+ class DataFrameStatFunctions
12
+ Proto = SparkConnect::Proto
13
+
14
+ # @param df [DataFrame]
15
+ def initialize(df)
16
+ @df = df
17
+ end
18
+
19
+ # Sample covariance of two columns.
20
+ # @return [Float]
21
+ def cov(col1, col2)
22
+ scalar(@df.build(cov: Proto::StatCov.new(input: @df.relation, col1: col1.to_s, col2: col2.to_s)))
23
+ end
24
+
25
+ # Correlation of two columns (`method` is `"pearson"`).
26
+ # @return [Float]
27
+ def corr(col1, col2, method = "pearson")
28
+ rel = Proto::StatCorr.new(input: @df.relation, col1: col1.to_s, col2: col2.to_s, method: method)
29
+ scalar(@df.build(corr: rel))
30
+ end
31
+
32
+ # Contingency table (cross-tabulation) of two columns.
33
+ # @return [DataFrame]
34
+ def crosstab(col1, col2)
35
+ @df.build(crosstab: Proto::StatCrosstab.new(input: @df.relation, col1: col1.to_s, col2: col2.to_s))
36
+ end
37
+
38
+ # Frequent items in the given columns.
39
+ #
40
+ # @param cols [Array<String>]
41
+ # @param support [Float]
42
+ # @return [DataFrame]
43
+ def freq_items(cols, support = 0.01)
44
+ rel = Proto::StatFreqItems.new(input: @df.relation, cols: Array(cols).map(&:to_s), support: support)
45
+ @df.build(freq_items: rel)
46
+ end
47
+
48
+ # Approximate quantiles of numeric columns.
49
+ #
50
+ # @param cols [String, Array<String>]
51
+ # @param probabilities [Array<Float>] values in 0.0..1.0.
52
+ # @param relative_error [Float]
53
+ # @return [Array<Float>, Array<Array<Float>>] one list per column.
54
+ def approx_quantile(cols, probabilities, relative_error)
55
+ single = !cols.is_a?(Array)
56
+ rel = Proto::StatApproxQuantile.new(
57
+ input: @df.relation, cols: Array(cols).map(&:to_s),
58
+ probabilities: probabilities, relative_error: relative_error
59
+ )
60
+ row = @df.build(approx_quantile: rel).collect.first
61
+ result = row.to_a
62
+ single ? result.first : result
63
+ end
64
+
65
+ # Stratified sample without replacement, keyed by `col`.
66
+ #
67
+ # @param col [String, Column]
68
+ # @param fractions [Hash{Object=>Float}] per-stratum sampling fraction.
69
+ # @param seed [Integer, nil]
70
+ # @return [DataFrame]
71
+ def sample_by(col, fractions, seed = nil)
72
+ col_expr = (col.is_a?(Column) ? col : Functions.col(col.to_s)).to_expr
73
+ frac = fractions.map do |stratum, fraction|
74
+ Proto::StatSampleBy::Fraction.new(stratum: Column.to_literal(stratum), fraction: fraction)
75
+ end
76
+ rel = Proto::StatSampleBy.new(input: @df.relation, col: col_expr, fractions: frac)
77
+ rel.seed = seed if seed
78
+ @df.build(sample_by: rel)
79
+ end
80
+
81
+ private
82
+
83
+ def scalar(df)
84
+ row = df.collect.first
85
+ row&.[](0)
86
+ end
87
+ end
88
+
89
+ # Reopen {DataFrame} to add the `describe`/`summary` actions, which are
90
+ # naturally statistical and share the Stat* relations.
91
+ class DataFrame
92
+ # Basic descriptive statistics (count, mean, stddev, min, max) per column.
93
+ #
94
+ # @param cols [Array<String>] columns to describe (all when empty).
95
+ # @return [DataFrame]
96
+ def describe(*cols)
97
+ build(describe: Proto::StatDescribe.new(input: @relation, cols: cols.flatten.map(&:to_s)))
98
+ end
99
+
100
+ # Configurable summary statistics.
101
+ #
102
+ # @param statistics [Array<String>] e.g. `"count"`, `"mean"`, `"stddev"`,
103
+ # `"min"`, `"25%"`, `"50%"`, `"75%"`, `"max"`.
104
+ # @return [DataFrame]
105
+ def summary(*statistics)
106
+ build(summary: Proto::StatSummary.new(input: @relation, statistics: statistics.flatten.map(&:to_s)))
107
+ end
108
+ end
109
+ end