/
InMemoryTable.scala
536 lines (455 loc) · 19.7 KB
/
InMemoryTable.scala
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
/*
* 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.
*/
package org.apache.spark.sql.connector
import java.time.{Instant, ZoneId}
import java.time.temporal.ChronoUnit
import java.util
import scala.collection.JavaConverters._
import scala.collection.mutable
import org.scalatest.Assertions._
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, JoinedRow}
import org.apache.spark.sql.catalyst.util.{CharVarcharUtils, DateTimeUtils}
import org.apache.spark.sql.connector.catalog._
import org.apache.spark.sql.connector.distributions.{Distribution, Distributions}
import org.apache.spark.sql.connector.expressions.{BucketTransform, DaysTransform, HoursTransform, IdentityTransform, MonthsTransform, SortOrder, Transform, YearsTransform}
import org.apache.spark.sql.connector.read._
import org.apache.spark.sql.connector.write._
import org.apache.spark.sql.connector.write.streaming.{StreamingDataWriterFactory, StreamingWrite}
import org.apache.spark.sql.sources.{AlwaysTrue, And, EqualNullSafe, EqualTo, Filter, IsNotNull, IsNull}
import org.apache.spark.sql.types.{DataType, DateType, IntegerType, StringType, StructField, StructType, TimestampType}
import org.apache.spark.sql.util.CaseInsensitiveStringMap
import org.apache.spark.unsafe.types.UTF8String
/**
* A simple in-memory table. Rows are stored as a buffered group produced by each output task.
*/
class InMemoryTable(
val name: String,
val schema: StructType,
override val partitioning: Array[Transform],
override val properties: util.Map[String, String],
val distribution: Distribution = Distributions.unspecified(),
val ordering: Array[SortOrder] = Array.empty,
val numPartitions: Option[Int] = None)
extends Table with SupportsRead with SupportsWrite with SupportsDelete
with SupportsMetadataColumns {
private object PartitionKeyColumn extends MetadataColumn {
override def name: String = "_partition"
override def dataType: DataType = StringType
override def comment: String = "Partition key used to store the row"
}
private object IndexColumn extends MetadataColumn {
override def name: String = "index"
override def dataType: DataType = IntegerType
override def comment: String = "Metadata column used to conflict with a data column"
}
// purposely exposes a metadata column that conflicts with a data column in some tests
override val metadataColumns: Array[MetadataColumn] = Array(IndexColumn, PartitionKeyColumn)
private val metadataColumnNames = metadataColumns.map(_.name).toSet -- schema.map(_.name)
private val allowUnsupportedTransforms =
properties.getOrDefault("allow-unsupported-transforms", "false").toBoolean
partitioning.foreach {
case _: IdentityTransform =>
case _: YearsTransform =>
case _: MonthsTransform =>
case _: DaysTransform =>
case _: HoursTransform =>
case _: BucketTransform =>
case t if !allowUnsupportedTransforms =>
throw new IllegalArgumentException(s"Transform $t is not a supported transform")
}
// The key `Seq[Any]` is the partition values.
val dataMap: mutable.Map[Seq[Any], BufferedRows] = mutable.Map.empty
def data: Array[BufferedRows] = dataMap.values.toArray
def rows: Seq[InternalRow] = dataMap.values.flatMap(_.rows).toSeq
private val partCols: Array[Array[String]] = partitioning.flatMap(_.references).map { ref =>
schema.findNestedField(ref.fieldNames(), includeCollections = false) match {
case Some(_) => ref.fieldNames()
case None => throw new IllegalArgumentException(s"${ref.describe()} does not exist.")
}
}
private val UTC = ZoneId.of("UTC")
private val EPOCH_LOCAL_DATE = Instant.EPOCH.atZone(UTC).toLocalDate
private def getKey(row: InternalRow): Seq[Any] = {
def extractor(
fieldNames: Array[String],
schema: StructType,
row: InternalRow): (Any, DataType) = {
val index = schema.fieldIndex(fieldNames(0))
val value = row.toSeq(schema).apply(index)
if (fieldNames.length > 1) {
(value, schema(index).dataType) match {
case (row: InternalRow, nestedSchema: StructType) =>
extractor(fieldNames.drop(1), nestedSchema, row)
case (_, dataType) =>
throw new IllegalArgumentException(s"Unsupported type, ${dataType.simpleString}")
}
} else {
(value, schema(index).dataType)
}
}
val cleanedSchema = CharVarcharUtils.replaceCharVarcharWithStringInSchema(schema)
partitioning.map {
case IdentityTransform(ref) =>
extractor(ref.fieldNames, cleanedSchema, row)._1
case YearsTransform(ref) =>
extractor(ref.fieldNames, cleanedSchema, row) match {
case (days: Int, DateType) =>
ChronoUnit.YEARS.between(EPOCH_LOCAL_DATE, DateTimeUtils.daysToLocalDate(days))
case (micros: Long, TimestampType) =>
val localDate = DateTimeUtils.microsToInstant(micros).atZone(UTC).toLocalDate
ChronoUnit.YEARS.between(EPOCH_LOCAL_DATE, localDate)
case (v, t) =>
throw new IllegalArgumentException(s"Match: unsupported argument(s) type - ($v, $t)")
}
case MonthsTransform(ref) =>
extractor(ref.fieldNames, cleanedSchema, row) match {
case (days: Int, DateType) =>
ChronoUnit.MONTHS.between(EPOCH_LOCAL_DATE, DateTimeUtils.daysToLocalDate(days))
case (micros: Long, TimestampType) =>
val localDate = DateTimeUtils.microsToInstant(micros).atZone(UTC).toLocalDate
ChronoUnit.MONTHS.between(EPOCH_LOCAL_DATE, localDate)
case (v, t) =>
throw new IllegalArgumentException(s"Match: unsupported argument(s) type - ($v, $t)")
}
case DaysTransform(ref) =>
extractor(ref.fieldNames, cleanedSchema, row) match {
case (days, DateType) =>
days
case (micros: Long, TimestampType) =>
ChronoUnit.DAYS.between(Instant.EPOCH, DateTimeUtils.microsToInstant(micros))
case (v, t) =>
throw new IllegalArgumentException(s"Match: unsupported argument(s) type - ($v, $t)")
}
case HoursTransform(ref) =>
extractor(ref.fieldNames, cleanedSchema, row) match {
case (micros: Long, TimestampType) =>
ChronoUnit.HOURS.between(Instant.EPOCH, DateTimeUtils.microsToInstant(micros))
case (v, t) =>
throw new IllegalArgumentException(s"Match: unsupported argument(s) type - ($v, $t)")
}
case BucketTransform(numBuckets, ref) =>
val (value, dataType) = extractor(ref.fieldNames, cleanedSchema, row)
val valueHashCode = if (value == null) 0 else value.hashCode
((valueHashCode + 31 * dataType.hashCode()) & Integer.MAX_VALUE) % numBuckets
}
}
protected def addPartitionKey(key: Seq[Any]): Unit = {}
protected def renamePartitionKey(
partitionSchema: StructType,
from: Seq[Any],
to: Seq[Any]): Boolean = {
val rows = dataMap.remove(from).getOrElse(new BufferedRows(from.mkString("/")))
val newRows = new BufferedRows(to.mkString("/"))
rows.rows.foreach { r =>
val newRow = new GenericInternalRow(r.numFields)
for (i <- 0 until r.numFields) newRow.update(i, r.get(i, schema(i).dataType))
for (i <- 0 until partitionSchema.length) {
val j = schema.fieldIndex(partitionSchema(i).name)
newRow.update(j, to(i))
}
newRows.withRow(newRow)
}
dataMap.put(to, newRows).foreach { _ =>
throw new IllegalStateException(
s"The ${to.mkString("[", ", ", "]")} partition exists already")
}
true
}
protected def removePartitionKey(key: Seq[Any]): Unit = dataMap.synchronized {
dataMap.remove(key)
}
protected def createPartitionKey(key: Seq[Any]): Unit = dataMap.synchronized {
if (!dataMap.contains(key)) {
val emptyRows = new BufferedRows(key.toArray.mkString("/"))
val rows = if (key.length == schema.length) {
emptyRows.withRow(InternalRow.fromSeq(key))
} else emptyRows
dataMap.put(key, rows)
}
}
protected def clearPartition(key: Seq[Any]): Unit = dataMap.synchronized {
assert(dataMap.contains(key))
dataMap(key).clear()
}
def withData(data: Array[BufferedRows]): InMemoryTable = dataMap.synchronized {
data.foreach(_.rows.foreach { row =>
val key = getKey(row)
dataMap += dataMap.get(key)
.map(key -> _.withRow(row))
.getOrElse(key -> new BufferedRows(key.toArray.mkString("/")).withRow(row))
addPartitionKey(key)
})
this
}
override def capabilities: util.Set[TableCapability] = Set(
TableCapability.BATCH_READ,
TableCapability.BATCH_WRITE,
TableCapability.STREAMING_WRITE,
TableCapability.OVERWRITE_BY_FILTER,
TableCapability.OVERWRITE_DYNAMIC,
TableCapability.TRUNCATE).asJava
override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder = {
new InMemoryScanBuilder(schema)
}
class InMemoryScanBuilder(tableSchema: StructType) extends ScanBuilder
with SupportsPushDownRequiredColumns {
private var schema: StructType = tableSchema
override def build: Scan =
new InMemoryBatchScan(data.map(_.asInstanceOf[InputPartition]), schema)
override def pruneColumns(requiredSchema: StructType): Unit = {
// if metadata columns are projected, return the table schema and metadata columns
val hasMetadataColumns = requiredSchema.map(_.name).exists(metadataColumnNames.contains)
if (hasMetadataColumns) {
schema = StructType(tableSchema ++ metadataColumnNames
.flatMap(name => metadataColumns.find(_.name == name))
.map(col => StructField(col.name, col.dataType, col.isNullable)))
}
}
}
class InMemoryBatchScan(data: Array[InputPartition], schema: StructType) extends Scan with Batch {
override def readSchema(): StructType = schema
override def toBatch: Batch = this
override def planInputPartitions(): Array[InputPartition] = data
override def createReaderFactory(): PartitionReaderFactory = {
val metadataColumns = schema.map(_.name).filter(metadataColumnNames.contains)
new BufferedRowsReaderFactory(metadataColumns)
}
}
override def newWriteBuilder(info: LogicalWriteInfo): WriteBuilder = {
InMemoryTable.maybeSimulateFailedTableWrite(new CaseInsensitiveStringMap(properties))
InMemoryTable.maybeSimulateFailedTableWrite(info.options)
new WriteBuilder with SupportsTruncate with SupportsOverwrite with SupportsDynamicOverwrite {
private var writer: BatchWrite = Append
private var streamingWriter: StreamingWrite = StreamingAppend
override def truncate(): WriteBuilder = {
assert(writer == Append)
writer = TruncateAndAppend
streamingWriter = StreamingTruncateAndAppend
this
}
override def overwrite(filters: Array[Filter]): WriteBuilder = {
assert(writer == Append)
writer = new Overwrite(filters)
streamingWriter = new StreamingNotSupportedOperation(s"overwrite ($filters)")
this
}
override def overwriteDynamicPartitions(): WriteBuilder = {
assert(writer == Append)
writer = DynamicOverwrite
streamingWriter = new StreamingNotSupportedOperation("overwriteDynamicPartitions")
this
}
override def build(): Write = new Write with RequiresDistributionAndOrdering {
override def requiredDistribution: Distribution = distribution
override def requiredOrdering: Array[SortOrder] = ordering
override def requiredNumPartitions(): Int = {
numPartitions.getOrElse(0)
}
override def toBatch: BatchWrite = writer
override def toStreaming: StreamingWrite = streamingWriter match {
case exc: StreamingNotSupportedOperation => exc.throwsException()
case s => s
}
}
}
}
private abstract class TestBatchWrite extends BatchWrite {
override def createBatchWriterFactory(info: PhysicalWriteInfo): DataWriterFactory = {
BufferedRowsWriterFactory
}
override def abort(messages: Array[WriterCommitMessage]): Unit = {}
}
private object Append extends TestBatchWrite {
override def commit(messages: Array[WriterCommitMessage]): Unit = dataMap.synchronized {
withData(messages.map(_.asInstanceOf[BufferedRows]))
}
}
private object DynamicOverwrite extends TestBatchWrite {
override def commit(messages: Array[WriterCommitMessage]): Unit = dataMap.synchronized {
val newData = messages.map(_.asInstanceOf[BufferedRows])
dataMap --= newData.flatMap(_.rows.map(getKey))
withData(newData)
}
}
private class Overwrite(filters: Array[Filter]) extends TestBatchWrite {
import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.MultipartIdentifierHelper
override def commit(messages: Array[WriterCommitMessage]): Unit = dataMap.synchronized {
val deleteKeys = InMemoryTable.filtersToKeys(
dataMap.keys, partCols.map(_.toSeq.quoted), filters)
dataMap --= deleteKeys
withData(messages.map(_.asInstanceOf[BufferedRows]))
}
}
private object TruncateAndAppend extends TestBatchWrite {
override def commit(messages: Array[WriterCommitMessage]): Unit = dataMap.synchronized {
dataMap.clear
withData(messages.map(_.asInstanceOf[BufferedRows]))
}
}
private abstract class TestStreamingWrite extends StreamingWrite {
def createStreamingWriterFactory(info: PhysicalWriteInfo): StreamingDataWriterFactory = {
BufferedRowsWriterFactory
}
def abort(epochId: Long, messages: Array[WriterCommitMessage]): Unit = {}
}
private class StreamingNotSupportedOperation(operation: String) extends TestStreamingWrite {
override def createStreamingWriterFactory(info: PhysicalWriteInfo): StreamingDataWriterFactory =
throwsException()
override def commit(epochId: Long, messages: Array[WriterCommitMessage]): Unit =
throwsException()
override def abort(epochId: Long, messages: Array[WriterCommitMessage]): Unit =
throwsException()
def throwsException[T](): T = throw new IllegalStateException("The operation " +
s"${operation} isn't supported for streaming query.")
}
private object StreamingAppend extends TestStreamingWrite {
override def commit(epochId: Long, messages: Array[WriterCommitMessage]): Unit = {
dataMap.synchronized {
withData(messages.map(_.asInstanceOf[BufferedRows]))
}
}
}
private object StreamingTruncateAndAppend extends TestStreamingWrite {
override def commit(epochId: Long, messages: Array[WriterCommitMessage]): Unit = {
dataMap.synchronized {
dataMap.clear
withData(messages.map(_.asInstanceOf[BufferedRows]))
}
}
}
override def canDeleteWhere(filters: Array[Filter]): Boolean = {
InMemoryTable.supportsFilters(filters)
}
override def deleteWhere(filters: Array[Filter]): Unit = dataMap.synchronized {
import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.MultipartIdentifierHelper
dataMap --= InMemoryTable.filtersToKeys(dataMap.keys, partCols.map(_.toSeq.quoted), filters)
}
}
object InMemoryTable {
val SIMULATE_FAILED_WRITE_OPTION = "spark.sql.test.simulateFailedWrite"
def filtersToKeys(
keys: Iterable[Seq[Any]],
partitionNames: Seq[String],
filters: Array[Filter]): Iterable[Seq[Any]] = {
keys.filter { partValues =>
filters.flatMap(splitAnd).forall {
case EqualTo(attr, value) =>
value == extractValue(attr, partitionNames, partValues)
case EqualNullSafe(attr, value) =>
val attrVal = extractValue(attr, partitionNames, partValues)
if (attrVal == null && value === null) {
true
} else if (attrVal == null || value === null) {
false
} else {
value == attrVal
}
case IsNull(attr) =>
null == extractValue(attr, partitionNames, partValues)
case IsNotNull(attr) =>
null != extractValue(attr, partitionNames, partValues)
case AlwaysTrue() => true
case f =>
throw new IllegalArgumentException(s"Unsupported filter type: $f")
}
}
}
def supportsFilters(filters: Array[Filter]): Boolean = {
filters.flatMap(splitAnd).forall {
case _: EqualTo => true
case _: EqualNullSafe => true
case _: IsNull => true
case _: IsNotNull => true
case _: AlwaysTrue => true
case _ => false
}
}
private def extractValue(
attr: String,
partFieldNames: Seq[String],
partValues: Seq[Any]): Any = {
partFieldNames.zipWithIndex.find(_._1 == attr) match {
case Some((_, partIndex)) =>
partValues(partIndex)
case _ =>
throw new IllegalArgumentException(s"Unknown filter attribute: $attr")
}
}
private def splitAnd(filter: Filter): Seq[Filter] = {
filter match {
case And(left, right) => splitAnd(left) ++ splitAnd(right)
case _ => filter :: Nil
}
}
def maybeSimulateFailedTableWrite(tableOptions: CaseInsensitiveStringMap): Unit = {
if (tableOptions.getBoolean(SIMULATE_FAILED_WRITE_OPTION, false)) {
throw new IllegalStateException("Manual write to table failure.")
}
}
}
class BufferedRows(
val key: String = "") extends WriterCommitMessage with InputPartition with Serializable {
val rows = new mutable.ArrayBuffer[InternalRow]()
def withRow(row: InternalRow): BufferedRows = {
rows.append(row)
this
}
def clear(): Unit = rows.clear()
}
private class BufferedRowsReaderFactory(
metadataColumns: Seq[String]) extends PartitionReaderFactory {
override def createReader(partition: InputPartition): PartitionReader[InternalRow] = {
new BufferedRowsReader(partition.asInstanceOf[BufferedRows], metadataColumns)
}
}
private class BufferedRowsReader(
partition: BufferedRows,
metadataColumns: Seq[String]) extends PartitionReader[InternalRow] {
private def addMetadata(row: InternalRow): InternalRow = {
val metadataRow = new GenericInternalRow(metadataColumns.map {
case "index" => index
case "_partition" => UTF8String.fromString(partition.key)
}.toArray)
new JoinedRow(row, metadataRow)
}
private var index: Int = -1
override def next(): Boolean = {
index += 1
index < partition.rows.length
}
override def get(): InternalRow = addMetadata(partition.rows(index))
override def close(): Unit = {}
}
private object BufferedRowsWriterFactory extends DataWriterFactory with StreamingDataWriterFactory {
override def createWriter(partitionId: Int, taskId: Long): DataWriter[InternalRow] = {
new BufferWriter
}
override def createWriter(
partitionId: Int,
taskId: Long,
epochId: Long): DataWriter[InternalRow] = {
new BufferWriter
}
}
private class BufferWriter extends DataWriter[InternalRow] {
private val buffer = new BufferedRows
override def write(row: InternalRow): Unit = buffer.rows.append(row.copy())
override def commit(): WriterCommitMessage = buffer
override def abort(): Unit = {}
override def close(): Unit = {}
}