Skip to content

Conversation

@andrewor14
Copy link
Contributor

Some users have reported difficulty in parsing the new event log format. Since we embed the metadata in the beginning of the file, when we compress the event log we need to skip the metadata because we need that information to parse the log later. This means we'll end up with a partially compressed file if event logging compression is turned on. The old format looks like:

sparkVersion = 1.3.0
compressionCodec = org.apache.spark.io.LZFCompressionCodec
=== LOG_HEADER_END ===
// actual events, could be compressed bytes

The new format in this patch puts the compression codec in the log file name instead. It reformats the metadata header into JSON and includes only the Spark version. This is the first line of every event log and looks like:

{"Event": "SparkListenerLogStart", "Spark Version": "1.3.0"} // could be compressed

and the new file name looks something like:

app_without_compression
app_123.lzf
app_456.snappy

I tested this with and without compression, using different compression codecs and event logging directories. I verified that both the Master and the HistoryServer can render both compressed and uncompressed logs as before.

This makes the event logs much easier to parse than before.
As of this commit the whole file is either entirely compressed
or not compressed, but not somewhere in between.
@andrewor14 andrewor14 changed the title [SPARK_6066] Make event log format easier to parse [SPARK-6066] Make event log format easier to parse Feb 27, 2015
@SparkQA
Copy link

SparkQA commented Feb 27, 2015

Test build #28098 has started for PR 4821 at commit 8db5a06.

  • This patch merges cleanly.

@andrewor14
Copy link
Contributor Author

Tests will fail to compile. I'm fixing this in the mean time.

Copy link
Contributor

Choose a reason for hiding this comment

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

I know this is not part of this change, but if feels like this whole method needs a try..finally to avoid leaking an open dstream when some exception is thrown.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

that's fine. I fixed this in my latest commit.

@vanzin
Copy link
Contributor

vanzin commented Feb 28, 2015

Looks reasonable, will wait for tests.

@SparkQA
Copy link

SparkQA commented Feb 28, 2015

Test build #28098 has finished for PR 4821 at commit 8db5a06.

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

@AmplabJenkins
Copy link

Test FAILed.
Refer to this link for build results (access rights to CI server needed):
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/28098/
Test FAILed.

@SparkQA
Copy link

SparkQA commented Feb 28, 2015

Test build #28102 has started for PR 4821 at commit ef69276.

  • This patch merges cleanly.

@SparkQA
Copy link

SparkQA commented Feb 28, 2015

Test build #28103 has started for PR 4821 at commit 519e51a.

  • This patch merges cleanly.

@SparkQA
Copy link

SparkQA commented Feb 28, 2015

Test build #28102 has finished for PR 4821 at commit ef69276.

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

@AmplabJenkins
Copy link

Test FAILed.
Refer to this link for build results (access rights to CI server needed):
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/28102/
Test FAILed.

@SparkQA
Copy link

SparkQA commented Feb 28, 2015

Test build #28103 has finished for PR 4821 at commit 519e51a.

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

@AmplabJenkins
Copy link

Test PASSed.
Refer to this link for build results (access rights to CI server needed):
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/28103/
Test PASSed.

Copy link
Contributor

Choose a reason for hiding this comment

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

I don't see where the EVENT_LOG prefix is actually applied here.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

you're right it's outdated

@pwendell
Copy link
Contributor

pwendell commented Mar 2, 2015

I took a pass on this with some feedback. Overall, it would be good to really minimize the scope of the changes since this is so late in the game. There is some clean-up and renaming etc that would be best just left out of the patch.

The main thing I'm wondering is why we need this header at all. It doesn't even ever get used by our own replay - we just ignore it. It seems like it was added for the purpose of conveying the compression codec to bootstrap replaying the file, however just having an extension seems like a better, much more standard way of doing that. The only argument I see for it is that the header could be used in the future to encode things that are necessary for proper replay of the logs. However, in that case I don't see why we can't just add it later if and when those things occur.

I guess I don't see a good argument against a straw man of just not having the header. Curious to hear thoughts from @andrewor14 and @vanzin.

Things changed in this commit:
(1) No more metadata in log content
(2) No more Spark version in log file name
(3) Use short name for compression codec in log file name
@SparkQA
Copy link

SparkQA commented Mar 2, 2015

Test build #28157 has started for PR 4821 at commit 40e1ff6.

  • This patch merges cleanly.

@andrewor14
Copy link
Contributor Author

As of the latest commit I have removed the Spark version and the metadata. Please have a look.

Copy link
Contributor

Choose a reason for hiding this comment

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

This doc might be better if it said "in Spark 1.2.X and earlier" instead of "in previous releases".

Copy link
Contributor

Choose a reason for hiding this comment

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

Actually I guess this handles multiple versions. My bad, fine to leave as is

@pwendell
Copy link
Contributor

pwendell commented Mar 2, 2015

@vanzin so just to be clear - you are anticipating a future case where we need to read the version to correctly parse the logs? Is that the argument for it? I am only making a simple observation that here we write something and never read it.

@vanzin
Copy link
Contributor

vanzin commented Mar 2, 2015

@pwendell yes, that's what I'm trying to say. Even today, logs generated by Spark 1.0, 1.1 and 1.2 (and soon 1.3) are not exactly the same - events have different properties. The metadata allows the parsing code to know what to expect.

(That is not a problem internally in Spark because the code handles all added/changed event properties using optionals and things like that, but there's no way to know whether we'll make a change in the future that prevents code like that from working without knowing the version of the data.)

@SparkQA
Copy link

SparkQA commented Mar 2, 2015

Test build #28180 has started for PR 4821 at commit 7f537cd.

  • This patch merges cleanly.

@SparkQA
Copy link

SparkQA commented Mar 2, 2015

Test build #28180 has finished for PR 4821 at commit 7f537cd.

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

@AmplabJenkins
Copy link

Test FAILed.
Refer to this link for build results (access rights to CI server needed):
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/28180/
Test FAILed.

@pwendell
Copy link
Contributor

pwendell commented Mar 2, 2015

@vanzin okay then, let's add back a JSON metadata header to the file.

@andrewor14
Copy link
Contributor Author

By the way I gained first-hand experience of the flakiness of FsHistoryProviderSuite from this patch. The tests passed locally but failed here, and I suspect that it has something to do with the modification time not varying enough between the file writes, though this is something we should fix separately.

@SparkQA
Copy link

SparkQA commented Mar 2, 2015

Test build #28182 has started for PR 4821 at commit 654883d.

  • This patch merges cleanly.

@vanzin
Copy link
Contributor

vanzin commented Mar 2, 2015

flakiness of FsHistoryProviderSuite

Do you have any theories for why this would be related to the log's modification time? The only place where it's used in FsHistoryProvider is for filtering entries in checkForLogs, and it's already a conservative check (it uses >= so that it picks up any new logs that show up with the same mod time as the last check, at the expense of re-parsing logs that haven't changed).

@SparkQA
Copy link

SparkQA commented Mar 2, 2015

Test build #28182 has finished for PR 4821 at commit 654883d.

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

@AmplabJenkins
Copy link

Test FAILed.
Refer to this link for build results (access rights to CI server needed):
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/28182/
Test FAILed.

Copy link
Contributor

Choose a reason for hiding this comment

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

I think your problem is that entries 0 and 1 have both the same start and end time, so their sort order is non-deterministic.

(Sorry for the edits. Diff is confusing to read.)

Copy link
Contributor Author

Choose a reason for hiding this comment

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

yeah, I just realized this independently. I fixed this in my latest commit.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

however that still doesn't explain why it passes locally but fails remotely.

Copy link
Contributor

Choose a reason for hiding this comment

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

It could be that you're using a different filesystem than then jenkins machine, and both return things in different order.

@SparkQA
Copy link

SparkQA commented Mar 2, 2015

Test build #28198 has started for PR 4821 at commit fdae14c.

  • This patch merges cleanly.

Copy link
Contributor

Choose a reason for hiding this comment

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

minor - but maybe call this sparkVersion so it's clear that this isn't a version for the logging format (since that doesn't have its own versioning).

Copy link
Contributor Author

Choose a reason for hiding this comment

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

(edit): misunderstood what you meant. I'll rename the variable NBD

@vanzin
Copy link
Contributor

vanzin commented Mar 2, 2015

LGTM.

@SparkQA
Copy link

SparkQA commented Mar 2, 2015

Test build #28201 has started for PR 4821 at commit 8511141.

  • This patch merges cleanly.

@pwendell
Copy link
Contributor

pwendell commented Mar 2, 2015

Okay I took a close look through this and it LGTM

@SparkQA
Copy link

SparkQA commented Mar 3, 2015

Test build #28198 has finished for PR 4821 at commit fdae14c.

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

@AmplabJenkins
Copy link

Test PASSed.
Refer to this link for build results (access rights to CI server needed):
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/28198/
Test PASSed.

asfgit pushed a commit that referenced this pull request Mar 3, 2015
Some users have reported difficulty in parsing the new event log format. Since we embed the metadata in the beginning of the file, when we compress the event log we need to skip the metadata because we need that information to parse the log later. This means we'll end up with a partially compressed file if event logging compression is turned on. The old format looks like:
```
sparkVersion = 1.3.0
compressionCodec = org.apache.spark.io.LZFCompressionCodec
=== LOG_HEADER_END ===
// actual events, could be compressed bytes
```
The new format in this patch puts the compression codec in the log file name instead. It also removes the metadata header altogether along with the Spark version, which was not needed. The new file name looks something like:
```
app_without_compression
app_123.lzf
app_456.snappy
```

I tested this with and without compression, using different compression codecs and event logging directories. I verified that both the `Master` and the `HistoryServer` can render both compressed and uncompressed logs as before.

Author: Andrew Or <andrew@databricks.com>

Closes #4821 from andrewor14/event-log-format and squashes the following commits:

8511141 [Andrew Or] Fix test
654883d [Andrew Or] Add back metadata with Spark version
7f537cd [Andrew Or] Address review feedback
7d6aa61 [Andrew Or] Make codec an extension
59abee9 [Andrew Or] Merge branch 'master' of github.com:apache/spark into event-log-format
27c9a6c [Andrew Or] Address review feedback
519e51a [Andrew Or] Address review feedback
ef69276 [Andrew Or] Merge branch 'master' of github.com:apache/spark into event-log-format
88a091d [Andrew Or] Add tests for new format and file name
f32d8d2 [Andrew Or] Fix tests
8db5a06 [Andrew Or] Embed metadata in the event log file name instead

(cherry picked from commit 6776cb3)
Signed-off-by: Patrick Wendell <patrick@databricks.com>
@asfgit asfgit closed this in 6776cb3 Mar 3, 2015
@andrewor14 andrewor14 deleted the event-log-format branch March 3, 2015 00:49
@SparkQA
Copy link

SparkQA commented Mar 3, 2015

Test build #28201 has finished for PR 4821 at commit 8511141.

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

@AmplabJenkins
Copy link

Test PASSed.
Refer to this link for build results (access rights to CI server needed):
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/28201/
Test PASSed.

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.

5 participants