Skip to content

Conversation

@frreiss
Copy link
Contributor

@frreiss frreiss commented Aug 9, 2016

What changes were proposed in this pull request?

This PR contains changes to the Source trait such that the scheduler can notify data sources when it is safe to discard buffered data. Summary of changes:

  • Added a method commit(end: Offset) that tells the Source that is OK to discard all offsets up end, inclusive.
  • Changed the semantics of a None value for the getBatch method to mean "from the very beginning of the stream"; as opposed to "all data present in the Source's buffer".
  • Added notes that the upper layers of the system will never call getBatch with a start value less than the last value passed to commit.
  • Added a lastCommittedOffset method to allow the scheduler to query the status of each Source on restart. This addition is not strictly necessary, but it seemed like a good idea -- Sources will be maintaining their own persistent state, and there may be bugs in the checkpointing code.
  • The scheduler in StreamExecution.scala now calls commit on its stream sources after marking each batch as complete in its checkpoint.
  • MemoryStream now cleans committed batches out of its internal buffer.
  • TextSocketSource now cleans committed batches from its internal buffer.

How was this patch tested?

Existing regression tests already exercise the new code.

@frreiss
Copy link
Contributor Author

frreiss commented Aug 22, 2016

These changes are now ready for review. The contents of this PR pass regression tests on my machines. Can one of the committers please start a Jenkins build?

@frreiss frreiss changed the title [WIP] [SPARK-16963] Initial version of changes to Source trait [SPARK-16963] Changes to Source trait and related implementation classes Aug 22, 2016
*/
def getLatest(): Option[(Long, T)]


Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Extra blank line

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Fixed in my local copy.

@frreiss
Copy link
Contributor Author

frreiss commented Aug 29, 2016

@rxin and @marmbrus, would it be possible to get this PR reviewed soon? I can split it into smaller chunks if that would make things easier; I just need to know.

@frreiss
Copy link
Contributor Author

frreiss commented Aug 31, 2016

@ScrapCodes, would you mind triggering a build of this PR?

@ScrapCodes
Copy link
Member

ok to test

@ScrapCodes
Copy link
Member

retest this please

@ScrapCodes
Copy link
Member

I have tested the PR with my MQTT connector. Looks like I do not have sufficient privilege to command jenkins.

@vanzin
Copy link
Contributor

vanzin commented Sep 7, 2016

ok to test

@SparkQA
Copy link

SparkQA commented Sep 8, 2016

Test build #65062 has finished for PR 14553 at commit 7c6a30d.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds the following public classes (experimental):
    • trait Source

Copy link
Member

@zsxwing zsxwing left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM except one nit.

offsetLog.purge(currentBatchId)
// the batch before the previous batch, and it is safe to discard the old metadata.
// Note that purge is exclusive, i.e. it purges everything before the target ID.
offsetLog.purge(currentBatchId - 1)
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: this can be offsetLog.purge(currentBatchId), it's exclusive, then you can revert changes to StreamingQuerySuite.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I can move this change to another JIRA if you'd like, but we really should change currentBatchId to currentBatchId - 1 at some point. The call to offsetLog.purge(currentBatchId), which I introduced in my PR for SPARK-17513, contains a subtle bug. The recovery logic in populateStartOffsets() reads the last and second-to-last entries in offsetLog. populateStartOffsets() uses those entries to populate availableOffsets and committedOffsets, respectively. Calling offsetLog.purge(currentBatchId) at line 350/366 results in the offsetLog being truncated to one entry, which in turn results in committedOffsets being left empty on recovery, which in turn causes the first call to getBatch() for any source to have None as its first argument. Sources that do not prune buffered data in their commit() methods will return a previously committed data in response to such a getBatch() call.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I see. Thanks for your clarifying.

@SparkQA
Copy link

SparkQA commented Oct 21, 2016

Test build #67350 has finished for PR 14553 at commit 47eee52.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@zsxwing
Copy link
Member

zsxwing commented Oct 24, 2016

retest this please

Copy link
Member

@zsxwing zsxwing left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM pending tests

@SparkQA
Copy link

SparkQA commented Oct 24, 2016

Test build #67463 has finished for PR 14553 at commit 47eee52.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@zsxwing
Copy link
Member

zsxwing commented Oct 24, 2016

@frreiss you need to reset lastOffsetCommitted in MemoryStream.reset. That's why the test fails.

}

override def commit(end: Offset): Unit = synchronized {
if (end.isInstanceOf[LongOffset]) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit:

end match {
  case newOffset: LongOffset =>
     ...
  case _ => sys.error(...)
}

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Corrected in my local copy.

lastOffsetCommitted = newOffset
} else {
sys.error(s"MemoryStream.commit() received an offset ($end) that did not originate with " +
s"an instance of this class")
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: s unnecessary.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Corrected in my local copy.

@brkyvz
Copy link
Contributor

brkyvz commented Oct 25, 2016

LGTM as well!

@zsxwing
Copy link
Member

zsxwing commented Oct 26, 2016

@frreiss any update?

@frreiss
Copy link
Contributor Author

frreiss commented Oct 26, 2016

Updated the branch and addressed new review comments. Looks like my last push missed a one-line change to memory.scala. Tests are running now.

@SparkQA
Copy link

SparkQA commented Oct 27, 2016

Test build #67603 has finished for PR 14553 at commit 0a56e4a.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@zsxwing
Copy link
Member

zsxwing commented Oct 27, 2016

LGMT. Merging to master and 2.0. Thanks!

asfgit pushed a commit that referenced this pull request Oct 27, 2016
…lementation classes

## What changes were proposed in this pull request?

This PR contains changes to the Source trait such that the scheduler can notify data sources when it is safe to discard buffered data. Summary of changes:
* Added a method `commit(end: Offset)` that tells the Source that is OK to discard all offsets up `end`, inclusive.
* Changed the semantics of a `None` value for the `getBatch` method to mean "from the very beginning of the stream"; as opposed to "all data present in the Source's buffer".
* Added notes that the upper layers of the system will never call `getBatch` with a start value less than the last value passed to `commit`.
* Added a `lastCommittedOffset` method to allow the scheduler to query the status of each Source on restart. This addition is not strictly necessary, but it seemed like a good idea -- Sources will be maintaining their own persistent state, and there may be bugs in the checkpointing code.
* The scheduler in `StreamExecution.scala` now calls `commit` on its stream sources after marking each batch as complete in its checkpoint.
* `MemoryStream` now cleans committed batches out of its internal buffer.
* `TextSocketSource` now cleans committed batches from its internal buffer.

## How was this patch tested?
Existing regression tests already exercise the new code.

Author: frreiss <[email protected]>

Closes #14553 from frreiss/fred-16963.

(cherry picked from commit 5b27598)
Signed-off-by: Shixiong Zhu <[email protected]>
@asfgit asfgit closed this in 5b27598 Oct 27, 2016
asfgit pushed a commit that referenced this pull request Oct 27, 2016
…ion"

## What changes were proposed in this pull request?

A follow up PR for #14553 to fix the flaky test. It's flaky because the file list API doesn't guarantee any order of the return list.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <[email protected]>

Closes #15661 from zsxwing/fix-StreamingQuerySuite.
asfgit pushed a commit that referenced this pull request Oct 27, 2016
…ion"

## What changes were proposed in this pull request?

A follow up PR for #14553 to fix the flaky test. It's flaky because the file list API doesn't guarantee any order of the return list.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <[email protected]>

Closes #15661 from zsxwing/fix-StreamingQuerySuite.

(cherry picked from commit 79fd0cc)
Signed-off-by: Shixiong Zhu <[email protected]>
robert3005 pushed a commit to palantir/spark that referenced this pull request Nov 1, 2016
…lementation classes

## What changes were proposed in this pull request?

This PR contains changes to the Source trait such that the scheduler can notify data sources when it is safe to discard buffered data. Summary of changes:
* Added a method `commit(end: Offset)` that tells the Source that is OK to discard all offsets up `end`, inclusive.
* Changed the semantics of a `None` value for the `getBatch` method to mean "from the very beginning of the stream"; as opposed to "all data present in the Source's buffer".
* Added notes that the upper layers of the system will never call `getBatch` with a start value less than the last value passed to `commit`.
* Added a `lastCommittedOffset` method to allow the scheduler to query the status of each Source on restart. This addition is not strictly necessary, but it seemed like a good idea -- Sources will be maintaining their own persistent state, and there may be bugs in the checkpointing code.
* The scheduler in `StreamExecution.scala` now calls `commit` on its stream sources after marking each batch as complete in its checkpoint.
* `MemoryStream` now cleans committed batches out of its internal buffer.
* `TextSocketSource` now cleans committed batches from its internal buffer.

## How was this patch tested?
Existing regression tests already exercise the new code.

Author: frreiss <[email protected]>

Closes apache#14553 from frreiss/fred-16963.
robert3005 pushed a commit to palantir/spark that referenced this pull request Nov 1, 2016
…ion"

## What changes were proposed in this pull request?

A follow up PR for apache#14553 to fix the flaky test. It's flaky because the file list API doesn't guarantee any order of the return list.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <[email protected]>

Closes apache#15661 from zsxwing/fix-StreamingQuerySuite.
uzadude pushed a commit to uzadude/spark that referenced this pull request Jan 27, 2017
…lementation classes

## What changes were proposed in this pull request?

This PR contains changes to the Source trait such that the scheduler can notify data sources when it is safe to discard buffered data. Summary of changes:
* Added a method `commit(end: Offset)` that tells the Source that is OK to discard all offsets up `end`, inclusive.
* Changed the semantics of a `None` value for the `getBatch` method to mean "from the very beginning of the stream"; as opposed to "all data present in the Source's buffer".
* Added notes that the upper layers of the system will never call `getBatch` with a start value less than the last value passed to `commit`.
* Added a `lastCommittedOffset` method to allow the scheduler to query the status of each Source on restart. This addition is not strictly necessary, but it seemed like a good idea -- Sources will be maintaining their own persistent state, and there may be bugs in the checkpointing code.
* The scheduler in `StreamExecution.scala` now calls `commit` on its stream sources after marking each batch as complete in its checkpoint.
* `MemoryStream` now cleans committed batches out of its internal buffer.
* `TextSocketSource` now cleans committed batches from its internal buffer.

## How was this patch tested?
Existing regression tests already exercise the new code.

Author: frreiss <[email protected]>

Closes apache#14553 from frreiss/fred-16963.
uzadude pushed a commit to uzadude/spark that referenced this pull request Jan 27, 2017
…ion"

## What changes were proposed in this pull request?

A follow up PR for apache#14553 to fix the flaky test. It's flaky because the file list API doesn't guarantee any order of the return list.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <[email protected]>

Closes apache#15661 from zsxwing/fix-StreamingQuerySuite.
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

8 participants