aboutsummaryrefslogtreecommitdiff
path: root/sql/core/src/test/scala/org/apache/spark/sql/streaming/MemorySinkSuite.scala
blob: e5bd0b47443eb485b002c1ae3d9bca03b3c4e0c8 (plain) (blame)
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
/*
 * 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.streaming

import scala.language.implicitConversions

import org.scalatest.BeforeAndAfter

import org.apache.spark.sql._
import org.apache.spark.sql.execution.streaming._
import org.apache.spark.sql.test.SharedSQLContext
import org.apache.spark.sql.types.{IntegerType, StructField, StructType}
import org.apache.spark.util.Utils

class MemorySinkSuite extends StreamTest with SharedSQLContext with BeforeAndAfter {

  import testImplicits._

  after {
    sqlContext.streams.active.foreach(_.stop())
  }

  test("directly add data in Append output mode") {
    implicit val schema = new StructType().add(new StructField("value", IntegerType))
    val sink = new MemorySink(schema, InternalOutputModes.Append)

    // Before adding data, check output
    assert(sink.latestBatchId === None)
    checkAnswer(sink.latestBatchData, Seq.empty)
    checkAnswer(sink.allData, Seq.empty)

    // Add batch 0 and check outputs
    sink.addBatch(0, 1 to 3)
    assert(sink.latestBatchId === Some(0))
    checkAnswer(sink.latestBatchData, 1 to 3)
    checkAnswer(sink.allData, 1 to 3)

    // Add batch 1 and check outputs
    sink.addBatch(1, 4 to 6)
    assert(sink.latestBatchId === Some(1))
    checkAnswer(sink.latestBatchData, 4 to 6)
    checkAnswer(sink.allData, 1 to 6)     // new data should get appended to old data

    // Re-add batch 1 with different data, should not be added and outputs should not be changed
    sink.addBatch(1, 7 to 9)
    assert(sink.latestBatchId === Some(1))
    checkAnswer(sink.latestBatchData, 4 to 6)
    checkAnswer(sink.allData, 1 to 6)

    // Add batch 2 and check outputs
    sink.addBatch(2, 7 to 9)
    assert(sink.latestBatchId === Some(2))
    checkAnswer(sink.latestBatchData, 7 to 9)
    checkAnswer(sink.allData, 1 to 9)
  }

  test("directly add data in Update output mode") {
    implicit val schema = new StructType().add(new StructField("value", IntegerType))
    val sink = new MemorySink(schema, InternalOutputModes.Update)

    // Before adding data, check output
    assert(sink.latestBatchId === None)
    checkAnswer(sink.latestBatchData, Seq.empty)
    checkAnswer(sink.allData, Seq.empty)

    // Add batch 0 and check outputs
    sink.addBatch(0, 1 to 3)
    assert(sink.latestBatchId === Some(0))
    checkAnswer(sink.latestBatchData, 1 to 3)
    checkAnswer(sink.allData, 1 to 3)

    // Add batch 1 and check outputs
    sink.addBatch(1, 4 to 6)
    assert(sink.latestBatchId === Some(1))
    checkAnswer(sink.latestBatchData, 4 to 6)
    checkAnswer(sink.allData, 1 to 6) // new data should get appended to old data

    // Re-add batch 1 with different data, should not be added and outputs should not be changed
    sink.addBatch(1, 7 to 9)
    assert(sink.latestBatchId === Some(1))
    checkAnswer(sink.latestBatchData, 4 to 6)
    checkAnswer(sink.allData, 1 to 6)

    // Add batch 2 and check outputs
    sink.addBatch(2, 7 to 9)
    assert(sink.latestBatchId === Some(2))
    checkAnswer(sink.latestBatchData, 7 to 9)
    checkAnswer(sink.allData, 1 to 9)
  }

  test("directly add data in Complete output mode") {
    implicit val schema = new StructType().add(new StructField("value", IntegerType))
    val sink = new MemorySink(schema, InternalOutputModes.Complete)

    // Before adding data, check output
    assert(sink.latestBatchId === None)
    checkAnswer(sink.latestBatchData, Seq.empty)
    checkAnswer(sink.allData, Seq.empty)

    // Add batch 0 and check outputs
    sink.addBatch(0, 1 to 3)
    assert(sink.latestBatchId === Some(0))
    checkAnswer(sink.latestBatchData, 1 to 3)
    checkAnswer(sink.allData, 1 to 3)

    // Add batch 1 and check outputs
    sink.addBatch(1, 4 to 6)
    assert(sink.latestBatchId === Some(1))
    checkAnswer(sink.latestBatchData, 4 to 6)
    checkAnswer(sink.allData, 4 to 6)     // new data should replace old data

    // Re-add batch 1 with different data, should not be added and outputs should not be changed
    sink.addBatch(1, 7 to 9)
    assert(sink.latestBatchId === Some(1))
    checkAnswer(sink.latestBatchData, 4 to 6)
    checkAnswer(sink.allData, 4 to 6)

    // Add batch 2 and check outputs
    sink.addBatch(2, 7 to 9)
    assert(sink.latestBatchId === Some(2))
    checkAnswer(sink.latestBatchData, 7 to 9)
    checkAnswer(sink.allData, 7 to 9)
  }


  test("registering as a table in Append output mode") {
    val input = MemoryStream[Int]
    val query = input.toDF().write
      .format("memory")
      .outputMode("append")
      .queryName("memStream")
      .startStream()
    input.addData(1, 2, 3)
    query.processAllAvailable()

    checkDataset(
      spark.table("memStream").as[Int],
      1, 2, 3)

    input.addData(4, 5, 6)
    query.processAllAvailable()
    checkDataset(
      spark.table("memStream").as[Int],
      1, 2, 3, 4, 5, 6)

    query.stop()
  }

  test("registering as a table in Complete output mode") {
    val input = MemoryStream[Int]
    val query = input.toDF()
      .groupBy("value")
      .count()
      .write
      .format("memory")
      .outputMode("complete")
      .queryName("memStream")
      .startStream()
    input.addData(1, 2, 3)
    query.processAllAvailable()

    checkDataset(
      spark.table("memStream").as[(Int, Long)],
      (1, 1L), (2, 1L), (3, 1L))

    input.addData(4, 5, 6)
    query.processAllAvailable()
    checkDataset(
      spark.table("memStream").as[(Int, Long)],
      (1, 1L), (2, 1L), (3, 1L), (4, 1L), (5, 1L), (6, 1L))

    query.stop()
  }

  ignore("stress test") {
    // Ignore the stress test as it takes several minutes to run
    (0 until 1000).foreach { _ =>
      val input = MemoryStream[Int]
      val query = input.toDF().write
        .format("memory")
        .queryName("memStream")
        .startStream()
      input.addData(1, 2, 3)
      query.processAllAvailable()

      checkDataset(
        spark.table("memStream").as[Int],
        1, 2, 3)

      input.addData(4, 5, 6)
      query.processAllAvailable()
      checkDataset(
        spark.table("memStream").as[Int],
        1, 2, 3, 4, 5, 6)

      query.stop()
    }
  }

  test("error when no name is specified") {
    val error = intercept[AnalysisException] {
      val input = MemoryStream[Int]
      val query = input.toDF().write
          .format("memory")
          .startStream()
    }

    assert(error.message contains "queryName must be specified")
  }

  test("error if attempting to resume specific checkpoint") {
    val location = Utils.createTempDir(namePrefix = "steaming.checkpoint").getCanonicalPath

    val input = MemoryStream[Int]
    val query = input.toDF().write
        .format("memory")
        .queryName("memStream")
        .option("checkpointLocation", location)
        .startStream()
    input.addData(1, 2, 3)
    query.processAllAvailable()
    query.stop()

    intercept[AnalysisException] {
      input.toDF().write
        .format("memory")
        .queryName("memStream")
        .option("checkpointLocation", location)
        .startStream()
    }
  }

  private def checkAnswer(rows: Seq[Row], expected: Seq[Int])(implicit schema: StructType): Unit = {
    checkAnswer(
      sqlContext.createDataFrame(sparkContext.makeRDD(rows), schema),
      intsToDF(expected)(schema))
  }

  private implicit def intsToDF(seq: Seq[Int])(implicit schema: StructType): DataFrame = {
    require(schema.fields.size === 1)
    sqlContext.createDataset(seq).toDF(schema.fieldNames.head)
  }
}