aboutsummaryrefslogtreecommitdiff
path: root/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala
blob: 2108b118bf059debe41b20c6d62f8f9df10e611f (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
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
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
1001
1002
1003
1004
1005
1006
1007
1008
1009
1010
1011
1012
1013
1014
1015
1016
1017
1018
1019
1020
1021
1022
1023
1024
1025
1026
1027
1028
1029
1030
1031
1032
1033
1034
1035
1036
1037
1038
1039
1040
1041
1042
1043
1044
1045
1046
1047
1048
1049
1050
1051
1052
1053
1054
1055
1056
1057
1058
1059
1060
1061
1062
1063
1064
1065
1066
1067
1068
1069
1070
1071
1072
1073
1074
1075
1076
1077
1078
1079
1080
1081
1082
1083
1084
1085
1086
1087
1088
1089
1090
1091
1092
1093
1094
1095
1096
1097
1098
1099
1100
1101
1102
1103
1104
1105
1106
1107
1108
1109
1110
1111
1112
1113
1114
1115
1116
1117
1118
1119
1120
1121
1122
1123
1124
1125
1126
1127
1128
1129
1130
1131
1132
1133
1134
1135
1136
1137
1138
1139
1140
1141
1142
1143
1144
1145
1146
1147
1148
1149
1150
1151
1152
1153
1154
1155
1156
1157
1158
1159
1160
1161
1162
1163
1164
1165
1166
1167
1168
1169
1170
1171
1172
1173
1174
1175
1176
1177
1178
1179
1180
1181
1182
1183
1184
1185
1186
1187
1188
1189
1190
1191
1192
1193
1194
1195
1196
1197
1198
1199
1200
1201
1202
1203
1204
1205
1206
1207
1208
1209
1210
1211
1212
1213
1214
1215
1216
1217
1218
1219
1220
1221
1222
1223
1224
1225
1226
1227
1228
1229
1230
1231
1232
1233
1234
1235
1236
1237
1238
1239
1240
1241
1242
1243
1244
1245
1246
1247
1248
1249
1250
1251
1252
1253
1254
1255
1256
1257
1258
1259
1260
1261
1262
1263
1264
1265
1266
1267
1268
1269
1270
1271
1272
1273
1274
1275
1276
1277
1278
1279
1280
1281
1282
1283
1284
1285
1286
1287
1288
1289
1290
1291
1292
1293
1294
1295
1296
1297
1298
1299
1300
1301
1302
1303
1304
1305
1306
1307
1308
1309
1310
1311
1312
1313
1314
1315
1316
1317
1318
1319
1320
1321
1322
1323
1324
1325
1326
1327
1328
1329
1330
1331
1332
1333
1334
1335
1336
1337
1338
1339
1340
1341
1342
1343
1344
1345
1346
1347
1348
1349
1350
1351
1352
1353
1354
1355
1356
1357
1358
1359
1360
1361
1362
1363
1364
1365
1366
1367
1368
/*
 * 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 java.io.File
import java.net.URI

import scala.util.Random

import org.apache.hadoop.fs.{FileStatus, Path, RawLocalFileSystem}
import org.scalatest.PrivateMethodTester
import org.scalatest.concurrent.Eventually._
import org.scalatest.time.SpanSugar._

import org.apache.spark.sql._
import org.apache.spark.sql.catalyst.util._
import org.apache.spark.sql.execution.streaming._
import org.apache.spark.sql.execution.streaming.FileStreamSource.{FileEntry, SeenFilesMap}
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.streaming.ExistsThrowsExceptionFileSystem._
import org.apache.spark.sql.streaming.util.StreamManualClock
import org.apache.spark.sql.test.SharedSQLContext
import org.apache.spark.sql.types._
import org.apache.spark.util.Utils

abstract class FileStreamSourceTest
  extends StreamTest with SharedSQLContext with PrivateMethodTester {

  import testImplicits._

  /**
   * A subclass `AddData` for adding data to files. This is meant to use the
   * `FileStreamSource` actually being used in the execution.
   */
  abstract class AddFileData extends AddData {
    override def addData(query: Option[StreamExecution]): (Source, Offset) = {
      require(
        query.nonEmpty,
        "Cannot add data when there is no query for finding the active file stream source")

      val sources = getSourcesFromStreamingQuery(query.get)
      if (sources.isEmpty) {
        throw new Exception(
          "Could not find file source in the StreamExecution logical plan to add data to")
      } else if (sources.size > 1) {
        throw new Exception(
          "Could not select the file source in the StreamExecution logical plan as there" +
            "are multiple file sources:\n\t" + sources.mkString("\n\t"))
      }
      val source = sources.head
      val newOffset = source.withBatchingLocked {
        addData(source)
        new FileStreamSourceOffset(source.currentLogOffset + 1)
      }
      logInfo(s"Added file to $source at offset $newOffset")
      (source, newOffset)
    }

    protected def addData(source: FileStreamSource): Unit
  }

  case class AddTextFileData(content: String, src: File, tmp: File)
    extends AddFileData {

    override def addData(source: FileStreamSource): Unit = {
      val tempFile = Utils.tempFileWith(new File(tmp, "text"))
      val finalFile = new File(src, tempFile.getName)
      src.mkdirs()
      require(stringToFile(tempFile, content).renameTo(finalFile))
      logInfo(s"Written text '$content' to file $finalFile")
    }
  }

  case class AddParquetFileData(data: DataFrame, src: File, tmp: File) extends AddFileData {
    override def addData(source: FileStreamSource): Unit = {
      AddParquetFileData.writeToFile(data, src, tmp)
    }
  }

  object AddParquetFileData {
    def apply(seq: Seq[String], src: File, tmp: File): AddParquetFileData = {
      AddParquetFileData(seq.toDS().toDF(), src, tmp)
    }

    /** Write parquet files in a temp dir, and move the individual files to the 'src' dir */
    def writeToFile(df: DataFrame, src: File, tmp: File): Unit = {
      val tmpDir = Utils.tempFileWith(new File(tmp, "parquet"))
      df.write.parquet(tmpDir.getCanonicalPath)
      src.mkdirs()
      tmpDir.listFiles().foreach { f =>
        f.renameTo(new File(src, s"${f.getName}"))
      }
    }
  }

  /** Use `format` and `path` to create FileStreamSource via DataFrameReader */
  def createFileStream(
      format: String,
      path: String,
      schema: Option[StructType] = None,
      options: Map[String, String] = Map.empty): DataFrame = {
    val reader =
      if (schema.isDefined) {
        spark.readStream.format(format).schema(schema.get).options(options)
      } else {
        spark.readStream.format(format).options(options)
      }
    reader.load(path)
  }

  protected def getSourceFromFileStream(df: DataFrame): FileStreamSource = {
    val checkpointLocation = Utils.createTempDir(namePrefix = "streaming.metadata").getCanonicalPath
    df.queryExecution.analyzed
      .collect { case StreamingRelation(dataSource, _, _) =>
        // There is only one source in our tests so just set sourceId to 0
        dataSource.createSource(s"$checkpointLocation/sources/0").asInstanceOf[FileStreamSource]
      }.head
  }

  protected def getSourcesFromStreamingQuery(query: StreamExecution): Seq[FileStreamSource] = {
    query.logicalPlan.collect {
      case StreamingExecutionRelation(source, _) if source.isInstanceOf[FileStreamSource] =>
        source.asInstanceOf[FileStreamSource]
    }
  }


  protected def withTempDirs(body: (File, File) => Unit) {
    val src = Utils.createTempDir(namePrefix = "streaming.src")
    val tmp = Utils.createTempDir(namePrefix = "streaming.tmp")
    try {
      body(src, tmp)
    } finally {
      Utils.deleteRecursively(src)
      Utils.deleteRecursively(tmp)
    }
  }

  val valueSchema = new StructType().add("value", StringType)
}

class FileStreamSourceSuite extends FileStreamSourceTest {

  import testImplicits._

  override val streamingTimeout = 20.seconds

  /** Use `format` and `path` to create FileStreamSource via DataFrameReader */
  private def createFileStreamSource(
      format: String,
      path: String,
      schema: Option[StructType] = None): FileStreamSource = {
    getSourceFromFileStream(createFileStream(format, path, schema))
  }

  private def createFileStreamSourceAndGetSchema(
      format: Option[String],
      path: Option[String],
      schema: Option[StructType] = None): StructType = {
    val reader = spark.readStream
    format.foreach(reader.format)
    schema.foreach(reader.schema)
    val df =
      if (path.isDefined) {
        reader.load(path.get)
      } else {
        reader.load()
      }
    df.queryExecution.analyzed
      .collect { case s @ StreamingRelation(dataSource, _, _) => s.schema }.head
  }

  // ============= Basic parameter exists tests ================

  test("FileStreamSource schema: no path") {
    def testError(): Unit = {
      val e = intercept[IllegalArgumentException] {
        createFileStreamSourceAndGetSchema(format = None, path = None, schema = None)
      }
      assert(e.getMessage.contains("path")) // reason is path, not schema
    }
    withSQLConf(SQLConf.STREAMING_SCHEMA_INFERENCE.key -> "false") { testError() }
    withSQLConf(SQLConf.STREAMING_SCHEMA_INFERENCE.key -> "true") { testError() }
  }

  test("FileStreamSource schema: path doesn't exist (without schema) should throw exception") {
    withTempDir { dir =>
      intercept[AnalysisException] {
        val userSchema = new StructType().add(new StructField("value", IntegerType))
        val schema = createFileStreamSourceAndGetSchema(
          format = None, path = Some(new File(dir, "1").getAbsolutePath), schema = None)
      }
    }
  }

  test("FileStreamSource schema: path doesn't exist (with schema) should throw exception") {
    withTempDir { dir =>
      intercept[AnalysisException] {
        val userSchema = new StructType().add(new StructField("value", IntegerType))
        val schema = createFileStreamSourceAndGetSchema(
          format = None, path = Some(new File(dir, "1").getAbsolutePath), schema = Some(userSchema))
      }
    }
  }


  // =============== Text file stream schema tests ================

  test("FileStreamSource schema: text, no existing files, no schema") {
    withTempDir { src =>
      val schema = createFileStreamSourceAndGetSchema(
        format = Some("text"), path = Some(src.getCanonicalPath), schema = None)
      assert(schema === new StructType().add("value", StringType))
    }
  }

  test("FileStreamSource schema: text, existing files, no schema") {
    withTempDir { src =>
      stringToFile(new File(src, "1"), "a\nb\nc")
      val schema = createFileStreamSourceAndGetSchema(
        format = Some("text"), path = Some(src.getCanonicalPath), schema = None)
      assert(schema === new StructType().add("value", StringType))
    }
  }

  test("FileStreamSource schema: text, existing files, schema") {
    withTempDir { src =>
      stringToFile(new File(src, "1"), "a\nb\nc")
      val userSchema = new StructType().add("userColumn", StringType)
      val schema = createFileStreamSourceAndGetSchema(
        format = Some("text"), path = Some(src.getCanonicalPath), schema = Some(userSchema))
      assert(schema === userSchema)
    }
  }

  // =============== Parquet file stream schema tests ================

  test("FileStreamSource schema: parquet, existing files, no schema") {
    withTempDir { src =>
      Seq("a", "b", "c").toDS().as("userColumn").toDF().write
        .mode(org.apache.spark.sql.SaveMode.Overwrite)
        .parquet(src.getCanonicalPath)

      // Without schema inference, should throw error
      withSQLConf(SQLConf.STREAMING_SCHEMA_INFERENCE.key -> "false") {
        intercept[IllegalArgumentException] {
          createFileStreamSourceAndGetSchema(
            format = Some("parquet"), path = Some(src.getCanonicalPath), schema = None)
        }
      }

      // With schema inference, should infer correct schema
      withSQLConf(SQLConf.STREAMING_SCHEMA_INFERENCE.key -> "true") {
        val schema = createFileStreamSourceAndGetSchema(
          format = Some("parquet"), path = Some(src.getCanonicalPath), schema = None)
        assert(schema === new StructType().add("value", StringType))
      }
    }
  }

  test("FileStreamSource schema: parquet, existing files, schema") {
    withTempPath { src =>
      Seq("a", "b", "c").toDS().as("oldUserColumn").toDF()
        .write.parquet(new File(src, "1").getCanonicalPath)
      val userSchema = new StructType().add("userColumn", StringType)
      val schema = createFileStreamSourceAndGetSchema(
        format = Some("parquet"), path = Some(src.getCanonicalPath), schema = Some(userSchema))
      assert(schema === userSchema)
    }
  }

  // =============== JSON file stream schema tests ================

  test("FileStreamSource schema: json, no existing files, no schema") {
    withTempDir { src =>
      withSQLConf(SQLConf.STREAMING_SCHEMA_INFERENCE.key -> "true") {

        val e = intercept[AnalysisException] {
          createFileStreamSourceAndGetSchema(
            format = Some("json"), path = Some(src.getCanonicalPath), schema = None)
        }
        assert("Unable to infer schema for JSON. It must be specified manually.;" === e.getMessage)
      }
    }
  }

  test("FileStreamSource schema: json, existing files, no schema") {
    withTempDir { src =>

      // Without schema inference, should throw error
      withSQLConf(SQLConf.STREAMING_SCHEMA_INFERENCE.key -> "false") {
        intercept[IllegalArgumentException] {
          createFileStreamSourceAndGetSchema(
            format = Some("json"), path = Some(src.getCanonicalPath), schema = None)
        }
      }

      // With schema inference, should infer correct schema
      withSQLConf(SQLConf.STREAMING_SCHEMA_INFERENCE.key -> "true") {
        stringToFile(new File(src, "1"), "{'c': '1'}\n{'c': '2'}\n{'c': '3'}")
        val schema = createFileStreamSourceAndGetSchema(
          format = Some("json"), path = Some(src.getCanonicalPath), schema = None)
        assert(schema === new StructType().add("c", StringType))
      }
    }
  }

  test("FileStreamSource schema: json, existing files, schema") {
    withTempDir { src =>
      stringToFile(new File(src, "1"), "{'c': '1'}\n{'c': '2'}\n{'c', '3'}")
      val userSchema = new StructType().add("userColumn", StringType)
      val schema = createFileStreamSourceAndGetSchema(
        format = Some("json"), path = Some(src.getCanonicalPath), schema = Some(userSchema))
      assert(schema === userSchema)
    }
  }

  // =============== Text file stream tests ================

  test("read from text files") {
    withTempDirs { case (src, tmp) =>
      val textStream = createFileStream("text", src.getCanonicalPath)
      val filtered = textStream.filter($"value" contains "keep")

      testStream(filtered)(
        AddTextFileData("drop1\nkeep2\nkeep3", src, tmp),
        CheckAnswer("keep2", "keep3"),
        StopStream,
        AddTextFileData("drop4\nkeep5\nkeep6", src, tmp),
        StartStream(),
        CheckAnswer("keep2", "keep3", "keep5", "keep6"),
        AddTextFileData("drop7\nkeep8\nkeep9", src, tmp),
        CheckAnswer("keep2", "keep3", "keep5", "keep6", "keep8", "keep9")
      )
    }
  }

  test("read from textfile") {
    withTempDirs { case (src, tmp) =>
      val textStream = spark.readStream.textFile(src.getCanonicalPath)
      val filtered = textStream.filter(_.contains("keep"))

      testStream(filtered)(
        AddTextFileData("drop1\nkeep2\nkeep3", src, tmp),
        CheckAnswer("keep2", "keep3"),
        StopStream,
        AddTextFileData("drop4\nkeep5\nkeep6", src, tmp),
        StartStream(),
        CheckAnswer("keep2", "keep3", "keep5", "keep6"),
        AddTextFileData("drop7\nkeep8\nkeep9", src, tmp),
        CheckAnswer("keep2", "keep3", "keep5", "keep6", "keep8", "keep9")
      )
    }
  }

  test("SPARK-17165 should not track the list of seen files indefinitely") {
    // This test works by:
    // 1. Create a file
    // 2. Get it processed
    // 3. Sleeps for a very short amount of time (larger than maxFileAge
    // 4. Add another file (at this point the original file should have been purged
    // 5. Test the size of the seenFiles internal data structure

    // Note that if we change maxFileAge to a very large number, the last step should fail.
    withTempDirs { case (src, tmp) =>
      val textStream: DataFrame =
        createFileStream("text", src.getCanonicalPath, options = Map("maxFileAge" -> "5ms"))

      testStream(textStream)(
        AddTextFileData("a\nb", src, tmp),
        CheckAnswer("a", "b"),

        // SLeeps longer than 5ms (maxFileAge)
        // Unfortunately since a lot of file system does not have modification time granularity
        // finer grained than 1 sec, we need to use 1 sec here.
        AssertOnQuery { _ => Thread.sleep(1000); true },

        AddTextFileData("c\nd", src, tmp),
        CheckAnswer("a", "b", "c", "d"),

        AssertOnQuery("seen files should contain only one entry") { streamExecution =>
          val source = getSourcesFromStreamingQuery(streamExecution).head
          assert(source.seenFiles.size == 1)
          true
        }
      )
    }
  }

  // =============== JSON file stream tests ================

  test("read from json files") {
    withTempDirs { case (src, tmp) =>
      val fileStream = createFileStream("json", src.getCanonicalPath, Some(valueSchema))
      val filtered = fileStream.filter($"value" contains "keep")

      testStream(filtered)(
        AddTextFileData(
          "{'value': 'drop1'}\n{'value': 'keep2'}\n{'value': 'keep3'}",
          src,
          tmp),
        CheckAnswer("keep2", "keep3"),
        StopStream,
        AddTextFileData(
          "{'value': 'drop4'}\n{'value': 'keep5'}\n{'value': 'keep6'}",
          src,
          tmp),
        StartStream(),
        CheckAnswer("keep2", "keep3", "keep5", "keep6"),
        AddTextFileData(
          "{'value': 'drop7'}\n{'value': 'keep8'}\n{'value': 'keep9'}",
          src,
          tmp),
        CheckAnswer("keep2", "keep3", "keep5", "keep6", "keep8", "keep9")
      )
    }
  }

  test("read from json files with inferring schema") {
    withTempDirs { case (src, tmp) =>
      withSQLConf(SQLConf.STREAMING_SCHEMA_INFERENCE.key -> "true") {

        // Add a file so that we can infer its schema
        stringToFile(new File(src, "existing"), "{'c': 'drop1'}\n{'c': 'keep2'}\n{'c': 'keep3'}")

        val fileStream = createFileStream("json", src.getCanonicalPath)
        assert(fileStream.schema === StructType(Seq(StructField("c", StringType))))

        // FileStreamSource should infer the column "c"
        val filtered = fileStream.filter($"c" contains "keep")

        testStream(filtered)(
          AddTextFileData("{'c': 'drop4'}\n{'c': 'keep5'}\n{'c': 'keep6'}", src, tmp),
          CheckAnswer("keep2", "keep3", "keep5", "keep6")
        )
      }
    }
  }

  test("reading from json files inside partitioned directory") {
    withTempDirs { case (baseSrc, tmp) =>
      withSQLConf(SQLConf.STREAMING_SCHEMA_INFERENCE.key -> "true") {
        val src = new File(baseSrc, "type=X")
        src.mkdirs()

        // Add a file so that we can infer its schema
        stringToFile(new File(src, "existing"), "{'c': 'drop1'}\n{'c': 'keep2'}\n{'c': 'keep3'}")

        val fileStream = createFileStream("json", src.getCanonicalPath)

        // FileStreamSource should infer the column "c"
        val filtered = fileStream.filter($"c" contains "keep")

        testStream(filtered)(
          AddTextFileData("{'c': 'drop4'}\n{'c': 'keep5'}\n{'c': 'keep6'}", src, tmp),
          CheckAnswer("keep2", "keep3", "keep5", "keep6")
        )
      }
    }
  }

  test("reading from json files with changing schema") {
    withTempDirs { case (src, tmp) =>
      withSQLConf(SQLConf.STREAMING_SCHEMA_INFERENCE.key -> "true") {

        // Add a file so that we can infer its schema
        stringToFile(new File(src, "existing"), "{'k': 'value0'}")

        val fileStream = createFileStream("json", src.getCanonicalPath)

        // FileStreamSource should infer the column "k"
        assert(fileStream.schema === StructType(Seq(StructField("k", StringType))))

        // After creating DF and before starting stream, add data with different schema
        // Should not affect the inferred schema any more
        stringToFile(new File(src, "existing2"), "{'k': 'value1', 'v': 'new'}")

        testStream(fileStream)(

          // Should not pick up column v in the file added before start
          AddTextFileData("{'k': 'value2'}", src, tmp),
          CheckAnswer("value0", "value1", "value2"),

          // Should read data in column k, and ignore v
          AddTextFileData("{'k': 'value3', 'v': 'new'}", src, tmp),
          CheckAnswer("value0", "value1", "value2", "value3"),

          // Should ignore rows that do not have the necessary k column
          AddTextFileData("{'v': 'value4'}", src, tmp),
          CheckAnswer("value0", "value1", "value2", "value3", null))
      }
    }
  }

  // =============== Parquet file stream tests ================

  test("read from parquet files") {
    withTempDirs { case (src, tmp) =>
      val fileStream = createFileStream("parquet", src.getCanonicalPath, Some(valueSchema))
      val filtered = fileStream.filter($"value" contains "keep")

      testStream(filtered)(
        AddParquetFileData(Seq("drop1", "keep2", "keep3"), src, tmp),
        CheckAnswer("keep2", "keep3"),
        StopStream,
        AddParquetFileData(Seq("drop4", "keep5", "keep6"), src, tmp),
        StartStream(),
        CheckAnswer("keep2", "keep3", "keep5", "keep6"),
        AddParquetFileData(Seq("drop7", "keep8", "keep9"), src, tmp),
        CheckAnswer("keep2", "keep3", "keep5", "keep6", "keep8", "keep9")
      )
    }
  }

  test("read from parquet files with changing schema") {

    withTempDirs { case (src, tmp) =>
      withSQLConf(SQLConf.STREAMING_SCHEMA_INFERENCE.key -> "true") {

        // Add a file so that we can infer its schema
        AddParquetFileData.writeToFile(Seq("value0").toDF("k"), src, tmp)

        val fileStream = createFileStream("parquet", src.getCanonicalPath)

        // FileStreamSource should infer the column "k"
        assert(fileStream.schema === StructType(Seq(StructField("k", StringType))))

        // After creating DF and before starting stream, add data with different schema
        // Should not affect the inferred schema any more
        AddParquetFileData.writeToFile(Seq(("value1", 0)).toDF("k", "v"), src, tmp)

        testStream(fileStream)(
          // Should not pick up column v in the file added before start
          AddParquetFileData(Seq("value2").toDF("k"), src, tmp),
          CheckAnswer("value0", "value1", "value2"),

          // Should read data in column k, and ignore v
          AddParquetFileData(Seq(("value3", 1)).toDF("k", "v"), src, tmp),
          CheckAnswer("value0", "value1", "value2", "value3"),

          // Should ignore rows that do not have the necessary k column
          AddParquetFileData(Seq("value5").toDF("v"), src, tmp),
          CheckAnswer("value0", "value1", "value2", "value3", null)
        )
      }
    }
  }

  // =============== file stream globbing tests ================

  test("read new files in nested directories with globbing") {
    withTempDirs { case (dir, tmp) =>

      // src/*/* should consider all the files and directories that matches that glob.
      // So any files that matches the glob as well as any files in directories that matches
      // this glob should be read.
      val fileStream = createFileStream("text", s"${dir.getCanonicalPath}/*/*")
      val filtered = fileStream.filter($"value" contains "keep")
      val subDir = new File(dir, "subdir")
      val subSubDir = new File(subDir, "subsubdir")
      val subSubSubDir = new File(subSubDir, "subsubsubdir")

      require(!subDir.exists())
      require(!subSubDir.exists())

      testStream(filtered)(
        // Create new dir/subdir and write to it, should read
        AddTextFileData("drop1\nkeep2", subDir, tmp),
        CheckAnswer("keep2"),

        // Add files to dir/subdir, should read
        AddTextFileData("keep3", subDir, tmp),
        CheckAnswer("keep2", "keep3"),

        // Create new dir/subdir/subsubdir and write to it, should read
        AddTextFileData("keep4", subSubDir, tmp),
        CheckAnswer("keep2", "keep3", "keep4"),

        // Add files to dir/subdir/subsubdir, should read
        AddTextFileData("keep5", subSubDir, tmp),
        CheckAnswer("keep2", "keep3", "keep4", "keep5"),

        // 1. Add file to src dir, should not read as globbing src/*/* does not capture files in
        //    dir, only captures files in dir/subdir/
        // 2. Add files to dir/subDir/subsubdir/subsubsubdir, should not read as src/*/* should
        //    not capture those files
        AddTextFileData("keep6", dir, tmp),
        AddTextFileData("keep7", subSubSubDir, tmp),
        AddTextFileData("keep8", subDir, tmp), // needed to make query detect new data
        CheckAnswer("keep2", "keep3", "keep4", "keep5", "keep8")
      )
    }
  }

  test("read new files in partitioned table with globbing, should not read partition data") {
    withTempDirs { case (dir, tmp) =>
      val partitionFooSubDir = new File(dir, "partition=foo")
      val partitionBarSubDir = new File(dir, "partition=bar")

      val schema = new StructType().add("value", StringType).add("partition", StringType)
      val fileStream = createFileStream("json", s"${dir.getCanonicalPath}/*/*", Some(schema))
      val filtered = fileStream.filter($"value" contains "keep")
      val nullStr = null.asInstanceOf[String]
      testStream(filtered)(
        // Create new partition=foo sub dir and write to it, should read only value, not partition
        AddTextFileData("{'value': 'drop1'}\n{'value': 'keep2'}", partitionFooSubDir, tmp),
        CheckAnswer(("keep2", nullStr)),

        // Append to same partition=1 sub dir, should read only value, not partition
        AddTextFileData("{'value': 'keep3'}", partitionFooSubDir, tmp),
        CheckAnswer(("keep2", nullStr), ("keep3", nullStr)),

        // Create new partition sub dir and write to it, should read only value, not partition
        AddTextFileData("{'value': 'keep4'}", partitionBarSubDir, tmp),
        CheckAnswer(("keep2", nullStr), ("keep3", nullStr), ("keep4", nullStr)),

        // Append to same partition=2 sub dir, should read only value, not partition
        AddTextFileData("{'value': 'keep5'}", partitionBarSubDir, tmp),
        CheckAnswer(("keep2", nullStr), ("keep3", nullStr), ("keep4", nullStr), ("keep5", nullStr))
      )
    }
  }

  // =============== other tests ================

  test("read new files in partitioned table without globbing, should read partition data") {
    withTempDirs { case (dir, tmp) =>
      val partitionFooSubDir = new File(dir, "partition=foo")
      val partitionBarSubDir = new File(dir, "partition=bar")

      val schema = new StructType().add("value", StringType).add("partition", StringType)
      val fileStream = createFileStream("json", s"${dir.getCanonicalPath}", Some(schema))
      val filtered = fileStream.filter($"value" contains "keep")
      testStream(filtered)(
        // Create new partition=foo sub dir and write to it
        AddTextFileData("{'value': 'drop1'}\n{'value': 'keep2'}", partitionFooSubDir, tmp),
        CheckAnswer(("keep2", "foo")),

        // Append to same partition=foo sub dir
        AddTextFileData("{'value': 'keep3'}", partitionFooSubDir, tmp),
        CheckAnswer(("keep2", "foo"), ("keep3", "foo")),

        // Create new partition sub dir and write to it
        AddTextFileData("{'value': 'keep4'}", partitionBarSubDir, tmp),
        CheckAnswer(("keep2", "foo"), ("keep3", "foo"), ("keep4", "bar")),

        // Append to same partition=bar sub dir
        AddTextFileData("{'value': 'keep5'}", partitionBarSubDir, tmp),
        CheckAnswer(("keep2", "foo"), ("keep3", "foo"), ("keep4", "bar"), ("keep5", "bar"))
      )
    }
  }

  test("read data from outputs of another streaming query") {
    withSQLConf(SQLConf.FILE_SINK_LOG_COMPACT_INTERVAL.key -> "3") {
      withTempDirs { case (outputDir, checkpointDir) =>
        // q1 is a streaming query that reads from memory and writes to text files
        val q1Source = MemoryStream[String]
        val q1 =
          q1Source
            .toDF()
            .writeStream
            .option("checkpointLocation", checkpointDir.getCanonicalPath)
            .format("text")
            .start(outputDir.getCanonicalPath)

        // q2 is a streaming query that reads q1's text outputs
        val q2 =
          createFileStream("text", outputDir.getCanonicalPath).filter($"value" contains "keep")

        def q1AddData(data: String*): StreamAction =
          Execute { _ =>
            q1Source.addData(data)
            q1.processAllAvailable()
          }
        def q2ProcessAllAvailable(): StreamAction = Execute { q2 => q2.processAllAvailable() }

        testStream(q2)(
          // batch 0
          q1AddData("drop1", "keep2"),
          q2ProcessAllAvailable(),
          CheckAnswer("keep2"),

          // batch 1
          Assert {
            // create a text file that won't be on q1's sink log
            // thus even if its content contains "keep", it should NOT appear in q2's answer
            val shouldNotKeep = new File(outputDir, "should_not_keep.txt")
            stringToFile(shouldNotKeep, "should_not_keep!!!")
            shouldNotKeep.exists()
          },
          q1AddData("keep3"),
          q2ProcessAllAvailable(),
          CheckAnswer("keep2", "keep3"),

          // batch 2: check that things work well when the sink log gets compacted
          q1AddData("keep4"),
          Assert {
            // compact interval is 3, so file "2.compact" should exist
            new File(outputDir, s"${FileStreamSink.metadataDir}/2.compact").exists()
          },
          q2ProcessAllAvailable(),
          CheckAnswer("keep2", "keep3", "keep4"),

          Execute { _ => q1.stop() }
        )
      }
    }
  }

  test("start before another streaming query, and read its output") {
    withTempDirs { case (outputDir, checkpointDir) =>
      // q1 is a streaming query that reads from memory and writes to text files
      val q1Source = MemoryStream[String]
      // define q1, but don't start it for now
      val q1Write =
        q1Source
          .toDF()
          .writeStream
          .option("checkpointLocation", checkpointDir.getCanonicalPath)
          .format("text")
      var q1: StreamingQuery = null

      val q2 = createFileStream("text", outputDir.getCanonicalPath).filter($"value" contains "keep")

      testStream(q2)(
        AssertOnQuery { q2 =>
          val fileSource = getSourcesFromStreamingQuery(q2).head
          // q1 has not started yet, verify that q2 doesn't know whether q1 has metadata
          fileSource.sourceHasMetadata === None
        },
        Execute { _ =>
          q1 = q1Write.start(outputDir.getCanonicalPath)
          q1Source.addData("drop1", "keep2")
          q1.processAllAvailable()
        },
        AssertOnQuery { q2 =>
          q2.processAllAvailable()
          val fileSource = getSourcesFromStreamingQuery(q2).head
          // q1 has started, verify that q2 knows q1 has metadata by now
          fileSource.sourceHasMetadata === Some(true)
        },
        CheckAnswer("keep2"),
        Execute { _ => q1.stop() }
      )
    }
  }

  test("when schema inference is turned on, should read partition data") {
    def createFile(content: String, src: File, tmp: File): Unit = {
      val tempFile = Utils.tempFileWith(new File(tmp, "text"))
      val finalFile = new File(src, tempFile.getName)
      require(!src.exists(), s"$src exists, dir: ${src.isDirectory}, file: ${src.isFile}")
      require(src.mkdirs(), s"Cannot create $src")
      require(src.isDirectory(), s"$src is not a directory")
      require(stringToFile(tempFile, content).renameTo(finalFile))
    }

    withSQLConf(SQLConf.STREAMING_SCHEMA_INFERENCE.key -> "true") {
      withTempDirs { case (dir, tmp) =>
        val partitionFooSubDir = new File(dir, "partition=foo")
        val partitionBarSubDir = new File(dir, "partition=bar")

        // Create file in partition, so we can infer the schema.
        createFile("{'value': 'drop0'}", partitionFooSubDir, tmp)

        val fileStream = createFileStream("json", s"${dir.getCanonicalPath}")
        val filtered = fileStream.filter($"value" contains "keep")
        testStream(filtered)(
          // Append to same partition=foo sub dir
          AddTextFileData("{'value': 'drop1'}\n{'value': 'keep2'}", partitionFooSubDir, tmp),
          CheckAnswer(("keep2", "foo")),

          // Append to same partition=foo sub dir
          AddTextFileData("{'value': 'keep3'}", partitionFooSubDir, tmp),
          CheckAnswer(("keep2", "foo"), ("keep3", "foo")),

          // Create new partition sub dir and write to it
          AddTextFileData("{'value': 'keep4'}", partitionBarSubDir, tmp),
          CheckAnswer(("keep2", "foo"), ("keep3", "foo"), ("keep4", "bar")),

          // Append to same partition=bar sub dir
          AddTextFileData("{'value': 'keep5'}", partitionBarSubDir, tmp),
          CheckAnswer(("keep2", "foo"), ("keep3", "foo"), ("keep4", "bar"), ("keep5", "bar")),

          AddTextFileData("{'value': 'keep6'}", partitionBarSubDir, tmp),
          CheckAnswer(("keep2", "foo"), ("keep3", "foo"), ("keep4", "bar"), ("keep5", "bar"),
            ("keep6", "bar"))
        )
      }
    }
  }

  test("fault tolerance") {
    withTempDirs { case (src, tmp) =>
      val fileStream = createFileStream("text", src.getCanonicalPath)
      val filtered = fileStream.filter($"value" contains "keep")

      testStream(filtered)(
        AddTextFileData("drop1\nkeep2\nkeep3", src, tmp),
        CheckAnswer("keep2", "keep3"),
        StopStream,
        AddTextFileData("drop4\nkeep5\nkeep6", src, tmp),
        StartStream(),
        CheckAnswer("keep2", "keep3", "keep5", "keep6"),
        AddTextFileData("drop7\nkeep8\nkeep9", src, tmp),
        CheckAnswer("keep2", "keep3", "keep5", "keep6", "keep8", "keep9")
      )
    }
  }

  test("max files per trigger") {
    withTempDir { case src =>
      var lastFileModTime: Option[Long] = None

      /** Create a text file with a single data item */
      def createFile(data: Int): File = {
        val file = stringToFile(new File(src, s"$data.txt"), data.toString)
        if (lastFileModTime.nonEmpty) file.setLastModified(lastFileModTime.get + 1000)
        lastFileModTime = Some(file.lastModified)
        file
      }

      createFile(1)
      createFile(2)
      createFile(3)

      // Set up a query to read text files 2 at a time
      val df = spark
        .readStream
        .option("maxFilesPerTrigger", 2)
        .text(src.getCanonicalPath)
      val q = df
        .writeStream
        .format("memory")
        .queryName("file_data")
        .start()
        .asInstanceOf[StreamingQueryWrapper]
        .streamingQuery
      q.processAllAvailable()
      val memorySink = q.sink.asInstanceOf[MemorySink]
      val fileSource = getSourcesFromStreamingQuery(q).head

      /** Check the data read in the last batch */
      def checkLastBatchData(data: Int*): Unit = {
        val schema = StructType(Seq(StructField("value", StringType)))
        val df = spark.createDataFrame(
          spark.sparkContext.makeRDD(memorySink.latestBatchData), schema)
        checkAnswer(df, data.map(_.toString).toDF("value"))
      }

      def checkAllData(data: Seq[Int]): Unit = {
        val schema = StructType(Seq(StructField("value", StringType)))
        val df = spark.createDataFrame(
          spark.sparkContext.makeRDD(memorySink.allData), schema)
        checkAnswer(df, data.map(_.toString).toDF("value"))
      }

      /** Check how many batches have executed since the last time this check was made */
      var lastBatchId = -1L
      def checkNumBatchesSinceLastCheck(numBatches: Int): Unit = {
        require(lastBatchId >= 0)
        assert(memorySink.latestBatchId.get === lastBatchId + numBatches)
        lastBatchId = memorySink.latestBatchId.get
      }

      checkLastBatchData(3)  // (1 and 2) should be in batch 1, (3) should be in batch 2 (last)
      checkAllData(1 to 3)
      lastBatchId = memorySink.latestBatchId.get

      fileSource.withBatchingLocked {
        createFile(4)
        createFile(5)   // 4 and 5 should be in a batch
        createFile(6)
        createFile(7)   // 6 and 7 should be in the last batch
      }
      q.processAllAvailable()
      checkNumBatchesSinceLastCheck(2)
      checkLastBatchData(6, 7)
      checkAllData(1 to 7)

      fileSource.withBatchingLocked {
        createFile(8)
        createFile(9)    // 8 and 9 should be in a batch
        createFile(10)
        createFile(11)   // 10 and 11 should be in a batch
        createFile(12)   // 12 should be in the last batch
      }
      q.processAllAvailable()
      checkNumBatchesSinceLastCheck(3)
      checkLastBatchData(12)
      checkAllData(1 to 12)

      q.stop()
    }
  }

  testQuietly("max files per trigger - incorrect values") {
    val testTable = "maxFilesPerTrigger_test"
    withTable(testTable) {
      withTempDir { case src =>
        def testMaxFilePerTriggerValue(value: String): Unit = {
          val df = spark.readStream.option("maxFilesPerTrigger", value).text(src.getCanonicalPath)
          val e = intercept[StreamingQueryException] {
            // Note: `maxFilesPerTrigger` is checked in the stream thread when creating the source
            val q = df.writeStream.format("memory").queryName(testTable).start()
            try {
              q.processAllAvailable()
            } finally {
              q.stop()
            }
          }
          assert(e.getCause.isInstanceOf[IllegalArgumentException])
          Seq("maxFilesPerTrigger", value, "positive integer").foreach { s =>
            assert(e.getMessage.contains(s))
          }
        }

        testMaxFilePerTriggerValue("not-a-integer")
        testMaxFilePerTriggerValue("-1")
        testMaxFilePerTriggerValue("0")
        testMaxFilePerTriggerValue("10.1")
      }
    }
  }

  test("explain") {
    withTempDirs { case (src, tmp) =>
      src.mkdirs()

      val df = spark.readStream.format("text").load(src.getCanonicalPath).map(_ + "-x")
      // Test `explain` not throwing errors
      df.explain()

      val q = df.writeStream.queryName("file_explain").format("memory").start()
        .asInstanceOf[StreamingQueryWrapper]
        .streamingQuery
      try {
        assert("No physical plan. Waiting for data." === q.explainInternal(false))
        assert("No physical plan. Waiting for data." === q.explainInternal(true))

        val tempFile = Utils.tempFileWith(new File(tmp, "text"))
        val finalFile = new File(src, tempFile.getName)
        require(stringToFile(tempFile, "foo").renameTo(finalFile))

        q.processAllAvailable()

        val explainWithoutExtended = q.explainInternal(false)
        // `extended = false` only displays the physical plan.
        assert("Relation.*text".r.findAllMatchIn(explainWithoutExtended).size === 0)
        assert(": Text".r.findAllMatchIn(explainWithoutExtended).size === 1)

        val explainWithExtended = q.explainInternal(true)
        // `extended = true` displays 3 logical plans (Parsed/Optimized/Optimized) and 1 physical
        // plan.
        assert("Relation.*text".r.findAllMatchIn(explainWithExtended).size === 3)
        assert(": Text".r.findAllMatchIn(explainWithExtended).size === 1)
      } finally {
        q.stop()
      }
    }
  }

  test("SPARK-17372 - write file names to WAL as Array[String]") {
    // Note: If this test takes longer than the timeout, then its likely that this is actually
    // running a Spark job with 10000 tasks. This test tries to avoid that by
    // 1. Setting the threshold for parallel file listing to very high
    // 2. Using a query that should use constant folding to eliminate reading of the files

    val numFiles = 10000

    // This is to avoid running a spark job to list of files in parallel
    // by the InMemoryFileIndex.
    spark.sessionState.conf.setConf(SQLConf.PARALLEL_PARTITION_DISCOVERY_THRESHOLD, numFiles * 2)

    withTempDirs { case (root, tmp) =>
      val src = new File(root, "a=1")
      src.mkdirs()

      (1 to numFiles).map { _.toString }.foreach { i =>
        val tempFile = Utils.tempFileWith(new File(tmp, "text"))
        val finalFile = new File(src, tempFile.getName)
        stringToFile(finalFile, i)
      }
      assert(src.listFiles().size === numFiles)

      val files = spark.readStream.text(root.getCanonicalPath).as[(String, Int)]

      // Note this query will use constant folding to eliminate the file scan.
      // This is to avoid actually running a Spark job with 10000 tasks
      val df = files.filter("1 == 0").groupBy().count()

      testStream(df, OutputMode.Complete)(
        AddTextFileData("0", src, tmp),
        CheckAnswer(0)
      )
    }
  }

  test("compact interval metadata log") {
    val _sources = PrivateMethod[Seq[Source]]('sources)
    val _metadataLog = PrivateMethod[FileStreamSourceLog]('metadataLog)

    def verify(
        execution: StreamExecution,
        batchId: Long,
        expectedBatches: Int,
        expectedCompactInterval: Int): Boolean = {
      import CompactibleFileStreamLog._

      val fileSource = (execution invokePrivate _sources()).head.asInstanceOf[FileStreamSource]
      val metadataLog = fileSource invokePrivate _metadataLog()

      if (isCompactionBatch(batchId, expectedCompactInterval)) {
        val path = metadataLog.batchIdToPath(batchId)

        // Assert path name should be ended with compact suffix.
        assert(path.getName.endsWith(COMPACT_FILE_SUFFIX),
          "path does not end with compact file suffix")

        // Compacted batch should include all entries from start.
        val entries = metadataLog.get(batchId)
        assert(entries.isDefined, "Entries not defined")
        assert(entries.get.length === metadataLog.allFiles().length, "clean up check")
        assert(metadataLog.get(None, Some(batchId)).flatMap(_._2).length ===
          entries.get.length, "Length check")
      }

      assert(metadataLog.allFiles().sortBy(_.batchId) ===
        metadataLog.get(None, Some(batchId)).flatMap(_._2).sortBy(_.batchId),
        "Batch id mismatch")

      metadataLog.get(None, Some(batchId)).flatMap(_._2).length === expectedBatches
    }

    withTempDirs { case (src, tmp) =>
      withSQLConf(
        SQLConf.FILE_SOURCE_LOG_COMPACT_INTERVAL.key -> "2"
      ) {
        val fileStream = createFileStream("text", src.getCanonicalPath)
        val filtered = fileStream.filter($"value" contains "keep")
        val updateConf = Map(SQLConf.FILE_SOURCE_LOG_COMPACT_INTERVAL.key -> "5")

        testStream(filtered)(
          AddTextFileData("drop1\nkeep2\nkeep3", src, tmp),
          CheckAnswer("keep2", "keep3"),
          AssertOnQuery(verify(_, 0L, 1, 2)),
          AddTextFileData("drop4\nkeep5\nkeep6", src, tmp),
          CheckAnswer("keep2", "keep3", "keep5", "keep6"),
          AssertOnQuery(verify(_, 1L, 2, 2)),
          AddTextFileData("drop7\nkeep8\nkeep9", src, tmp),
          CheckAnswer("keep2", "keep3", "keep5", "keep6", "keep8", "keep9"),
          AssertOnQuery(verify(_, 2L, 3, 2)),
          StopStream,
          StartStream(additionalConfs = updateConf),
          AssertOnQuery(verify(_, 2L, 3, 2)),
          AddTextFileData("drop10\nkeep11", src, tmp),
          CheckAnswer("keep2", "keep3", "keep5", "keep6", "keep8", "keep9", "keep11"),
          AssertOnQuery(verify(_, 3L, 4, 2)),
          AddTextFileData("drop12\nkeep13", src, tmp),
          CheckAnswer("keep2", "keep3", "keep5", "keep6", "keep8", "keep9", "keep11", "keep13"),
          AssertOnQuery(verify(_, 4L, 5, 2))
        )
      }
    }
  }

  test("get arbitrary batch from FileStreamSource") {
    withTempDirs { case (src, tmp) =>
      withSQLConf(
        SQLConf.FILE_SOURCE_LOG_COMPACT_INTERVAL.key -> "2",
        // Force deleting the old logs
        SQLConf.FILE_SOURCE_LOG_CLEANUP_DELAY.key -> "1"
      ) {
        val fileStream = createFileStream("text", src.getCanonicalPath)
        val filtered = fileStream.filter($"value" contains "keep")

        testStream(filtered)(
          AddTextFileData("keep1", src, tmp),
          CheckAnswer("keep1"),
          AddTextFileData("keep2", src, tmp),
          CheckAnswer("keep1", "keep2"),
          AddTextFileData("keep3", src, tmp),
          CheckAnswer("keep1", "keep2", "keep3"),
          AssertOnQuery("check getBatch") { execution: StreamExecution =>
            val _sources = PrivateMethod[Seq[Source]]('sources)
            val fileSource =
              (execution invokePrivate _sources()).head.asInstanceOf[FileStreamSource]

            def verify(startId: Option[Int], endId: Int, expected: String*): Unit = {
              val start = startId.map(new FileStreamSourceOffset(_))
              val end = FileStreamSourceOffset(endId)
              assert(fileSource.getBatch(start, end).as[String].collect().toSeq === expected)
            }

            verify(startId = None, endId = 2, "keep1", "keep2", "keep3")
            verify(startId = Some(0), endId = 1, "keep2")
            verify(startId = Some(0), endId = 2, "keep2", "keep3")
            verify(startId = Some(1), endId = 2, "keep3")
            true
          }
        )
      }
    }
  }

  test("input row metrics") {
    withTempDirs { case (src, tmp) =>
      val input = spark.readStream.format("text").load(src.getCanonicalPath)
      testStream(input)(
        AddTextFileData("100", src, tmp),
        CheckAnswer("100"),
        AssertOnQuery { query =>
          val actualProgress = query.recentProgress
              .find(_.numInputRows > 0)
              .getOrElse(sys.error("Could not find records with data."))
          assert(actualProgress.numInputRows === 1)
          assert(actualProgress.sources(0).processedRowsPerSecond > 0.0)
          true
        }
      )
    }
  }

  test("SPARK-18433: Improve DataSource option keys to be more case-insensitive") {
    val options = new FileStreamOptions(Map("maxfilespertrigger" -> "1"))
    assert(options.maxFilesPerTrigger == Some(1))
  }

  test("FileStreamSource offset - read Spark 2.1.0 offset json format") {
    val offset = readOffsetFromResource("file-source-offset-version-2.1.0-json.txt")
    assert(FileStreamSourceOffset(offset) === FileStreamSourceOffset(345))
  }

  test("FileStreamSource offset - read Spark 2.1.0 offset long format") {
    val offset = readOffsetFromResource("file-source-offset-version-2.1.0-long.txt")
    assert(FileStreamSourceOffset(offset) === FileStreamSourceOffset(345))
  }

  test("FileStreamSourceLog - read Spark 2.1.0 log format") {
    assert(readLogFromResource("file-source-log-version-2.1.0") === Seq(
      FileEntry("/a/b/0", 1480730949000L, 0L),
      FileEntry("/a/b/1", 1480730950000L, 1L),
      FileEntry("/a/b/2", 1480730950000L, 2L),
      FileEntry("/a/b/3", 1480730950000L, 3L),
      FileEntry("/a/b/4", 1480730951000L, 4L)
    ))
  }

  private def readLogFromResource(dir: String): Seq[FileEntry] = {
    val input = getClass.getResource(s"/structured-streaming/$dir")
    val log = new FileStreamSourceLog(FileStreamSourceLog.VERSION, spark, input.toString)
    log.allFiles()
  }

  private def readOffsetFromResource(file: String): SerializedOffset = {
    import scala.io.Source
    val str = Source.fromFile(getClass.getResource(s"/structured-streaming/$file").toURI).mkString
    SerializedOffset(str.trim)
  }

  private def runTwoBatchesAndVerifyResults(
      src: File,
      latestFirst: Boolean,
      firstBatch: String,
      secondBatch: String,
      maxFileAge: Option[String] = None): Unit = {
    val srcOptions = Map("latestFirst" -> latestFirst.toString, "maxFilesPerTrigger" -> "1") ++
      maxFileAge.map("maxFileAge" -> _)
    val fileStream = createFileStream(
      "text",
      src.getCanonicalPath,
      options = srcOptions)
    val clock = new StreamManualClock()
    testStream(fileStream)(
      StartStream(trigger = ProcessingTime(10), triggerClock = clock),
      AssertOnQuery { _ =>
        // Block until the first batch finishes.
        eventually(timeout(streamingTimeout)) {
          assert(clock.isStreamWaitingAt(0))
        }
        true
      },
      CheckLastBatch(firstBatch),
      AdvanceManualClock(10),
      AssertOnQuery { _ =>
        // Block until the second batch finishes.
        eventually(timeout(streamingTimeout)) {
          assert(clock.isStreamWaitingAt(10))
        }
        true
      },
      CheckLastBatch(secondBatch)
    )
  }

  test("FileStreamSource - latestFirst") {
    withTempDir { src =>
      // Prepare two files: 1.txt, 2.txt, and make sure they have different modified time.
      val f1 = stringToFile(new File(src, "1.txt"), "1")
      val f2 = stringToFile(new File(src, "2.txt"), "2")
      f2.setLastModified(f1.lastModified + 1000)

      // Read oldest files first, so the first batch is "1", and the second batch is "2".
      runTwoBatchesAndVerifyResults(src, latestFirst = false, firstBatch = "1", secondBatch = "2")

      // Read latest files first, so the first batch is "2", and the second batch is "1".
      runTwoBatchesAndVerifyResults(src, latestFirst = true, firstBatch = "2", secondBatch = "1")
    }
  }

  test("SPARK-19813: Ignore maxFileAge when maxFilesPerTrigger and latestFirst is used") {
    withTempDir { src =>
      // Prepare two files: 1.txt, 2.txt, and make sure they have different modified time.
      val f1 = stringToFile(new File(src, "1.txt"), "1")
      val f2 = stringToFile(new File(src, "2.txt"), "2")
      f2.setLastModified(f1.lastModified + 3600 * 1000 /* 1 hour later */)

      runTwoBatchesAndVerifyResults(src, latestFirst = true, firstBatch = "2", secondBatch = "1",
        maxFileAge = Some("1m") /* 1 minute */)
    }
  }

  test("SeenFilesMap") {
    val map = new SeenFilesMap(maxAgeMs = 10, fileNameOnly = false)

    map.add("a", 5)
    assert(map.size == 1)
    map.purge()
    assert(map.size == 1)

    // Add a new entry and purge should be no-op, since the gap is exactly 10 ms.
    map.add("b", 15)
    assert(map.size == 2)
    map.purge()
    assert(map.size == 2)

    // Add a new entry that's more than 10 ms than the first entry. We should be able to purge now.
    map.add("c", 16)
    assert(map.size == 3)
    map.purge()
    assert(map.size == 2)

    // Override existing entry shouldn't change the size
    map.add("c", 25)
    assert(map.size == 2)

    // Not a new file because we have seen c before
    assert(!map.isNewFile("c", 20))

    // Not a new file because timestamp is too old
    assert(!map.isNewFile("d", 5))

    // Finally a new file: never seen and not too old
    assert(map.isNewFile("e", 20))
  }

  test("SeenFilesMap with fileNameOnly = true") {
    val map = new SeenFilesMap(maxAgeMs = 10, fileNameOnly = true)

    map.add("file:///a/b/c/d", 5)
    map.add("file:///a/b/c/e", 5)
    assert(map.size === 2)

    assert(!map.isNewFile("d", 5))
    assert(!map.isNewFile("file:///d", 5))
    assert(!map.isNewFile("file:///x/d", 5))
    assert(!map.isNewFile("file:///x/y/d", 5))

    map.add("s3:///bucket/d", 5)
    map.add("s3n:///bucket/d", 5)
    map.add("s3a:///bucket/d", 5)
    assert(map.size === 2)
  }

  test("SeenFilesMap should only consider a file old if it is earlier than last purge time") {
    val map = new SeenFilesMap(maxAgeMs = 10, fileNameOnly = false)

    map.add("a", 20)
    assert(map.size == 1)

    // Timestamp 5 should still considered a new file because purge time should be 0
    assert(map.isNewFile("b", 9))
    assert(map.isNewFile("b", 10))

    // Once purge, purge time should be 10 and then b would be a old file if it is less than 10.
    map.purge()
    assert(!map.isNewFile("b", 9))
    assert(map.isNewFile("b", 10))
  }

  test("do not recheck that files exist during getBatch") {
    withTempDir { temp =>
      spark.conf.set(
        s"fs.$scheme.impl",
        classOf[ExistsThrowsExceptionFileSystem].getName)
      // add the metadata entries as a pre-req
      val dir = new File(temp, "dir") // use non-existent directory to test whether log make the dir
    val metadataLog =
      new FileStreamSourceLog(FileStreamSourceLog.VERSION, spark, dir.getAbsolutePath)
      assert(metadataLog.add(0, Array(FileEntry(s"$scheme:///file1", 100L, 0))))

      val newSource = new FileStreamSource(spark, s"$scheme:///", "parquet", StructType(Nil), Nil,
        dir.getAbsolutePath, Map.empty)
      // this method should throw an exception if `fs.exists` is called during resolveRelation
      newSource.getBatch(None, FileStreamSourceOffset(1))
    }
  }
}

class FileStreamSourceStressTestSuite extends FileStreamSourceTest {

  import testImplicits._

  testQuietly("file source stress test") {
    val src = Utils.createTempDir(namePrefix = "streaming.src")
    val tmp = Utils.createTempDir(namePrefix = "streaming.tmp")

    val fileStream = createFileStream("text", src.getCanonicalPath)
    val ds = fileStream.as[String].map(_.toInt + 1)
    runStressTest(ds, data => {
      AddTextFileData(data.mkString("\n"), src, tmp)
    })

    Utils.deleteRecursively(src)
    Utils.deleteRecursively(tmp)
  }
}

/**
 * Fake FileSystem to test whether the method `fs.exists` is called during
 * `DataSource.resolveRelation`.
 */
class ExistsThrowsExceptionFileSystem extends RawLocalFileSystem {
  override def getUri: URI = {
    URI.create(s"$scheme:///")
  }

  override def exists(f: Path): Boolean = {
    throw new IllegalArgumentException("Exists shouldn't have been called!")
  }

  /** Simply return an empty file for now. */
  override def listStatus(file: Path): Array[FileStatus] = {
    val emptyFile = new FileStatus()
    emptyFile.setPath(file)
    Array(emptyFile)
  }
}

object ExistsThrowsExceptionFileSystem {
  val scheme = s"FileStreamSourceSuite${math.abs(Random.nextInt)}fs"
}