Repository: spark
Updated Branches:
  refs/heads/branch-2.1 e43209fe2 -> fcd22e538


[SPARK-18776][SS] Make Offset for FileStreamSource corrected formatted in json

## What changes were proposed in this pull request?

- Changed FileStreamSource to use new FileStreamSourceOffset rather than 
LongOffset. The field is named as `logOffset` to make it more clear that this 
is a offset in the file stream log.
- Fixed bug in FileStreamSourceLog, the field endId in the 
FileStreamSourceLog.get(startId, endId) was not being used at all. No test 
caught it earlier. Only my updated tests caught it.

Other minor changes
- Dont use batchId in the FileStreamSource, as calling it batch id is extremely 
miss leading. With multiple sources, it may happen that a new batch has no new 
data from a file source. So offset of FileStreamSource != batchId after that 
batch.

## How was this patch tested?

Updated unit test.

Author: Tathagata Das <[email protected]>

Closes #16205 from tdas/SPARK-18776.

(cherry picked from commit 458fa3325e5f8c21c50e406ac8059d6236f93a9c)
Signed-off-by: Tathagata Das <[email protected]>


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/fcd22e53
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/fcd22e53
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/fcd22e53

Branch: refs/heads/branch-2.1
Commit: fcd22e5389a7dffda32be0e143d772f611a0f3d9
Parents: e43209f
Author: Tathagata Das <[email protected]>
Authored: Thu Dec 8 17:53:34 2016 -0800
Committer: Tathagata Das <[email protected]>
Committed: Thu Dec 8 17:53:45 2016 -0800

----------------------------------------------------------------------
 .../sql/kafka010/KafkaSourceOffsetSuite.scala   |  2 +-
 .../execution/streaming/FileStreamSource.scala  | 32 ++++++------
 .../streaming/FileStreamSourceLog.scala         |  2 +-
 .../streaming/FileStreamSourceOffset.scala      | 53 ++++++++++++++++++++
 .../file-source-offset-version-2.1.0-json.txt   |  1 +
 .../file-source-offset-version-2.1.0-long.txt   |  1 +
 .../file-source-offset-version-2.1.0.txt        |  1 -
 .../offset-log-version-2.1.0/0                  |  4 +-
 .../streaming/FileStreamSourceSuite.scala       |  2 +-
 .../execution/streaming/OffsetSeqLogSuite.scala |  2 +-
 .../sql/streaming/FileStreamSourceSuite.scala   | 30 +++++++----
 11 files changed, 96 insertions(+), 34 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/fcd22e53/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceOffsetSuite.scala
----------------------------------------------------------------------
diff --git 
a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceOffsetSuite.scala
 
b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceOffsetSuite.scala
index 22668fd..10b35c7 100644
--- 
a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceOffsetSuite.scala
+++ 
b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceOffsetSuite.scala
@@ -90,7 +90,7 @@ class KafkaSourceOffsetSuite extends OffsetSuite with 
SharedSQLContext {
     }
   }
 
-  test("read Spark 2.1.0 log format") {
+  test("read Spark 2.1.0 offset format") {
     val offset = readFromResource("kafka-source-offset-version-2.1.0.txt")
     assert(KafkaSourceOffset(offset) ===
       KafkaSourceOffset(("topic1", 0, 456L), ("topic1", 1, 789L), ("topic2", 
0, 0L)))

http://git-wip-us.apache.org/repos/asf/spark/blob/fcd22e53/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala
----------------------------------------------------------------------
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala
index 8494aef..20e0dce 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala
@@ -57,7 +57,7 @@ class FileStreamSource(
 
   private val metadataLog =
     new FileStreamSourceLog(FileStreamSourceLog.VERSION, sparkSession, 
metadataPath)
-  private var maxBatchId = metadataLog.getLatest().map(_._1).getOrElse(-1L)
+  private var metadataLogCurrentOffset = 
metadataLog.getLatest().map(_._1).getOrElse(-1L)
 
   /** Maximum number of new files to be considered in each batch */
   private val maxFilesPerBatch = sourceOptions.maxFilesPerTrigger
@@ -79,7 +79,7 @@ class FileStreamSource(
    * `synchronized` on this method is for solving race conditions in tests. In 
the normal usage,
    * there is no race here, so the cost of `synchronized` should be rare.
    */
-  private def fetchMaxOffset(): LongOffset = synchronized {
+  private def fetchMaxOffset(): FileStreamSourceOffset = synchronized {
     // All the new files found - ignore aged files and files that we have seen.
     val newFiles = fetchAllFiles().filter {
       case (path, timestamp) => seenFiles.isNewFile(path, timestamp)
@@ -104,14 +104,14 @@ class FileStreamSource(
        """.stripMargin)
 
     if (batchFiles.nonEmpty) {
-      maxBatchId += 1
-      metadataLog.add(maxBatchId, batchFiles.map { case (path, timestamp) =>
-        FileEntry(path = path, timestamp = timestamp, batchId = maxBatchId)
+      metadataLogCurrentOffset += 1
+      metadataLog.add(metadataLogCurrentOffset, batchFiles.map { case (p, 
timestamp) =>
+        FileEntry(path = p, timestamp = timestamp, batchId = 
metadataLogCurrentOffset)
       }.toArray)
-      logInfo(s"Max batch id increased to $maxBatchId with ${batchFiles.size} 
new files")
+      logInfo(s"Log offset set to $metadataLogCurrentOffset with 
${batchFiles.size} new files")
     }
 
-    new LongOffset(maxBatchId)
+    FileStreamSourceOffset(metadataLogCurrentOffset)
   }
 
   /**
@@ -122,21 +122,19 @@ class FileStreamSource(
     func
   }
 
-  /** Return the latest offset in the source */
-  def currentOffset: LongOffset = synchronized {
-    new LongOffset(maxBatchId)
-  }
+  /** Return the latest offset in the [[FileStreamSourceLog]] */
+  def currentLogOffset: Long = synchronized { metadataLogCurrentOffset }
 
   /**
    * Returns the data that is between the offsets (`start`, `end`].
    */
   override def getBatch(start: Option[Offset], end: Offset): DataFrame = {
-    val startId = 
start.flatMap(LongOffset.convert(_)).getOrElse(LongOffset(-1L)).offset
-    val endId = LongOffset.convert(end).getOrElse(LongOffset(0)).offset
+    val startOffset = 
start.map(FileStreamSourceOffset(_).logOffset).getOrElse(-1L)
+    val endOffset = FileStreamSourceOffset(end).logOffset
 
-    assert(startId <= endId)
-    val files = metadataLog.get(Some(startId + 1), Some(endId)).flatMap(_._2)
-    logInfo(s"Processing ${files.length} files from ${startId + 1}:$endId")
+    assert(startOffset <= endOffset)
+    val files = metadataLog.get(Some(startOffset + 1), 
Some(endOffset)).flatMap(_._2)
+    logInfo(s"Processing ${files.length} files from ${startOffset + 
1}:$endOffset")
     logTrace(s"Files are:\n\t" + files.mkString("\n\t"))
     val newDataSource =
       DataSource(
@@ -172,7 +170,7 @@ class FileStreamSource(
     files
   }
 
-  override def getOffset: Option[Offset] = 
Some(fetchMaxOffset()).filterNot(_.offset == -1)
+  override def getOffset: Option[Offset] = 
Some(fetchMaxOffset()).filterNot(_.logOffset == -1)
 
   override def toString: String = s"FileStreamSource[$qualifiedBasePath]"
 

http://git-wip-us.apache.org/repos/asf/spark/blob/fcd22e53/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSourceLog.scala
----------------------------------------------------------------------
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSourceLog.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSourceLog.scala
index 327b3ac..81908c0 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSourceLog.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSourceLog.scala
@@ -78,7 +78,7 @@ class FileStreamSourceLog(
 
   override def get(startId: Option[Long], endId: Option[Long]): Array[(Long, 
Array[FileEntry])] = {
     val startBatchId = startId.getOrElse(0L)
-    val endBatchId = getLatest().map(_._1).getOrElse(0L)
+    val endBatchId = endId.orElse(getLatest().map(_._1)).getOrElse(0L)
 
     val (existedBatches, removedBatches) = (startBatchId to endBatchId).map { 
id =>
       if (isCompactionBatch(id, compactInterval) && 
fileEntryCache.containsKey(id)) {

http://git-wip-us.apache.org/repos/asf/spark/blob/fcd22e53/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSourceOffset.scala
----------------------------------------------------------------------
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSourceOffset.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSourceOffset.scala
new file mode 100644
index 0000000..06d0fe6
--- /dev/null
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSourceOffset.scala
@@ -0,0 +1,53 @@
+/*
+ * 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.execution.streaming
+
+import scala.util.control.Exception._
+
+import org.json4s.NoTypeHints
+import org.json4s.jackson.Serialization
+
+/**
+ * Offset for the [[FileStreamSource]].
+ * @param logOffset  Position in the [[FileStreamSourceLog]]
+ */
+case class FileStreamSourceOffset(logOffset: Long) extends Offset {
+  override def json: String = {
+    Serialization.write(this)(FileStreamSourceOffset.format)
+  }
+}
+
+object FileStreamSourceOffset {
+  implicit val format = Serialization.formats(NoTypeHints)
+
+  def apply(offset: Offset): FileStreamSourceOffset = {
+    offset match {
+      case f: FileStreamSourceOffset => f
+      case SerializedOffset(str) =>
+        catching(classOf[NumberFormatException]).opt {
+          FileStreamSourceOffset(str.toLong)
+        }.getOrElse {
+          Serialization.read[FileStreamSourceOffset](str)
+        }
+      case _ =>
+        throw new IllegalArgumentException(
+          s"Invalid conversion from offset of ${offset.getClass} to 
FileStreamSourceOffset")
+    }
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/spark/blob/fcd22e53/sql/core/src/test/resources/structured-streaming/file-source-offset-version-2.1.0-json.txt
----------------------------------------------------------------------
diff --git 
a/sql/core/src/test/resources/structured-streaming/file-source-offset-version-2.1.0-json.txt
 
b/sql/core/src/test/resources/structured-streaming/file-source-offset-version-2.1.0-json.txt
new file mode 100644
index 0000000..e266a47
--- /dev/null
+++ 
b/sql/core/src/test/resources/structured-streaming/file-source-offset-version-2.1.0-json.txt
@@ -0,0 +1 @@
+{"logOffset":345}

http://git-wip-us.apache.org/repos/asf/spark/blob/fcd22e53/sql/core/src/test/resources/structured-streaming/file-source-offset-version-2.1.0-long.txt
----------------------------------------------------------------------
diff --git 
a/sql/core/src/test/resources/structured-streaming/file-source-offset-version-2.1.0-long.txt
 
b/sql/core/src/test/resources/structured-streaming/file-source-offset-version-2.1.0-long.txt
new file mode 100644
index 0000000..51b4008
--- /dev/null
+++ 
b/sql/core/src/test/resources/structured-streaming/file-source-offset-version-2.1.0-long.txt
@@ -0,0 +1 @@
+345

http://git-wip-us.apache.org/repos/asf/spark/blob/fcd22e53/sql/core/src/test/resources/structured-streaming/file-source-offset-version-2.1.0.txt
----------------------------------------------------------------------
diff --git 
a/sql/core/src/test/resources/structured-streaming/file-source-offset-version-2.1.0.txt
 
b/sql/core/src/test/resources/structured-streaming/file-source-offset-version-2.1.0.txt
deleted file mode 100644
index 51b4008..0000000
--- 
a/sql/core/src/test/resources/structured-streaming/file-source-offset-version-2.1.0.txt
+++ /dev/null
@@ -1 +0,0 @@
-345

http://git-wip-us.apache.org/repos/asf/spark/blob/fcd22e53/sql/core/src/test/resources/structured-streaming/offset-log-version-2.1.0/0
----------------------------------------------------------------------
diff --git 
a/sql/core/src/test/resources/structured-streaming/offset-log-version-2.1.0/0 
b/sql/core/src/test/resources/structured-streaming/offset-log-version-2.1.0/0
index fe5c1d4..988a98a 100644
--- 
a/sql/core/src/test/resources/structured-streaming/offset-log-version-2.1.0/0
+++ 
b/sql/core/src/test/resources/structured-streaming/offset-log-version-2.1.0/0
@@ -1,4 +1,4 @@
 v1
 {"batchWatermarkMs":0,"batchTimestampMs":1480981499528}
-0
-{"topic-0":{"0":1}}
\ No newline at end of file
+{"logOffset":345}
+{"topic-0":{"0":1}}

http://git-wip-us.apache.org/repos/asf/spark/blob/fcd22e53/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/FileStreamSourceSuite.scala
----------------------------------------------------------------------
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/FileStreamSourceSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/FileStreamSourceSuite.scala
index 4a47c04..40d0643 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/FileStreamSourceSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/FileStreamSourceSuite.scala
@@ -97,7 +97,7 @@ class FileStreamSourceSuite extends SparkFunSuite with 
SharedSQLContext {
       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, LongOffset(1))
+      newSource.getBatch(None, FileStreamSourceOffset(1))
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/fcd22e53/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/OffsetSeqLogSuite.scala
----------------------------------------------------------------------
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/OffsetSeqLogSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/OffsetSeqLogSuite.scala
index d139efa..bb4274a 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/OffsetSeqLogSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/OffsetSeqLogSuite.scala
@@ -74,7 +74,7 @@ class OffsetSeqLogSuite extends SparkFunSuite with 
SharedSQLContext {
     val (batchId, offsetSeq) = readFromResource("offset-log-version-2.1.0")
     assert(batchId === 0)
     assert(offsetSeq.offsets === Seq(
-      Some(SerializedOffset("0")),
+      Some(SerializedOffset("""{"logOffset":345}""")),
       Some(SerializedOffset("""{"topic-0":{"0":1}}"""))
     ))
     assert(offsetSeq.metadata === Some(OffsetSeqMetadata(0L, 1480981499528L)))

http://git-wip-us.apache.org/repos/asf/spark/blob/fcd22e53/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala
----------------------------------------------------------------------
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala
index 267c462..bcb6852 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala
@@ -61,7 +61,7 @@ abstract class FileStreamSourceTest
       val source = sources.head
       val newOffset = source.withBatchingLocked {
         addData(source)
-        source.currentOffset + 1
+        new FileStreamSourceOffset(source.currentLogOffset + 1)
       }
       logInfo(s"Added file to $source at offset $newOffset")
       (source, newOffset)
@@ -987,12 +987,17 @@ class FileStreamSourceSuite extends FileStreamSourceTest {
             val _sources = PrivateMethod[Seq[Source]]('sources)
             val fileSource =
               (execution invokePrivate 
_sources()).head.asInstanceOf[FileStreamSource]
-            assert(fileSource.getBatch(None, 
LongOffset(2)).as[String].collect() ===
-              List("keep1", "keep2", "keep3"))
-            assert(fileSource.getBatch(Some(LongOffset(0)), 
LongOffset(2)).as[String].collect() ===
-              List("keep2", "keep3"))
-            assert(fileSource.getBatch(Some(LongOffset(1)), 
LongOffset(2)).as[String].collect() ===
-              List("keep3"))
+
+            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
           }
         )
@@ -1023,9 +1028,14 @@ class FileStreamSourceSuite extends FileStreamSourceTest 
{
     assert(options.maxFilesPerTrigger == Some(1))
   }
 
-  test("FileStreamSource offset - read Spark 2.1.0 log format") {
-    val offset = readOffsetFromResource("file-source-offset-version-2.1.0.txt")
-    assert(LongOffset.convert(offset) === Some(LongOffset(345)))
+  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") {


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to