You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@carbondata.apache.org by "Ramakrishna S (JIRA)" <ji...@apache.org> on 2017/11/21 13:52:00 UTC

[jira] [Updated] (CARBONDATA-1790) (Carbon1.3.0 - Streaming) Data load in Stream Segment fails if batch load is performed in between the streaming

     [ 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)