/
TextSocketStreamSuite.scala
433 lines (375 loc) · 14.7 KB
/
TextSocketStreamSuite.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
/*
* 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.execution.streaming.sources
import java.net.{InetSocketAddress, SocketException}
import java.nio.ByteBuffer
import java.nio.channels.ServerSocketChannel
import java.sql.Timestamp
import java.util.concurrent.LinkedBlockingQueue
import java.util.concurrent.TimeUnit._
import scala.collection.JavaConverters._
import org.scalatest.BeforeAndAfterEach
import org.apache.spark.internal.Logging
import org.apache.spark.sql.AnalysisException
import org.apache.spark.sql.execution.datasources.DataSource
import org.apache.spark.sql.execution.datasources.v2.StreamingDataSourceV2Relation
import org.apache.spark.sql.execution.streaming._
import org.apache.spark.sql.execution.streaming.continuous._
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.sources.v2.reader.streaming.{Offset, SparkDataStream}
import org.apache.spark.sql.streaming.{StreamingQueryException, StreamTest}
import org.apache.spark.sql.test.SharedSQLContext
import org.apache.spark.sql.types._
import org.apache.spark.sql.util.CaseInsensitiveStringMap
class TextSocketStreamSuite extends StreamTest with SharedSQLContext with BeforeAndAfterEach {
override def afterEach() {
sqlContext.streams.active.foreach(_.stop())
if (serverThread != null) {
serverThread.interrupt()
serverThread.join()
serverThread = null
}
}
private var serverThread: ServerThread = null
case class AddSocketData(data: String*) extends AddData {
override def addData(query: Option[StreamExecution]): (SparkDataStream, Offset) = {
require(
query.nonEmpty,
"Cannot add data when there is no query for finding the active socket source")
val sources = query.get.logicalPlan.collect {
case r: StreamingDataSourceV2Relation
if r.stream.isInstanceOf[TextSocketMicroBatchStream] =>
r.stream.asInstanceOf[TextSocketMicroBatchStream]
}
if (sources.isEmpty) {
throw new Exception(
"Could not find socket source in the StreamExecution logical plan to add data to")
} else if (sources.size > 1) {
throw new Exception(
"Could not select the socket source in the StreamExecution logical plan as there" +
"are multiple socket sources:\n\t" + sources.mkString("\n\t"))
}
val socketSource = sources.head
assert(serverThread != null && serverThread.port != 0)
val currOffset = socketSource.getCurrentOffset()
data.foreach(serverThread.enqueue)
val newOffset = LongOffset(currOffset.offset + data.size)
(socketSource, newOffset)
}
override def toString: String = s"AddSocketData(data = $data)"
}
test("backward compatibility with old path") {
val ds = DataSource.lookupDataSource(
"org.apache.spark.sql.execution.streaming.TextSocketSourceProvider",
spark.sqlContext.conf).newInstance()
assert(ds.isInstanceOf[TextSocketSourceProvider], "Could not find socket source")
}
test("basic usage") {
serverThread = new ServerThread()
serverThread.start()
withSQLConf(SQLConf.UNSUPPORTED_OPERATION_CHECK_ENABLED.key -> "false") {
val ref = spark
import ref.implicits._
val socket = spark
.readStream
.format("socket")
.options(Map("host" -> "localhost", "port" -> serverThread.port.toString))
.load()
.as[String]
assert(socket.schema === StructType(StructField("value", StringType) :: Nil))
testStream(socket)(
StartStream(),
AddSocketData("hello"),
CheckAnswer("hello"),
AddSocketData("world"),
CheckLastBatch("world"),
CheckAnswer("hello", "world"),
StopStream
)
}
}
test("timestamped usage") {
serverThread = new ServerThread()
serverThread.start()
withSQLConf(SQLConf.UNSUPPORTED_OPERATION_CHECK_ENABLED.key -> "false") {
val socket = spark
.readStream
.format("socket")
.options(Map(
"host" -> "localhost",
"port" -> serverThread.port.toString,
"includeTimestamp" -> "true"))
.load()
assert(socket.schema === StructType(StructField("value", StringType) ::
StructField("timestamp", TimestampType) :: Nil))
var batch1Stamp: Timestamp = null
var batch2Stamp: Timestamp = null
val curr = System.currentTimeMillis()
testStream(socket)(
StartStream(),
AddSocketData("hello"),
CheckAnswerRowsByFunc(
rows => {
assert(rows.size === 1)
assert(rows.head.getAs[String](0) === "hello")
batch1Stamp = rows.head.getAs[Timestamp](1)
Thread.sleep(10)
},
true),
AddSocketData("world"),
CheckAnswerRowsByFunc(
rows => {
assert(rows.size === 1)
assert(rows.head.getAs[String](0) === "world")
batch2Stamp = rows.head.getAs[Timestamp](1)
},
true),
StopStream
)
// Timestamp for rate stream is round to second which leads to milliseconds lost, that will
// make batch1stamp smaller than current timestamp if both of them are in the same second.
// Comparing by second to make sure the correct behavior.
assert(batch1Stamp.getTime >= SECONDS.toMillis(MILLISECONDS.toSeconds(curr)))
assert(!batch2Stamp.before(batch1Stamp))
}
}
test("params not given") {
val provider = new TextSocketSourceProvider
intercept[AnalysisException] {
provider.getTable(CaseInsensitiveStringMap.empty())
}
intercept[AnalysisException] {
provider.getTable(new CaseInsensitiveStringMap(Map("host" -> "localhost").asJava))
}
intercept[AnalysisException] {
provider.getTable(new CaseInsensitiveStringMap(Map("port" -> "1234").asJava))
}
}
test("non-boolean includeTimestamp") {
val provider = new TextSocketSourceProvider
val params = Map("host" -> "localhost", "port" -> "1234", "includeTimestamp" -> "fasle")
intercept[AnalysisException] {
provider.getTable(new CaseInsensitiveStringMap(params.asJava))
}
}
test("user-specified schema given") {
val provider = new TextSocketSourceProvider
val userSpecifiedSchema = StructType(
StructField("name", StringType) ::
StructField("area", StringType) :: Nil)
val params = Map("host" -> "localhost", "port" -> "1234")
val exception = intercept[UnsupportedOperationException] {
provider.getTable(new CaseInsensitiveStringMap(params.asJava), userSpecifiedSchema)
}
assert(exception.getMessage.contains(
"TextSocketSourceProvider source does not support user-specified schema"))
}
test("input row metrics") {
serverThread = new ServerThread()
serverThread.start()
withSQLConf(SQLConf.UNSUPPORTED_OPERATION_CHECK_ENABLED.key -> "false") {
val ref = spark
import ref.implicits._
val socket = spark
.readStream
.format("socket")
.options(Map("host" -> "localhost", "port" -> serverThread.port.toString))
.load()
.as[String]
assert(socket.schema === StructType(StructField("value", StringType) :: Nil))
testStream(socket)(
StartStream(),
AddSocketData("hello"),
CheckAnswer("hello"),
AssertOnQuery { q =>
val numRowMetric =
q.lastExecution.executedPlan.collectLeaves().head.metrics.get("numOutputRows")
numRowMetric.nonEmpty && numRowMetric.get.value == 1
},
StopStream
)
}
}
test("verify ServerThread only accepts the first connection") {
serverThread = new ServerThread()
serverThread.start()
withSQLConf(SQLConf.UNSUPPORTED_OPERATION_CHECK_ENABLED.key -> "false") {
val ref = spark
import ref.implicits._
val socket = spark
.readStream
.format("socket")
.options(Map("host" -> "localhost", "port" -> serverThread.port.toString))
.load()
.as[String]
assert(socket.schema === StructType(StructField("value", StringType) :: Nil))
testStream(socket)(
StartStream(),
AddSocketData("hello"),
CheckAnswer("hello"),
AddSocketData("world"),
CheckLastBatch("world"),
CheckAnswer("hello", "world"),
StopStream
)
// we are trying to connect to the server once again which should fail
try {
val socket2 = spark
.readStream
.format("socket")
.options(Map("host" -> "localhost", "port" -> serverThread.port.toString))
.load()
.as[String]
testStream(socket2)(
StartStream(),
AddSocketData("hello"),
CheckAnswer("hello"),
AddSocketData("world"),
CheckLastBatch("world"),
CheckAnswer("hello", "world"),
StopStream
)
fail("StreamingQueryException is expected!")
} catch {
case e: StreamingQueryException if e.cause.isInstanceOf[SocketException] => // pass
}
}
}
test("continuous data") {
serverThread = new ServerThread()
serverThread.start()
val stream = new TextSocketContinuousStream(
host = "localhost",
port = serverThread.port,
numPartitions = 2,
options = CaseInsensitiveStringMap.empty())
val partitions = stream.planInputPartitions(stream.initialOffset())
assert(partitions.length == 2)
val numRecords = 10
val data = scala.collection.mutable.ListBuffer[Int]()
val offsets = scala.collection.mutable.ListBuffer[Int]()
val readerFactory = stream.createContinuousReaderFactory()
import org.scalatest.time.SpanSugar._
failAfter(5.seconds) {
// inject rows, read and check the data and offsets
for (i <- 0 until numRecords) {
serverThread.enqueue(i.toString)
}
partitions.foreach {
case t: TextSocketContinuousInputPartition =>
val r = readerFactory.createReader(t).asInstanceOf[TextSocketContinuousPartitionReader]
for (i <- 0 until numRecords / 2) {
r.next()
offsets.append(r.getOffset().asInstanceOf[ContinuousRecordPartitionOffset].offset)
data.append(r.get().get(0, DataTypes.StringType).asInstanceOf[String].toInt)
// commit the offsets in the middle and validate if processing continues
if (i == 2) {
commitOffset(t.partitionId, i + 1)
}
}
assert(offsets.toSeq == Range.inclusive(1, 5))
assert(data.toSeq == Range(t.partitionId, 10, 2))
offsets.clear()
data.clear()
case _ => throw new IllegalStateException("Unexpected task type")
}
assert(stream.startOffset.offsets == List(3, 3))
stream.commit(TextSocketOffset(List(5, 5)))
assert(stream.startOffset.offsets == List(5, 5))
}
def commitOffset(partition: Int, offset: Int): Unit = {
val offsetsToCommit = stream.startOffset.offsets.updated(partition, offset)
stream.commit(TextSocketOffset(offsetsToCommit))
assert(stream.startOffset.offsets == offsetsToCommit)
}
}
test("continuous data - invalid commit") {
serverThread = new ServerThread()
serverThread.start()
val stream = new TextSocketContinuousStream(
host = "localhost",
port = serverThread.port,
numPartitions = 2,
options = CaseInsensitiveStringMap.empty())
stream.startOffset = TextSocketOffset(List(5, 5))
assertThrows[IllegalStateException] {
stream.commit(TextSocketOffset(List(6, 6)))
}
}
test("continuous data with timestamp") {
serverThread = new ServerThread()
serverThread.start()
val stream = new TextSocketContinuousStream(
host = "localhost",
port = serverThread.port,
numPartitions = 2,
options = new CaseInsensitiveStringMap(Map("includeTimestamp" -> "true").asJava))
val partitions = stream.planInputPartitions(stream.initialOffset())
assert(partitions.size == 2)
val numRecords = 4
// inject rows, read and check the data and offsets
for (i <- 0 until numRecords) {
serverThread.enqueue(i.toString)
}
val readerFactory = stream.createContinuousReaderFactory()
partitions.foreach {
case t: TextSocketContinuousInputPartition =>
val r = readerFactory.createReader(t).asInstanceOf[TextSocketContinuousPartitionReader]
for (_ <- 0 until numRecords / 2) {
r.next()
assert(r.get().get(0, TextSocketReader.SCHEMA_TIMESTAMP).isInstanceOf[(_, _)])
}
case _ => throw new IllegalStateException("Unexpected task type")
}
}
/**
* This class tries to mimic the behavior of netcat, so that we can ensure
* TextSocketStream supports netcat, which only accepts the first connection
* and exits the process when the first connection is closed.
*
* Please refer SPARK-24466 for more details.
*/
private class ServerThread extends Thread with Logging {
private val serverSocketChannel = ServerSocketChannel.open()
serverSocketChannel.bind(new InetSocketAddress(0))
private val messageQueue = new LinkedBlockingQueue[String]()
val port = serverSocketChannel.socket().getLocalPort
override def run(): Unit = {
try {
val clientSocketChannel = serverSocketChannel.accept()
// Close server socket channel immediately to mimic the behavior that
// only first connection will be made and deny any further connections
// Note that the first client socket channel will be available
serverSocketChannel.close()
clientSocketChannel.configureBlocking(false)
clientSocketChannel.socket().setTcpNoDelay(true)
while (true) {
val line = messageQueue.take() + "\n"
clientSocketChannel.write(ByteBuffer.wrap(line.getBytes("UTF-8")))
}
} catch {
case e: InterruptedException =>
} finally {
// no harm to call close() again...
serverSocketChannel.close()
}
}
def enqueue(line: String): Unit = {
messageQueue.put(line)
}
}
}