[ https://issues.apache.org/jira/browse/CARBONDATA-1790?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ramakrishna S updated CARBONDATA-1790: -------------------------------------- Description: Steps : 1. Create a streaming table and do a batch load 2. Set up the Streaming , so that it does streaming in chunk of 1000 records 20 times 3. Do another batch load on the table 4. Do one more time streaming +-------------+------------+--------------------------+--------------------------+--------------+------------+--+ | Segment Id | Status | Load Start Time | Load End Time | File Format | Merged To | +-------------+------------+--------------------------+--------------------------+--------------+------------+--+ | 2 | Success | 2017-11-21 21:42:36.77 | 2017-11-21 21:42:40.396 | COLUMNAR_V3 | NA | | 1 | Streaming | 2017-11-21 21:40:46.2 | NULL | ROW_V1 | NA | | 0 | Success | 2017-11-21 21:40:39.782 | 2017-11-21 21:40:43.168 | COLUMNAR_V3 | NA | +-------------+------------+--------------------------+--------------------------+--------------+------------+--+ *+Expected:+* Data should be loaded *+Actual+* : Data load fiails 1. One addition offset file is created(marked in bold) -rw-r--r-- 2 root users 62 2017-11-21 21:40 /user/hive/warehouse/Ram/default/stream_table5/.streaming/checkpoint/offsets/0 -rw-r--r-- 2 root users 63 2017-11-21 21:40 /user/hive/warehouse/Ram/default/stream_table5/.streaming/checkpoint/offsets/1 -rw-r--r-- 2 root users 63 2017-11-21 21:42 /user/hive/warehouse/Ram/default/stream_table5/.streaming/checkpoint/offsets/10 -rw-r--r-- 2 root users 63 2017-11-21 21:40 /user/hive/warehouse/Ram/default/stream_table5/.streaming/checkpoint/offsets/2 -rw-r--r-- 2 root users 63 2017-11-21 21:41 /user/hive/warehouse/Ram/default/stream_table5/.streaming/checkpoint/offsets/3 -rw-r--r-- 2 root users 64 2017-11-21 21:41 /user/hive/warehouse/Ram/default/stream_table5/.streaming/checkpoint/offsets/4 -rw-r--r-- 2 root users 64 2017-11-21 21:41 /user/hive/warehouse/Ram/default/stream_table5/.streaming/checkpoint/offsets/5 -rw-r--r-- 2 root users 64 2017-11-21 21:41 /user/hive/warehouse/Ram/default/stream_table5/.streaming/checkpoint/offsets/6 -rw-r--r-- 2 root users 64 2017-11-21 21:41 /user/hive/warehouse/Ram/default/stream_table5/.streaming/checkpoint/offsets/7 -rw-r--r-- 2 root users 64 2017-11-21 21:41 /user/hive/warehouse/Ram/default/stream_table5/.streaming/checkpoint/offsets/8 *-rw-r--r-- 2 root users 63 2017-11-21 21:42 /user/hive/warehouse/Ram/default/stream_table5/.streaming/checkpoint/offsets/9* 2. Following error thrown: === Streaming Query === Identifier: [id = 3a5334bc-d471-4676-b6ce-f21105d491d1, runId = b2be9f97-8141-46be-89db-9a0f98d13369] Current Offsets: {org.apache.spark.sql.execution.streaming.TextSocketSource@14c45193: 1000} Current State: ACTIVE Thread State: RUNNABLE Logical Plan: org.apache.spark.sql.execution.streaming.TextSocketSource@14c45193 at org.apache.spark.sql.execution.streaming.StreamExecution.org$apache$spark$sql$execution$streaming$StreamExecution$$runBatches(StreamExecution.scala:284) at org.apache.spark.sql.execution.streaming.StreamExecution$$anon$1.run(StreamExecution.scala:177) Caused by: java.lang.RuntimeException: Offsets committed out of order: 20019 followed by 1000 at scala.sys.package$.error(package.scala:27) at org.apache.spark.sql.execution.streaming.TextSocketSource.commit(socket.scala:151) at org.apache.spark.sql.execution.streaming.StreamExecution$$anonfun$org$apache$spark$sql$execution$streaming$StreamExecution$$constructNextBatch$2$$anonfun$apply$mcV$sp$4.apply(StreamExecution.scala:421) at org.apache.spark.sql.execution.streaming.StreamExecution$$anonfun$org$apache$spark$sql$execution$streaming$StreamExecution$$constructNextBatch$2$$anonfun$apply$mcV$sp$4.apply(StreamExecution.scala:420) at scala.collection.Iterator$class.foreach(Iterator.scala:893) at scala.collection.AbstractIterator.foreach(Iterator.scala:1336) at scala.collection.IterableLike$class.foreach(IterableLike.scala:72) at org.apache.spark.sql.execution.streaming.StreamProgress.foreach(StreamProgress.scala:25) at org.apache.spark.sql.execution.streaming.StreamExecution$$anonfun$org$apache$spark$sql$execution$streaming$StreamExecution$$constructNextBatch$2.apply$mcV$sp(StreamExecution.scala:420) at org.apache.spark.sql.execution.streaming.StreamExecution$$anonfun$org$apache$spark$sql$execution$streaming$StreamExecution$$constructNextBatch$2.apply(StreamExecution.scala:404) at org.apache.spark.sql.execution.streaming.StreamExecution$$anonfun$org$apache$spark$sql$execution$streaming$StreamExecution$$constructNextBatch$2.apply(StreamExecution.scala:404) at org.apache.spark.sql.execution.streaming.ProgressReporter$class.reportTimeTaken(ProgressReporter.scala:262) at org.apache.spark.sql.execution.streaming.StreamExecution.reportTimeTaken(StreamExecution.scala:46) at org.apache.spark.sql.execution.streaming.StreamExecution.org$apache$spark$sql$execution$streaming$StreamExecution$$constructNextBatch(StreamExecution.scala:404) at org.apache.spark.sql.execution.streaming.StreamExecution$$anonfun$org$apache$spark$sql$execution$streaming$StreamExecution$$runBatches$1$$anonfun$1.apply$mcV$sp(StreamExecution.scala:250) at org.apache.spark.sql.execution.streaming.StreamExecution$$anonfun$org$apache$spark$sql$execution$streaming$StreamExecution$$runBatches$1$$anonfun$1.apply(StreamExecution.scala:244) at org.apache.spark.sql.execution.streaming.StreamExecution$$anonfun$org$apache$spark$sql$execution$streaming$StreamExecution$$runBatches$1$$anonfun$1.apply(StreamExecution.scala:244) at org.apache.spark.sql.execution.streaming.ProgressReporter$class.reportTimeTaken(ProgressReporter.scala:262) at org.apache.spark.sql.execution.streaming.StreamExecution.reportTimeTaken(StreamExecution.scala:46) at org.apache.spark.sql.execution.streaming.StreamExecution$$anonfun$org$apache$spark$sql$execution$streaming$StreamExecution$$runBatches$1.apply$mcZ$sp(StreamExecution.scala:244) at org.apache.spark.sql.execution.streaming.ProcessingTimeExecutor.execute(TriggerExecutor.scala:43) at org.apache.spark.sql.execution.streaming.StreamExecution.org$apache$spark$sql$execution$streaming$StreamExecution$$runBatches(StreamExecution.scala:239) ... 1 more Done reading and writing streaming data Socket closed was: Steps : User starts the thrift server using the command - bin/spark-submit --master yarn-client --executor-memory 10G --executor-cores 5 --driver-memory 5G --num-executors 3 --class org.apache.carbondata.spark.thriftserver.CarbonThriftServer /srv/spark2.2Bigdata/install/spark/sparkJdbc/carbonlib/carbondata_2.11-1.3.0-SNAPSHOT-shade-hadoop2.7.2.jar "hdfs://hacluster/user/hive/warehouse/carbon.store" User connects to spark shell using the command - bin/spark-shell --master yarn-client --executor-memory 10G --executor-cores 5 --driver-memory 5G --num-executors 3 --jars /srv/spark2.2Bigdata/install/spark/sparkJdbc/carbonlib/carbondata_2.11-1.3.0-SNAPSHOT-shade-hadoop2.7.2.jar In spark shell User creates a table and does streaming load in the table as per the below socket streaming script. import java.io.{File, PrintWriter} import java.net.ServerSocket import org.apache.spark.sql.{CarbonEnv, SparkSession} import org.apache.spark.sql.hive.CarbonRelation import org.apache.spark.sql.streaming.{ProcessingTime, StreamingQuery} import org.apache.carbondata.core.constants.CarbonCommonConstants import org.apache.carbondata.core.util.CarbonProperties import org.apache.carbondata.core.util.path.{CarbonStorePath, CarbonTablePath} CarbonProperties.getInstance().addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "yyyy/MM/dd") import org.apache.spark.sql.CarbonSession._ val carbonSession = SparkSession. builder(). appName("StreamExample"). getOrCreateCarbonSession("hdfs://hacluster/user/hive/warehouse/david") carbonSession.sparkContext.setLogLevel("INFO") def sql(sql: String) = carbonSession.sql(sql) def writeSocket(serverSocket: ServerSocket): Thread = { val thread = new Thread() { override def run(): Unit = { // wait for client to connection request and accept val clientSocket = serverSocket.accept() val socketWriter = new PrintWriter(clientSocket.getOutputStream()) var index = 0 for (_ <- 1 to 1000) { // write 5 records per iteration for (_ <- 0 to 100) { index = index + 1 socketWriter.println(index.toString + ",name_" + index + ",city_" + index + "," + (index * 10000.00).toString + ",school_" + index + ":school_" + index + index + "$" + index) } socketWriter.flush() Thread.sleep(2000) } socketWriter.close() System.out.println("Socket closed") } } thread.start() thread } def startStreaming(spark: SparkSession, tablePath: CarbonTablePath, tableName: String, port: Int): Thread = { val thread = new Thread() { override def run(): Unit = { var qry: StreamingQuery = null try { val readSocketDF = spark.readStream .format("socket") .option("host", "10.18.98.34") .option("port", port) .load() qry = readSocketDF.writeStream .format("carbondata") .trigger(ProcessingTime("5 seconds")) .option("checkpointLocation", tablePath.getStreamingCheckpointDir) .option("tablePath", tablePath.getPath).option("tableName", tableName) .start() qry.awaitTermination() } catch { case ex: Throwable => ex.printStackTrace() println("Done reading and writing streaming data") } finally { qry.stop() } } } thread.start() thread } val streamTableName = "stream_table" sql(s"CREATE TABLE $streamTableName (id INT,name STRING,city STRING,salary FLOAT) STORED BY 'carbondata' TBLPROPERTIES('streaming'='true', 'sort_columns'='name')") sql(s"LOAD DATA LOCAL INPATH 'hdfs://hacluster/tmp/streamSample.csv' INTO TABLE $streamTableName OPTIONS('HEADER'='true')") sql(s"select * from $streamTableName").show val carbonTable = CarbonEnv.getInstance(carbonSession).carbonMetastore. lookupRelation(Some("default"), streamTableName)(carbonSession).asInstanceOf[CarbonRelation].carbonTable val tablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier) val port = 7995 val serverSocket = new ServerSocket(port) val socketThread = writeSocket(serverSocket) val streamingThread = startStreaming(carbonSession, tablePath, streamTableName, port) While load is in progress user executes select query on the streaming table from beeline. 0: jdbc:hive2://10.18.98.34:23040> select * from stream_table; *Issue : The Select query fails with java.io.EOFException when socket streaming is in progress.* 0: jdbc:hive2://10.18.98.34:23040> select * from stream_table; Error: org.apache.spark.SparkException: Job aborted due to stage failure: Task 3 in stage 1.0 failed 4 times, most recent failure: Lost task 3.3 in stage 1.0 (TID 38, BLR1000014278, executor 7): java.io.EOFException at org.apache.carbondata.hadoop.streaming.StreamBlockletReader.readBytesFromStream(StreamBlockletReader.java:182) at org.apache.carbondata.hadoop.streaming.StreamBlockletReader.readBlockletData(StreamBlockletReader.java:116) at org.apache.carbondata.hadoop.streaming.CarbonStreamRecordReader.scanBlockletAndFillVector(CarbonStreamRecordReader.java:406) at org.apache.carbondata.hadoop.streaming.CarbonStreamRecordReader.nextColumnarBatch(CarbonStreamRecordReader.java:317) at org.apache.carbondata.hadoop.streaming.CarbonStreamRecordReader.nextKeyValue(CarbonStreamRecordReader.java:298) at org.apache.carbondata.spark.rdd.CarbonScanRDD$$anon$1.hasNext(CarbonScanRDD.scala:298) at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator.scan_nextBatch$(Unknown Source) at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator.processNext(Unknown Source) at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43) at org.apache.spark.sql.execution.WholeStageCodegenExec$$anonfun$8$$anon$1.hasNext(WholeStageCodegenExec.scala:377) at org.apache.spark.sql.execution.SparkPlan$$anonfun$2.apply(SparkPlan.scala:231) at org.apache.spark.sql.execution.SparkPlan$$anonfun$2.apply(SparkPlan.scala:225) at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$25.apply(RDD.scala:826) at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$25.apply(RDD.scala:826) at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:323) at org.apache.spark.rdd.RDD.iterator(RDD.scala:287) at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87) at org.apache.spark.scheduler.Task.run(Task.scala:99) at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) at java.lang.Thread.run(Thread.java:745) Driver stacktrace: (state=,code=0) *Also when user checks the spark shell terminal there are exceptions thrown.* scala> org.apache.spark.sql.streaming.StreamingQueryException: Offsets committed out of order: 100999 followed by 100 scala.sys.package$.error(package.scala:27) org.apache.spark.sql.execution.streaming.TextSocketSource.commit(socket.scala:151) org.apache.spark.sql.execution.streaming.StreamExecution$$anonfun$org$apache$spark$sql$execution$streaming$StreamExecution$$constructNextBatch$2$$anonfun$apply$mcV$sp$4.apply(StreamExecution.scala:421) org.apache.spark.sql.execution.streaming.StreamExecution$$anonfun$org$apache$spark$sql$execution$streaming$StreamExecution$$constructNextBatch$2$$anonfun$apply$mcV$sp$4.apply(StreamExecution.scala:420) scala.collection.Iterator$class.foreach(Iterator.scala:893) scala.collection.AbstractIterator.foreach(Iterator.scala:1336) scala.collection.IterableLike$class.foreach(IterableLike.scala:72) org.apache.spark.sql.execution.streaming.StreamProgress.foreach(StreamProgress.scala:25) org.apache.spark.sql.execution.streaming.StreamExecution$$anonfun$org$apache$spark$sql$execution$streaming$StreamExecution$$constructNextBatch$2.apply$mcV$sp(StreamExecution.scala:420) org.apache.spark.sql.execution.streaming.StreamExecution$$anonfun$org$apache$spark$sql$execution$streaming$StreamExecution$$constructNextBatch$2.apply(StreamExecution.scala:404) === Streaming Query === Identifier: [id = d23c5633-e747-457e-a5c0-69ec09bb183f, runId = 2db93553-fe97-4fa6-b425-278128a42f50] Current Offsets: {org.apache.spark.sql.execution.streaming.TextSocketSource@750267f5: 100} Current State: ACTIVE Thread State: RUNNABLE Logical Plan: org.apache.spark.sql.execution.streaming.TextSocketSource@750267f5 at org.apache.spark.sql.execution.streaming.StreamExecution.org$apache$spark$sql$execution$streaming$StreamExecution$$runBatches(StreamExecution.scala:284) at org.apache.spark.sql.execution.streaming.StreamExecution$$anon$1.run(StreamExecution.scala:177) Caused by: java.lang.RuntimeException: Offsets committed out of order: 100999 followed by 100 at scala.sys.package$.error(package.scala:27) at org.apache.spark.sql.execution.streaming.TextSocketSource.commit(socket.scala:151) at org.apache.spark.sql.execution.streaming.StreamExecution$$anonfun$org$apache$spark$sql$execution$streaming$StreamExecution$$constructNextBatch$2$$anonfun$apply$mcV$sp$4.apply(StreamExecution.scala:421) at org.apache.spark.sql.execution.streaming.StreamExecution$$anonfun$org$apache$spark$sql$execution$streaming$StreamExecution$$constructNextBatch$2$$anonfun$apply$mcV$sp$4.apply(StreamExecution.scala:420) at scala.collection.Iterator$class.foreach(Iterator.scala:893) at scala.collection.AbstractIterator.foreach(Iterator.scala:1336) at scala.collection.IterableLike$class.foreach(IterableLike.scala:72) at org.apache.spark.sql.execution.streaming.StreamProgress.foreach(StreamProgress.scala:25) at org.apache.spark.sql.execution.streaming.StreamExecution$$anonfun$org$apache$spark$sql$execution$streaming$StreamExecution$$constructNextBatch$2.apply$mcV$sp(StreamExecution.scala:420) at org.apache.spark.sql.execution.streaming.StreamExecution$$anonfun$org$apache$spark$sql$execution$streaming$StreamExecution$$constructNextBatch$2.apply(StreamExecution.scala:404) at org.apache.spark.sql.execution.streaming.StreamExecution$$anonfun$org$apache$spark$sql$execution$streaming$StreamExecution$$constructNextBatch$2.apply(StreamExecution.scala:404) at org.apache.spark.sql.execution.streaming.ProgressReporter$class.reportTimeTaken(ProgressReporter.scala:262) at org.apache.spark.sql.execution.streaming.StreamExecution.reportTimeTaken(StreamExecution.scala:46) at org.apache.spark.sql.execution.streaming.StreamExecution.org$apache$spark$sql$execution$streaming$StreamExecution$$constructNextBatch(StreamExecution.scala:404) at org.apache.spark.sql.execution.streaming.StreamExecution$$anonfun$org$apache$spark$sql$execution$streaming$StreamExecution$$runBatches$1$$anonfun$1.apply$mcV$sp(StreamExecution.scala:250) at org.apache.spark.sql.execution.streaming.StreamExecution$$anonfun$org$apache$spark$sql$execution$streaming$StreamExecution$$runBatches$1$$anonfun$1.apply(StreamExecution.scala:244) at org.apache.spark.sql.execution.streaming.StreamExecution$$anonfun$org$apache$spark$sql$execution$streaming$StreamExecution$$runBatches$1$$anonfun$1.apply(StreamExecution.scala:244) at org.apache.spark.sql.execution.streaming.ProgressReporter$class.reportTimeTaken(ProgressReporter.scala:262) at org.apache.spark.sql.execution.streaming.StreamExecution.reportTimeTaken(StreamExecution.scala:46) at org.apache.spark.sql.execution.streaming.StreamExecution$$anonfun$org$apache$spark$sql$execution$streaming$StreamExecution$$runBatches$1.apply$mcZ$sp(StreamExecution.scala:244) at org.apache.spark.sql.execution.streaming.ProcessingTimeExecutor.execute(TriggerExecutor.scala:43) at org.apache.spark.sql.execution.streaming.StreamExecution.org$apache$spark$sql$execution$streaming$StreamExecution$$runBatches(StreamExecution.scala:239) ... 1 more Done reading and writing streaming data *Expected Output : select query should be successful from beeline on the streaming table.* > (Carbon1.3.0 - Streaming) Data load in Stream Segment fails if batch load is performed in between the streaming > --------------------------------------------------------------------------------------------------------------- > > Key: CARBONDATA-1790 > URL: https://issues.apache.org/jira/browse/CARBONDATA-1790 > Project: CarbonData > Issue Type: Bug > Components: data-query > Affects Versions: 1.3.0 > Environment: 3 node ant cluster > Reporter: Ramakrishna S > Labels: DFX > > Steps : > 1. Create a streaming table and do a batch load > 2. Set up the Streaming , so that it does streaming in chunk of 1000 records 20 times > 3. Do another batch load on the table > 4. Do one more time streaming > +-------------+------------+--------------------------+--------------------------+--------------+------------+--+ > | Segment Id | Status | Load Start Time | Load End Time | File Format | Merged To | > +-------------+------------+--------------------------+--------------------------+--------------+------------+--+ > | 2 | Success | 2017-11-21 21:42:36.77 | 2017-11-21 21:42:40.396 | COLUMNAR_V3 | NA | > | 1 | Streaming | 2017-11-21 21:40:46.2 | NULL | ROW_V1 | NA | > | 0 | Success | 2017-11-21 21:40:39.782 | 2017-11-21 21:40:43.168 | COLUMNAR_V3 | NA | > +-------------+------------+--------------------------+--------------------------+--------------+------------+--+ > *+Expected:+* Data should be loaded > *+Actual+* : Data load fiails > 1. One addition offset file is created(marked in bold) > -rw-r--r-- 2 root users 62 2017-11-21 21:40 /user/hive/warehouse/Ram/default/stream_table5/.streaming/checkpoint/offsets/0 > -rw-r--r-- 2 root users 63 2017-11-21 21:40 /user/hive/warehouse/Ram/default/stream_table5/.streaming/checkpoint/offsets/1 > -rw-r--r-- 2 root users 63 2017-11-21 21:42 /user/hive/warehouse/Ram/default/stream_table5/.streaming/checkpoint/offsets/10 > -rw-r--r-- 2 root users 63 2017-11-21 21:40 /user/hive/warehouse/Ram/default/stream_table5/.streaming/checkpoint/offsets/2 > -rw-r--r-- 2 root users 63 2017-11-21 21:41 /user/hive/warehouse/Ram/default/stream_table5/.streaming/checkpoint/offsets/3 > -rw-r--r-- 2 root users 64 2017-11-21 21:41 /user/hive/warehouse/Ram/default/stream_table5/.streaming/checkpoint/offsets/4 > -rw-r--r-- 2 root users 64 2017-11-21 21:41 /user/hive/warehouse/Ram/default/stream_table5/.streaming/checkpoint/offsets/5 > -rw-r--r-- 2 root users 64 2017-11-21 21:41 /user/hive/warehouse/Ram/default/stream_table5/.streaming/checkpoint/offsets/6 > -rw-r--r-- 2 root users 64 2017-11-21 21:41 /user/hive/warehouse/Ram/default/stream_table5/.streaming/checkpoint/offsets/7 > -rw-r--r-- 2 root users 64 2017-11-21 21:41 /user/hive/warehouse/Ram/default/stream_table5/.streaming/checkpoint/offsets/8 > *-rw-r--r-- 2 root users 63 2017-11-21 21:42 /user/hive/warehouse/Ram/default/stream_table5/.streaming/checkpoint/offsets/9* > 2. Following error thrown: > === Streaming Query === > Identifier: [id = 3a5334bc-d471-4676-b6ce-f21105d491d1, runId = b2be9f97-8141-46be-89db-9a0f98d13369] > Current Offsets: {org.apache.spark.sql.execution.streaming.TextSocketSource@14c45193: 1000} > Current State: ACTIVE > Thread State: RUNNABLE > Logical Plan: > org.apache.spark.sql.execution.streaming.TextSocketSource@14c45193 > at org.apache.spark.sql.execution.streaming.StreamExecution.org$apache$spark$sql$execution$streaming$StreamExecution$$runBatches(StreamExecution.scala:284) > at org.apache.spark.sql.execution.streaming.StreamExecution$$anon$1.run(StreamExecution.scala:177) > Caused by: java.lang.RuntimeException: Offsets committed out of order: 20019 followed by 1000 > at scala.sys.package$.error(package.scala:27) > at org.apache.spark.sql.execution.streaming.TextSocketSource.commit(socket.scala:151) > at org.apache.spark.sql.execution.streaming.StreamExecution$$anonfun$org$apache$spark$sql$execution$streaming$StreamExecution$$constructNextBatch$2$$anonfun$apply$mcV$sp$4.apply(StreamExecution.scala:421) > at org.apache.spark.sql.execution.streaming.StreamExecution$$anonfun$org$apache$spark$sql$execution$streaming$StreamExecution$$constructNextBatch$2$$anonfun$apply$mcV$sp$4.apply(StreamExecution.scala:420) > at scala.collection.Iterator$class.foreach(Iterator.scala:893) > at scala.collection.AbstractIterator.foreach(Iterator.scala:1336) > at scala.collection.IterableLike$class.foreach(IterableLike.scala:72) > at org.apache.spark.sql.execution.streaming.StreamProgress.foreach(StreamProgress.scala:25) > at org.apache.spark.sql.execution.streaming.StreamExecution$$anonfun$org$apache$spark$sql$execution$streaming$StreamExecution$$constructNextBatch$2.apply$mcV$sp(StreamExecution.scala:420) > at org.apache.spark.sql.execution.streaming.StreamExecution$$anonfun$org$apache$spark$sql$execution$streaming$StreamExecution$$constructNextBatch$2.apply(StreamExecution.scala:404) > at org.apache.spark.sql.execution.streaming.StreamExecution$$anonfun$org$apache$spark$sql$execution$streaming$StreamExecution$$constructNextBatch$2.apply(StreamExecution.scala:404) > at org.apache.spark.sql.execution.streaming.ProgressReporter$class.reportTimeTaken(ProgressReporter.scala:262) > at org.apache.spark.sql.execution.streaming.StreamExecution.reportTimeTaken(StreamExecution.scala:46) > at org.apache.spark.sql.execution.streaming.StreamExecution.org$apache$spark$sql$execution$streaming$StreamExecution$$constructNextBatch(StreamExecution.scala:404) > at org.apache.spark.sql.execution.streaming.StreamExecution$$anonfun$org$apache$spark$sql$execution$streaming$StreamExecution$$runBatches$1$$anonfun$1.apply$mcV$sp(StreamExecution.scala:250) > at org.apache.spark.sql.execution.streaming.StreamExecution$$anonfun$org$apache$spark$sql$execution$streaming$StreamExecution$$runBatches$1$$anonfun$1.apply(StreamExecution.scala:244) > at org.apache.spark.sql.execution.streaming.StreamExecution$$anonfun$org$apache$spark$sql$execution$streaming$StreamExecution$$runBatches$1$$anonfun$1.apply(StreamExecution.scala:244) > at org.apache.spark.sql.execution.streaming.ProgressReporter$class.reportTimeTaken(ProgressReporter.scala:262) > at org.apache.spark.sql.execution.streaming.StreamExecution.reportTimeTaken(StreamExecution.scala:46) > at org.apache.spark.sql.execution.streaming.StreamExecution$$anonfun$org$apache$spark$sql$execution$streaming$StreamExecution$$runBatches$1.apply$mcZ$sp(StreamExecution.scala:244) > at org.apache.spark.sql.execution.streaming.ProcessingTimeExecutor.execute(TriggerExecutor.scala:43) > at org.apache.spark.sql.execution.streaming.StreamExecution.org$apache$spark$sql$execution$streaming$StreamExecution$$runBatches(StreamExecution.scala:239) > ... 1 more > Done reading and writing streaming data > Socket closed -- This message was sent by Atlassian JIRA (v6.4.14#64029) |
Free forum by Nabble | Edit this page |