Uploaded image for project: 'Spark'
  1. Spark
  2. SPARK-41053

Better Spark UI scalability and Driver stability for large applications

    XMLWordPrintableJSON

Details

    • Umbrella
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • 3.4.0
    • 3.4.0
    • Spark Core, Web UI

    Description

      After SPARK-18085, the Spark history server(SHS) becomes more scalable for processing large applications by supporting a persistent KV-store(LevelDB/RocksDB) as the storage layer.

      As for the live Spark UI, all the data is still stored in memory, which can bring memory pressures to the Spark driver for large applications.

      For better Spark UI scalability and Driver stability, I propose to

      • Support storing all the UI data in a persistent KV store. RocksDB/LevelDB provides low memory overhead. Their write/read performance is fast enough to serve the write/read workload for live UI. SHS can leverage the persistent KV store to fasten its startup.
      • Support a new Protobuf serializer for all the UI data. The new serializer is supposed to be faster, according to benchmarks. It will be the default serializer for the persistent KV store of live UI. As for event logs, it is optional. The current serializer for UI data is JSON. When writing persistent KV-store, there is GZip compression. Since there is compression support in RocksDB/LevelDB, the new serializer won’t compress the output before writing to the persistent KV store. Here is a benchmark of writing/reading 100,000 SQLExecutionUIData to/from RocksDB:

       

      Serializer Avg Write time(μs) Avg Read time(μs) RocksDB File Total Size(MB) Result total size in memory(MB)
      Spark’s KV Serializer(JSON+gzip) 352.2 119.26 837 868
      Protobuf 109.9 34.3 858 2105

      I am also proposing to support RocksDB instead of both LevelDB & RocksDB in the live UI.

      SPIP: https://docs.google.com/document/d/1cuKnFwlTodyVhUQPMuakq2YDaLH05jaY9FRu_aD1zMo/edit?usp=sharing

      SPIP vote: https://lists.apache.org/thread/lom4zcob6237q6nnj46jylkzwmmsxvgj

      Attachments

        1.
        Support disk-based KVStore in live UI Sub-task Resolved Gengliang Wang
        2.
        Introducing a Protobuf serializer for UI data on KV store Sub-task Resolved Gengliang Wang
        3.
        Protobuf serializer for ApplicationInfoWrapper Sub-task Resolved Sandeep Singh
        4.
        Protobuf serializer for ApplicationEnvironmentInfoWrapper Sub-task Resolved Sandeep Singh
        5.
        Protobuf serializer for ExecutorSummaryWrapper Sub-task Resolved Sandeep Singh
        6.
        Protobuf serializer for StageDataWrapper Sub-task Resolved BingKun Pan
        7.
        Protobuf serializer for TaskDataWrapper Sub-task Resolved Gengliang Wang
        8.
        Protobuf serializer for RDDStorageInfoWrapper Sub-task Resolved Sandeep Singh
        9.
        Protobuf serializer for ResourceProfileWrapper Sub-task Resolved Sandeep Singh
        10.
        Protobuf serializer for ExecutorStageSummaryWrapper Sub-task Resolved Gengliang Wang
        11.
        Protobuf serializer for SpeculationStageSummaryWrapper Sub-task Resolved Sandeep Singh
        12.
        Protobuf serializer for RDDOperationGraphWrapper Sub-task Resolved Sandeep Singh
        13.
        Protobuf serializer for ProcessSummaryWrapper Sub-task Resolved Sandeep Singh
        14.
        Protobuf serializer for SQLExecutionUIData Sub-task Resolved Yang Jie
        15.
        Protobuf serializer for SparkPlanGraphWrapper Sub-task Resolved Sandeep Singh
        16.
        protoc-3.21.9-linux-x86_64.exe requires GLIBC_2.14 Sub-task Resolved Haonan Jiang
        17.
        Unify the environment variable of *_PROTOC_EXEC_PATH Sub-task Resolved Unassigned
        18.
        Introducing SPI mechanism to make it easy for other modules to register ProtoBufSerializer Sub-task Resolved Yang Jie
        19.
        Protobuf serializer for StreamingQueryData Sub-task Resolved Yang Jie
        20.
        Protobuf serializer for StreamingQueryProgressWrapper Sub-task Resolved Yang Jie
        21.
        Protobuf serializer for LogInfo Sub-task Resolved Unassigned
        22.
        Protobuf serializer for StreamBlockData Sub-task Resolved Gengliang Wang
        23.
        Protobuf serializer for CachedQuantile Sub-task Resolved Gengliang Wang
        24.
        Protobuf serializer for ApplicationStoreInfo Sub-task Resolved Unassigned
        25.
        Support optional using Protobuf serializer for KVStore in History server Sub-task Resolved Gengliang Wang
        26.
        Add new config to clean up `spark.ui.store.path` directory when SparkContext.stop() Sub-task Resolved Yang Jie
        27.
        Explicitly define `Seq` as `collection.Seq` to reduce `toSeq` when create ui objects from protobuf objects for Scala 2.13 Sub-task Resolved Yang Jie
        28.
        Add simple developer guides for UI protobuf serializer Sub-task Resolved Gengliang Wang
        29.
        Use `weakIntern` on string values in create new objects during deserialization Sub-task Resolved BingKun Pan
        30.
        Refactor the definition of enum - `JobExecutionStatus` to follow with the code style Sub-task Resolved BingKun Pan
        31.
        Add tests for SQLAppStatusStore with RocksDB Backend Sub-task Resolved Yang Jie
        32.
        Reduce `toSeq` in `RDDOperationGraphWrapperSerializer`/SparkPlanGraphWrapperSerializer` for Scala 2.13 Sub-task Resolved Yang Jie
        33.
        Add tests for streaming UI with RocksDB backend Sub-task Resolved Gengliang Wang
        34.
        Add Github action test job with RocksDB as UI backend Sub-task Resolved Gengliang Wang
        35.
        Refactor ProtobufSerDe to ProtobufSerDe[T] Sub-task Resolved Yang Jie
        36.
        Protobuf serializer for AppSummary and PoolData Sub-task Resolved Yang Jie
        37.
        Protect null `SQLExecutionUIData#description` in `SQLExecutionUIDataSerializer` Sub-task Resolved Yang Jie
        38.
        Handle null string values in AccumulableInfo and ProcessSummary Sub-task Resolved Gengliang Wang
        39.
        Handle null string values in JobData/TaskDataWrapper/ExecutorStageSummaryWrapper Sub-task Resolved Gengliang Wang
        40.
        Handle null string values in SQLExecutionUIData/SQLPlanMetric/SparkPlanGraphWrapper Sub-task Resolved Yang Jie
        41.
        Handle null string values in ApplicationEnvironmentInfoWrapper/ApplicationInfoWrapper Sub-task Resolved Yang Jie
        42.
        Handle null string values in ApplicationInfo/ApplicationAttemptInfo Sub-task Resolved Unassigned
        43.
        Handle null string values in CachedQuantile/ExecutorSummary/PoolData Sub-task Resolved Gengliang Wang
        44.
        Handle null string values in RDDStorageInfo/RDDDataDistribution/RDDPartitionInfo Sub-task Resolved Gengliang Wang
        45.
        Handle null string values in StageData/StreamBlockData/StreamingQueryData Sub-task Resolved Yang Jie
        46.
        Handle null string values in SparkPlanGraphNode/SparkPlanGraphClusterWrapper Sub-task Resolved Unassigned
        47.
        Refactor `Utils#setStringField` to make maven build pass when sql module use this method Sub-task Resolved Yang Jie
        48.
        Handle null string values in PairStrings/RDDOperationNode/RDDOperationClusterWrapper Sub-task Resolved Gengliang Wang
        49.
        Handle remaining null string values in ui protobuf serializer and add tests Sub-task Resolved Gengliang Wang

        Activity

          People

            Gengliang.Wang Gengliang Wang
            Gengliang.Wang Gengliang Wang
            Votes:
            0 Vote for this issue
            Watchers:
            13 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: