&#010;&#010;== Results ==&#010;!== Correct Answer - 3 == == Spark Answer - 2 ==&#010;!struct<_1:bigint,_2:int> struct<CAST((CAST(value AS BIGINT) / 86400) AS BIGINT):bigint,count(1):bigint>&#010;![0,1] [10,1]&#010;![10,1] [5,2]&#010;![5,2] &#010; &#010;&#010;== Progress ==&#010; StartStream(ProcessingTime(864000000),org.apache.spark.sql.streaming.util.StreamManualClock@1a812534,Map(),null)&#010; AddData to MemoryStream[value#13875L]: 0,5,5,10&#010; AdvanceManualClock(864000000)&#010;=> CheckLastBatch: [0,1],[5,2],[10,1]&#010; AddData to MemoryStream[value#13875L]: 15,15,20&#010; AdvanceManualClock(864000000)&#010; CheckLastBatch: [10,1],[15,2],[20,1]&#010; AddData to MemoryStream[value#13875L]: 85&#010; AdvanceManualClock(864000000)&#010; CheckLastBatch: [20,1],[85,1]&#010; StopStream&#010; AssertOnQuery(<condition>, )&#010; StartStream(ProcessingTime(864000000),org.apache.spark.sql.streaming.util.StreamManualClock@1a812534,Map(),null)&#010; CheckLastBatch: [20,1],[85,1]&#010; AddData to MemoryStream[value#13875L]: 85,90,100,105&#010; AdvanceManualClock(864000000)&#010; CheckLastBatch: [90,1],[100,1],[105,1]&#010;&#010;== Stream ==&#010;Output Mode: Complete&#010;Stream state: {MemoryStream[value#13875L]: 0}&#010;Thread state: alive&#010;Thread stack trace: java.base@11.0.1/java.lang.Object.wait(Native Method)&#010;app//org.apache.spark.util.ManualClock.waitTillTime(ManualClock.scala:61)&#010;app//org.apache.spark.sql.streaming.util.StreamManualClock.waitTillTime(StreamManualClock.scala:34)&#010;app//org.apache.spark.sql.execution.streaming.ProcessingTimeExecutor.execute(TriggerExecutor.scala:65)&#010;app//org.apache.spark.sql.execution.streaming.MicroBatchExecution.runActivatedStream(MicroBatchExecution.scala:172)&#010;app//org.apache.spark.sql.execution.streaming.StreamExecution.org$apache$spark$sql$execution$streaming$StreamExecution$$runStream(StreamExecution.scala:331)&#010;app//org.apache.spark.sql.execution.streaming.StreamExecution$$anon$1.run(StreamExecution.scala:243)&#010;&#010;&#010;== Sink ==&#010;0: [10,1] [5,2]&#010;&#010;&#010;== Plan ==&#010;== Parsed Logical Plan ==&#010;Project [cast((cast(cast(value#13879 as bigint) as double) / cast(86400 as double)) as bigint) AS CAST((CAST(value AS BIGINT) / 86400) AS BIGINT)#13889L, count(1)#13884L]&#010;+- Filter (cast(value#13879 as date) >= date_sub(current_batch_timestamp(864000000, DateType, Some(UTC)), 10))&#010; +- Aggregate [value#13879], [value#13879, count(1) AS count(1)#13884L]&#010; +- Project [to_timestamp(from_unixtime((`value` * 86400L), 'yyyy-MM-dd HH:mm:ss'))#13877 AS value#13879]&#010; +- Project [to_timestamp(from_unixtime(('value * 86400), yyyy-MM-dd HH:mm:ss, None), None) AS to_timestamp(from_unixtime((`value` * 86400L), 'yyyy-MM-dd HH:mm:ss'))#13877]&#010; +- StreamingDataSourceV2Relation [value#13875L], org.apache.spark.sql.execution.streaming.MemoryStreamScanBuilder@238c3e16, MemoryStream[value#13875L], -1, 0&#010;&#010;== Analyzed Logical Plan ==&#010;CAST((CAST(value AS BIGINT) / 86400) AS BIGINT): bigint, count(1): bigint&#010;Project [cast((cast(cast(value#13879 as bigint) as double) / cast(86400 as double)) as bigint) AS CAST((CAST(value AS BIGINT) / 86400) AS BIGINT)#13889L, count(1)#13884L]&#010;+- Filter (cast(value#13879 as date) >= date_sub(current_batch_timestamp(864000000, DateType, Some(UTC)), 10))&#010; +- Aggregate [value#13879], [value#13879, count(1) AS count(1)#13884L]&#010; +- Project [to_timestamp(from_unixtime((`value` * 86400L), 'yyyy-MM-dd HH:mm:ss'))#13877 AS value#13879]&#010; +- Project [to_timestamp(from_unixtime(('value * 86400), yyyy-MM-dd HH:mm:ss, None), None) AS to_timestamp(from_unixtime((`value` * 86400L), 'yyyy-MM-dd HH:mm:ss'))#13877]&#010; +- StreamingDataSourceV2Relation [value#13875L], org.apache.spark.sql.execution.streaming.MemoryStreamScanBuilder@238c3e16, MemoryStream[value#13875L], -1, 0&#010;&#010;== Optimized Logical Plan ==&#010;Project [cast((cast(cast(value#13879 as bigint) as double) / 86400.0) as bigint) AS CAST((CAST(value AS BIGINT) / 86400) AS BIGINT)#13889L, count(1)#13884L]&#010;+- Filter (cast(value#13879 as date) >= date_sub(10, 10))&#010; +- Aggregate [value#13879], [value#13879, count(1) AS count(1)#13884L]&#010; +- Project [cast(from_unixtime((value#13875L * 86400), yyyy-MM-dd HH:mm:ss, Some(America/Los_Angeles)) as timestamp) AS value#13879]&#010; +- Filter isnotnull(cast(from_unixtime((value#13875L * 86400), yyyy-MM-dd HH:mm:ss, Some(America/Los_Angeles)) as timestamp))&#010; +- StreamingDataSourceV2Relation [value#13875L], org.apache.spark.sql.execution.streaming.MemoryStreamScanBuilder@238c3e16, MemoryStream[value#13875L], -1, 0&#010;&#010;== Physical Plan ==&#010;*(3) Project [cast((cast(cast(value#13879 as bigint) as double) / 86400.0) as bigint) AS CAST((CAST(value AS BIGINT) / 86400) AS BIGINT)#13889L, count(1)#13884L]&#010;+- *(3) Filter (cast(value#13879 as date) >= date_sub(10, 10))&#010; +- *(3) HashAggregate(keys=[value#13879], functions=[count(1)], output=[value#13879, count(1)#13884L])&#010; +- StateStoreSave [value#13879], state info [ checkpoint = file:/home/jenkins/workspace/spark-master-test-maven-hadoop-2.7-jdk-11-ubuntu-testing@2/sql/core/target/tmp/streaming.metadata-0c641ea2-7fd2-475c-9163-523feeeba8fc/state, runId = 62081e9d-8c7d-4382-b2db-a6071d313e25, opId = 0, ver = 0, numPartitions = 5], Complete, 0, 2&#010; +- *(2) HashAggregate(keys=[value#13879], functions=[merge_count(1)], output=[value#13879, count#13941L])&#010; +- StateStoreRestore [value#13879], state info [ checkpoint = file:/home/jenkins/workspace/spark-master-test-maven-hadoop-2.7-jdk-11-ubuntu-testing@2/sql/core/target/tmp/streaming.metadata-0c641ea2-7fd2-475c-9163-523feeeba8fc/state, runId = 62081e9d-8c7d-4382-b2db-a6071d313e25, opId = 0, ver = 0, numPartitions = 5], 2&#010; +- Exchange hashpartitioning(value#13879, 5)&#010; +- *(1) HashAggregate(keys=[value#13879], functions=[merge_count(1)], output=[value#13879, count#13941L])&#010; +- *(1) HashAggregate(keys=[value#13879], functions=[partial_count(1)], output=[value#13879, count#13941L])&#010; +- *(1) Project [cast(from_unixtime((value#13875L * 86400), yyyy-MM-dd HH:mm:ss, Some(America/Los_Angeles)) as timestamp) AS value#13879]&#010; +- *(1) Filter isnotnull(cast(from_unixtime((value#13875L * 86400), yyyy-MM-dd HH:mm:ss, Some(America/Los_Angeles)) as timestamp))&#010; +- *(1) Project [value#13875L]&#010; +- *(1) MicroBatchScan[value#13875L] MemoryStreamDataSource&#010;&#010; &#010;


      org.scalatest.exceptions.TestFailedException: 

== Results ==
!== Correct Answer - 3 ==   == Spark Answer - 2 ==
!struct<_1:bigint,_2:int>   struct<CAST((CAST(value AS BIGINT) / 86400) AS BIGINT):bigint,count(1):bigint>
![0,1]                      [10,1]
![10,1]                     [5,2]
![5,2]                      
    

== Progress ==
   StartStream(ProcessingTime(864000000),org.apache.spark.sql.streaming.util.StreamManualClock@1a812534,Map(),null)
   AddData to MemoryStream[value#13875L]: 0,5,5,10
   AdvanceManualClock(864000000)
=> CheckLastBatch: [0,1],[5,2],[10,1]
   AddData to MemoryStream[value#13875L]: 15,15,20
   AdvanceManualClock(864000000)
   CheckLastBatch: [10,1],[15,2],[20,1]
   AddData to MemoryStream[value#13875L]: 85
   AdvanceManualClock(864000000)
   CheckLastBatch: [20,1],[85,1]
   StopStream
   AssertOnQuery(<condition>, )
   StartStream(ProcessingTime(864000000),org.apache.spark.sql.streaming.util.StreamManualClock@1a812534,Map(),null)
   CheckLastBatch: [20,1],[85,1]
   AddData to MemoryStream[value#13875L]: 85,90,100,105
   AdvanceManualClock(864000000)
   CheckLastBatch: [90,1],[100,1],[105,1]

== Stream ==
Output Mode: Complete
Stream state: {MemoryStream[value#13875L]: 0}
Thread state: alive
Thread stack trace: java.base@11.0.1/java.lang.Object.wait(Native Method)
app//org.apache.spark.util.ManualClock.waitTillTime(ManualClock.scala:61)
app//org.apache.spark.sql.streaming.util.StreamManualClock.waitTillTime(StreamManualClock.scala:34)
app//org.apache.spark.sql.execution.streaming.ProcessingTimeExecutor.execute(TriggerExecutor.scala:65)
app//org.apache.spark.sql.execution.streaming.MicroBatchExecution.runActivatedStream(MicroBatchExecution.scala:172)
app//org.apache.spark.sql.execution.streaming.StreamExecution.org$apache$spark$sql$execution$streaming$StreamExecution$$runStream(StreamExecution.scala:331)
app//org.apache.spark.sql.execution.streaming.StreamExecution$$anon$1.run(StreamExecution.scala:243)


== Sink ==
0: [10,1] [5,2]


== Plan ==
== Parsed Logical Plan ==
Project [cast((cast(cast(value#13879 as bigint) as double) / cast(86400 as double)) as bigint) AS CAST((CAST(value AS BIGINT) / 86400) AS BIGINT)#13889L, count(1)#13884L]
+- Filter (cast(value#13879 as date) >= date_sub(current_batch_timestamp(864000000, DateType, Some(UTC)), 10))
   +- Aggregate [value#13879], [value#13879, count(1) AS count(1)#13884L]
      +- Project [to_timestamp(from_unixtime((`value` * 86400L), 'yyyy-MM-dd HH:mm:ss'))#13877 AS value#13879]
         +- Project [to_timestamp(from_unixtime(('value * 86400), yyyy-MM-dd HH:mm:ss, None), None) AS to_timestamp(from_unixtime((`value` * 86400L), 'yyyy-MM-dd HH:mm:ss'))#13877]
            +- StreamingDataSourceV2Relation [value#13875L], org.apache.spark.sql.execution.streaming.MemoryStreamScanBuilder@238c3e16, MemoryStream[value#13875L], -1, 0

== Analyzed Logical Plan ==
CAST((CAST(value AS BIGINT) / 86400) AS BIGINT): bigint, count(1): bigint
Project [cast((cast(cast(value#13879 as bigint) as double) / cast(86400 as double)) as bigint) AS CAST((CAST(value AS BIGINT) / 86400) AS BIGINT)#13889L, count(1)#13884L]
+- Filter (cast(value#13879 as date) >= date_sub(current_batch_timestamp(864000000, DateType, Some(UTC)), 10))
   +- Aggregate [value#13879], [value#13879, count(1) AS count(1)#13884L]
      +- Project [to_timestamp(from_unixtime((`value` * 86400L), 'yyyy-MM-dd HH:mm:ss'))#13877 AS value#13879]
         +- Project [to_timestamp(from_unixtime(('value * 86400), yyyy-MM-dd HH:mm:ss, None), None) AS to_timestamp(from_unixtime((`value` * 86400L), 'yyyy-MM-dd HH:mm:ss'))#13877]
            +- StreamingDataSourceV2Relation [value#13875L], org.apache.spark.sql.execution.streaming.MemoryStreamScanBuilder@238c3e16, MemoryStream[value#13875L], -1, 0

== Optimized Logical Plan ==
Project [cast((cast(cast(value#13879 as bigint) as double) / 86400.0) as bigint) AS CAST((CAST(value AS BIGINT) / 86400) AS BIGINT)#13889L, count(1)#13884L]
+- Filter (cast(value#13879 as date) >= date_sub(10, 10))
   +- Aggregate [value#13879], [value#13879, count(1) AS count(1)#13884L]
      +- Project [cast(from_unixtime((value#13875L * 86400), yyyy-MM-dd HH:mm:ss, Some(America/Los_Angeles)) as timestamp) AS value#13879]
         +- Filter isnotnull(cast(from_unixtime((value#13875L * 86400), yyyy-MM-dd HH:mm:ss, Some(America/Los_Angeles)) as timestamp))
            +- StreamingDataSourceV2Relation [value#13875L], org.apache.spark.sql.execution.streaming.MemoryStreamScanBuilder@238c3e16, MemoryStream[value#13875L], -1, 0

== Physical Plan ==
*(3) Project [cast((cast(cast(value#13879 as bigint) as double) / 86400.0) as bigint) AS CAST((CAST(value AS BIGINT) / 86400) AS BIGINT)#13889L, count(1)#13884L]
+- *(3) Filter (cast(value#13879 as date) >= date_sub(10, 10))
   +- *(3) HashAggregate(keys=[value#13879], functions=[count(1)], output=[value#13879, count(1)#13884L])
      +- StateStoreSave [value#13879], state info [ checkpoint = file:/home/jenkins/workspace/spark-master-test-maven-hadoop-2.7-jdk-11-ubuntu-testing@2/sql/core/target/tmp/streaming.metadata-0c641ea2-7fd2-475c-9163-523feeeba8fc/state, runId = 62081e9d-8c7d-4382-b2db-a6071d313e25, opId = 0, ver = 0, numPartitions = 5], Complete, 0, 2
         +- *(2) HashAggregate(keys=[value#13879], functions=[merge_count(1)], output=[value#13879, count#13941L])
            +- StateStoreRestore [value#13879], state info [ checkpoint = file:/home/jenkins/workspace/spark-master-test-maven-hadoop-2.7-jdk-11-ubuntu-testing@2/sql/core/target/tmp/streaming.metadata-0c641ea2-7fd2-475c-9163-523feeeba8fc/state, runId = 62081e9d-8c7d-4382-b2db-a6071d313e25, opId = 0, ver = 0, numPartitions = 5], 2
               +- Exchange hashpartitioning(value#13879, 5)
                  +- *(1) HashAggregate(keys=[value#13879], functions=[merge_count(1)], output=[value#13879, count#13941L])
                     +- *(1) HashAggregate(keys=[value#13879], functions=[partial_count(1)], output=[value#13879, count#13941L])
                        +- *(1) Project [cast(from_unixtime((value#13875L * 86400), yyyy-MM-dd HH:mm:ss, Some(America/Los_Angeles)) as timestamp) AS value#13879]
                           +- *(1) Filter isnotnull(cast(from_unixtime((value#13875L * 86400), yyyy-MM-dd HH:mm:ss, Some(America/Los_Angeles)) as timestamp))
                              +- *(1) Project [value#13875L]
                                 +- *(1) MicroBatchScan[value#13875L] MemoryStreamDataSource

         
         
      at org.scalatest.Assertions.newAssertionFailedException(Assertions.scala:528)
      at org.scalatest.Assertions.newAssertionFailedException$(Assertions.scala:527)
      at org.scalatest.FunSuite.newAssertionFailedException(FunSuite.scala:1560)
      at org.scalatest.Assertions.fail(Assertions.scala:1089)
      at org.scalatest.Assertions.fail$(Assertions.scala:1085)
      at org.scalatest.FunSuite.fail(FunSuite.scala:1560)
      at org.apache.spark.sql.streaming.StreamTest.failTest$1(StreamTest.scala:452)
      at org.apache.spark.sql.streaming.StreamTest.$anonfun$testStream$53(StreamTest.scala:733)
      at scala.Option.foreach(Option.scala:274)
      at org.apache.spark.sql.streaming.StreamTest.executeAction$1(StreamTest.scala:733)
      at org.apache.spark.sql.streaming.StreamTest.$anonfun$testStream$56(StreamTest.scala:778)
      at org.apache.spark.sql.streaming.StreamTest.$anonfun$testStream$56$adapted(StreamTest.scala:765)
      at scala.collection.IndexedSeqOptimized.foreach(IndexedSeqOptimized.scala:36)
      at scala.collection.IndexedSeqOptimized.foreach$(IndexedSeqOptimized.scala:33)
      at scala.collection.mutable.WrappedArray.foreach(WrappedArray.scala:39)
      at org.apache.spark.sql.streaming.StreamTest.liftedTree1$1(StreamTest.scala:765)
      at org.apache.spark.sql.streaming.StreamTest.testStream(StreamTest.scala:764)
      at org.apache.spark.sql.streaming.StreamTest.testStream$(StreamTest.scala:328)
      at org.apache.spark.sql.streaming.StreamingAggregationSuite.testStream(StreamingAggregationSuite.scala:49)
      at org.apache.spark.sql.streaming.StreamingAggregationSuite.$anonfun$new$35(StreamingAggregationSuite.scala:388)
      at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
      at org.apache.spark.sql.streaming.StreamingAggregationSuite.$anonfun$executeFuncWithStateVersionSQLConf$1(StreamingAggregationSuite.scala:59)
      at org.apache.spark.sql.catalyst.plans.SQLHelper.withSQLConf(SQLHelper.scala:47)
      at org.apache.spark.sql.catalyst.plans.SQLHelper.withSQLConf$(SQLHelper.scala:31)
      at org.apache.spark.sql.streaming.StreamingAggregationSuite.org$apache$spark$sql$test$SQLTestUtilsBase$$super$withSQLConf(StreamingAggregationSuite.scala:49)
      at org.apache.spark.sql.test.SQLTestUtilsBase.withSQLConf(SQLTestUtils.scala:230)
      at org.apache.spark.sql.test.SQLTestUtilsBase.withSQLConf$(SQLTestUtils.scala:228)
      at org.apache.spark.sql.streaming.StreamingAggregationSuite.withSQLConf(StreamingAggregationSuite.scala:49)
      at org.apache.spark.sql.streaming.StreamingAggregationSuite.executeFuncWithStateVersionSQLConf(StreamingAggregationSuite.scala:59)
      at org.apache.spark.sql.streaming.StreamingAggregationSuite.$anonfun$testWithAllStateVersions$2(StreamingAggregationSuite.scala:67)
      at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
      at org.scalatest.OutcomeOf.outcomeOf(OutcomeOf.scala:85)
      at org.scalatest.OutcomeOf.outcomeOf$(OutcomeOf.scala:83)
      at org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104)
      at org.scalatest.Transformer.apply(Transformer.scala:22)
      at org.scalatest.Transformer.apply(Transformer.scala:20)
      at org.scalatest.FunSuiteLike$$anon$1.apply(FunSuiteLike.scala:186)
      at org.apache.spark.SparkFunSuite.withFixture(SparkFunSuite.scala:105)
      at org.scalatest.FunSuiteLike.invokeWithFixture$1(FunSuiteLike.scala:184)
      at org.scalatest.FunSuiteLike.$anonfun$runTest$1(FunSuiteLike.scala:196)
      at org.scalatest.SuperEngine.runTestImpl(Engine.scala:289)
      at org.scalatest.FunSuiteLike.runTest(FunSuiteLike.scala:196)
      at org.scalatest.FunSuiteLike.runTest$(FunSuiteLike.scala:178)
      at org.apache.spark.sql.streaming.StreamingAggregationSuite.org$scalatest$BeforeAndAfterEach$$super$runTest(StreamingAggregationSuite.scala:49)
      at org.scalatest.BeforeAndAfterEach.runTest(BeforeAndAfterEach.scala:221)
      at org.scalatest.BeforeAndAfterEach.runTest$(BeforeAndAfterEach.scala:214)
      at org.apache.spark.sql.streaming.StreamingAggregationSuite.runTest(StreamingAggregationSuite.scala:49)
      at org.scalatest.FunSuiteLike.$anonfun$runTests$1(FunSuiteLike.scala:229)
      at org.scalatest.SuperEngine.$anonfun$runTestsInBranch$1(Engine.scala:396)
      at scala.collection.immutable.List.foreach(List.scala:392)
      at org.scalatest.SuperEngine.traverseSubNodes$1(Engine.scala:384)
      at org.scalatest.SuperEngine.runTestsInBranch(Engine.scala:379)
      at org.scalatest.SuperEngine.runTestsImpl(Engine.scala:461)
      at org.scalatest.FunSuiteLike.runTests(FunSuiteLike.scala:229)
      at org.scalatest.FunSuiteLike.runTests$(FunSuiteLike.scala:228)
      at org.scalatest.FunSuite.runTests(FunSuite.scala:1560)
      at org.scalatest.Suite.run(Suite.scala:1147)
      at org.scalatest.Suite.run$(Suite.scala:1129)
      at org.scalatest.FunSuite.org$scalatest$FunSuiteLike$$super$run(FunSuite.scala:1560)
      at org.scalatest.FunSuiteLike.$anonfun$run$1(FunSuiteLike.scala:233)
      at org.scalatest.SuperEngine.runImpl(Engine.scala:521)
      at org.scalatest.FunSuiteLike.run(FunSuiteLike.scala:233)
      at org.scalatest.FunSuiteLike.run$(FunSuiteLike.scala:232)
      at org.apache.spark.SparkFunSuite.org$scalatest$BeforeAndAfterAll$$super$run(SparkFunSuite.scala:54)
      at org.scalatest.BeforeAndAfterAll.liftedTree1$1(BeforeAndAfterAll.scala:213)
      at org.scalatest.BeforeAndAfterAll.run(BeforeAndAfterAll.scala:210)
      at org.scalatest.BeforeAndAfterAll.run$(BeforeAndAfterAll.scala:208)
      at org.apache.spark.SparkFunSuite.run(SparkFunSuite.scala:54)
      at org.scalatest.Suite.callExecuteOnSuite$1(Suite.scala:1210)
      at org.scalatest.Suite.$anonfun$runNestedSuites$1(Suite.scala:1257)
      at scala.collection.IndexedSeqOptimized.foreach(IndexedSeqOptimized.scala:36)
      at scala.collection.IndexedSeqOptimized.foreach$(IndexedSeqOptimized.scala:33)
      at scala.collection.mutable.ArrayOps$ofRef.foreach(ArrayOps.scala:198)
      at org.scalatest.Suite.runNestedSuites(Suite.scala:1255)
      at org.scalatest.Suite.runNestedSuites$(Suite.scala:1189)
      at org.scalatest.tools.DiscoverySuite.runNestedSuites(DiscoverySuite.scala:30)
      at org.scalatest.Suite.run(Suite.scala:1144)
      at org.scalatest.Suite.run$(Suite.scala:1129)
      at org.scalatest.tools.DiscoverySuite.run(DiscoverySuite.scala:30)
      at org.scalatest.tools.SuiteRunner.run(SuiteRunner.scala:45)
      at org.scalatest.tools.Runner$.$anonfun$doRunRunRunDaDoRunRun$13(Runner.scala:1346)
      at org.scalatest.tools.Runner$.$anonfun$doRunRunRunDaDoRunRun$13$adapted(Runner.scala:1340)
      at scala.collection.immutable.List.foreach(List.scala:392)
      at org.scalatest.tools.Runner$.doRunRunRunDaDoRunRun(Runner.scala:1340)
      at org.scalatest.tools.Runner$.$anonfun$runOptionallyWithPassFailReporter$24(Runner.scala:1031)
      at org.scalatest.tools.Runner$.$anonfun$runOptionallyWithPassFailReporter$24$adapted(Runner.scala:1010)
      at org.scalatest.tools.Runner$.withClassLoaderAndDispatchReporter(Runner.scala:1506)
      at org.scalatest.tools.Runner$.runOptionallyWithPassFailReporter(Runner.scala:1010)
      at org.scalatest.tools.Runner$.main(Runner.scala:827)
      at org.scalatest.tools.Runner.main(Runner.scala)