[Bug] Fix skip() api maybe skip unexpected bytes which makes inconsistent data (#40) (#52)

### What changes were proposed in this pull request?
Fix bug when call `inputstream.skip()` which may return unexpected result


### Why are the changes needed?
Get exception messages as following, and it maybe caused by unexpected data from `Local` storage
```
com.tencent.rss.common.exception.RssException: Unexpected crc value for blockId[9992363390829154], expected:2562548848, actual:2244862586
        at com.tencent.rss.client.impl.ShuffleReadClientImpl.readShuffleBlockData(ShuffleReadClientImpl.java:184)
        at org.apache.spark.shuffle.reader.RssShuffleDataIterator.hasNext(RssShuffleDataIterator.java:99)
        at org.apache.spark.InterruptibleIterator.hasNext(InterruptibleIterator.scala:39)
        at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:408)
```


### Does this PR introduce _any_ user-facing change?
No


### How was this patch tested?
With current UTs
1 file changed
tree: a75cc2015f14759c6ac4ddfbffb38badeffd82af
  1. .github/
  2. bin/
  3. client/
  4. client-mr/
  5. client-spark/
  6. common/
  7. conf/
  8. coordinator/
  9. docs/
  10. integration-test/
  11. internal-client/
  12. proto/
  13. server/
  14. spark-patches/
  15. storage/
  16. .gitignore
  17. build_distribution.sh
  18. checkstyle-suppressions.xml
  19. checkstyle.xml
  20. CONTRIBUTING.md
  21. LICENSE
  22. pom.xml
  23. README.md
  24. spotbugs-exclude.xml
README.md

What is Firestorm

Firestorm is a Remote Shuffle Service, and provides the capability for Apache Spark applications to store shuffle data on remote servers.

Architecture

Rss Architecture Firestorm contains coordinator cluster, shuffle server cluster and remote storage(eg, HDFS) if necessary.

Coordinator will collect status of shuffle server and do the assignment for the job.

Shuffle server will receive the shuffle data, merge them and write to storage.

Depend on different situation, Firestorm supports Memory & Local, Memory & Remote Storage(eg, HDFS), Memory & Local & Remote Storage(recommendation for production environment).

Shuffle Process with Firestorm

  • Spark driver ask coordinator to get shuffle server for shuffle process

  • Spark task write shuffle data to shuffle server with following step: Rss Shuffle_Write

    1. Send KV data to buffer
    2. Flush buffer to queue when buffer is full or buffer manager is full
    3. Thread pool get data from queue
    4. Request memory from shuffle server first and send the shuffle data
    5. Shuffle server cache data in memory first and flush to queue when buffer manager is full
    6. Thread pool get data from queue
    7. Write data to storage with index file and data file
    8. After write data, task report all blockId to shuffle server, this step is used for data validation later
    9. Store taskAttemptId in MapStatus to support Spark speculation
  • Depend on different storage type, spark task read shuffle data from shuffle server or remote storage or both of them.

Shuffle file format

The shuffle data is stored with index file and data file. Data file has all blocks for specific partition and index file has metadata for every block.

Rss Shuffle_Write

Supported Spark Version

Current support Spark 2.3.x, Spark 2.4.x, Spark3.0.x, Spark 3.1.x, Spark 3.2.x

Note: To support dynamic allocation, the patch(which is included in client-spark/patch folder) should be applied to Spark

Supported MapReduce Version

Current support Hadoop 2.8.5's MapReduce framework.

Building Firestorm

Firestorm is built using Apache Maven. To build it, run:

mvn -DskipTests clean package

To package the Firestorm, run:

./build_distribution.sh

rss-xxx.tgz will be generated for deployment

Deploy

Deploy Coordinator

  1. unzip package to RSS_HOME
  2. update RSS_HOME/bin/rss-env.sh, eg,
      JAVA_HOME=<java_home>
      HADOOP_HOME=<hadoop home>
      XMX_SIZE="16g"
    
  3. update RSS_HOME/conf/coordinator.conf, eg,
      rss.rpc.server.port 19999
      rss.jetty.http.port 19998
      rss.coordinator.server.heartbeat.timeout 30000
      rss.coordinator.app.expired 60000
      rss.coordinator.shuffle.nodes.max 5
      # enable dynamicClientConf, and coordinator will be responsible for most of client conf
      rss.coordinator.dynamicClientConf.enabled true
      # config the path of client conf
      rss.coordinator.dynamicClientConf.path <RSS_HOME>/conf/dynamic_client.conf
      # config the path of excluded shuffle server
      rss.coordinator.exclude.nodes.file.path <RSS_HOME>/conf/exclude_nodes
    
  4. update <RSS_HOME>/conf/dynamic_client.conf, rss client will get default conf from coordinator eg,
     # MEMORY_LOCALFILE_HDFS is recommandation for production environment
     rss.storage.type MEMORY_LOCALFILE_HDFS
     # multiple remote storages are supported, and client will get assignment from coordinator
     rss.coordinator.remote.storage.path hdfs://cluster1/path,hdfs://cluster2/path
     rss.writer.require.memory.retryMax 1200
     rss.client.retry.max 100
     rss.writer.send.check.timeout 600000
     rss.client.read.buffer.size 14m
    
  5. start Coordinator
     bash RSS_HOME/bin/start-coordnator.sh
    

Deploy Shuffle Server

  1. unzip package to RSS_HOME
  2. update RSS_HOME/bin/rss-env.sh, eg,
      JAVA_HOME=<java_home>
      HADOOP_HOME=<hadoop home>
      XMX_SIZE="80g"
    
  3. update RSS_HOME/conf/server.conf, eg,
      rss.rpc.server.port 19999
      rss.jetty.http.port 19998
      rss.rpc.executor.size 2000
      # it should be configed the same as in coordinator
      rss.storage.type MEMORY_LOCALFILE_HDFS
      rss.coordinator.quorum <coordinatorIp1>:19999,<coordinatorIp2>:19999
      # local storage path for shuffle server
      rss.storage.basePath /data1/rssdata,/data2/rssdata....
      # it's better to config thread num according to local disk num
      rss.server.flush.thread.alive 5
      rss.server.flush.threadPool.size 10
      rss.server.buffer.capacity 40g
      rss.server.read.buffer.capacity 20g
      rss.server.heartbeat.timeout 60000
      rss.server.heartbeat.interval 10000
      rss.rpc.message.max.size 1073741824
      rss.server.preAllocation.expired 120000
      rss.server.commit.timeout 600000
      rss.server.app.expired.withoutHeartbeat 120000
      # note: the default value of rss.server.flush.cold.storage.threshold.size is 64m
      # there will be no data written to DFS if set it as 100g even rss.storage.type=MEMORY_LOCALFILE_HDFS
      # please set proper value if DFS is used, eg, 64m, 128m.
      rss.server.flush.cold.storage.threshold.size 100g
    
  4. start Shuffle Server
     bash RSS_HOME/bin/start-shuffle-server.sh
    

Deploy Spark Client

  1. Add client jar to Spark classpath, eg, SPARK_HOME/jars/

    The jar for Spark2 is located in <RSS_HOME>/jars/client/spark2/rss-client-XXXXX-shaded.jar

    The jar for Spark3 is located in <RSS_HOME>/jars/client/spark3/rss-client-XXXXX-shaded.jar

  2. Update Spark conf to enable Firestorm, eg,

    spark.shuffle.manager org.apache.spark.shuffle.RssShuffleManager
    spark.rss.coordinator.quorum <coordinatorIp1>:19999,<coordinatorIp2>:19999
    

Support Spark dynamic allocation

To support spark dynamic allocation with Firestorm, spark code should be updated. There are 2 patches for spark-2.4.6 and spark-3.1.2 in spark-patches folder for reference.

After apply the patch and rebuild spark, add following configuration in spark conf to enable dynamic allocation:

spark.shuffle.service.enabled false
spark.dynamicAllocation.enabled true

Deploy MapReduce Client

  1. Add client jar to the classpath of each NodeManager, e.g., /share/hadoop/mapreduce/

The jar for MapReduce is located in <RSS_HOME>/jars/client/mr/rss-client-mr-XXXXX-shaded.jar

  1. Update MapReduce conf to enable Firestorm, eg,

    -Dmapreduce.rss.coordinator.quorum=<coordinatorIp1>:19999,<coordinatorIp2>:19999
    -Dyarn.app.mapreduce.am.command-opts=org.apache.hadoop.mapreduce.v2.app.RssMRAppMaster
    -Dmapreduce.job.map.output.collector.class=org.apache.hadoop.mapred.RssMapOutputCollector
    -Dmapreduce.job.reduce.shuffle.consumer.plugin.class=org.apache.hadoop.mapreduce.task.reduce.RssShuffle
    

Note that the RssMRAppMaster will automatically disable slow start (i.e., mapreduce.job.reduce.slowstart.completedmaps=1) and job recovery (i.e., yarn.app.mapreduce.am.job.recovery.enable=false)

Configuration

The important configuration is listed as following.

Coordinator

Property NameDefaultDescription
rss.coordinator.server.heartbeat.timeout30000Timeout if can't get heartbeat from shuffle server
rss.coordinator.assignment.strategyPARTITION_BALANCEStrategy for assigning shuffle server, PARTITION_BALANCE should be used for workload balance
rss.coordinator.app.expired60000Application expired time (ms), the heartbeat interval should be less than it
rss.coordinator.shuffle.nodes.max9The max number of shuffle server when do the assignment
rss.coordinator.dynamicClientConf.path-The path of configuration file which have default conf for rss client
rss.coordinator.exclude.nodes.file.path-The path of configuration file which have exclude nodes
rss.coordinator.exclude.nodes.check.interval.ms60000Update interval (ms) for exclude nodes
rss.rpc.server.port-RPC port for coordinator
rss.jetty.http.port-Http port for coordinator

Shuffle Server

Property NameDefaultDescription
rss.coordinator.quorum-Coordinator quorum
rss.rpc.server.port-RPC port for Shuffle server
rss.jetty.http.port-Http port for Shuffle server
rss.server.buffer.capacity-Max memory of buffer manager for shuffle server
rss.server.memory.shuffle.highWaterMark.percentage75.0Threshold of spill data to storage, percentage of rss.server.buffer.capacity
rss.server.memory.shuffle.lowWaterMark.percentage25.0Threshold of keep data in memory, percentage of rss.server.buffer.capacity
rss.server.read.buffer.capacity-Max size of buffer for reading data
rss.server.heartbeat.interval10000Heartbeat interval to Coordinator (ms)
rss.server.flush.threadPool.size10Thread pool for flush data to file
rss.server.commit.timeout600000Timeout when commit shuffle data (ms)
rss.storage.type-Supports MEMORY_LOCALFILE, MEMORY_HDFS, MEMORY_LOCALFILE_HDFS
rss.server.flush.cold.storage.threshold.size64MThe threshold of data size for LOACALFILE and HDFS if MEMORY_LOCALFILE_HDFS is used

Spark Client

Property NameDefaultDescription
spark.rss.writer.buffer.size3mBuffer size for single partition data
spark.rss.writer.buffer.spill.size128mBuffer size for total partition data
spark.rss.coordinator.quorum-Coordinator quorum
spark.rss.storage.type-Supports MEMORY_LOCALFILE, MEMORY_HDFS, MEMORY_LOCALFILE_HDFS
spark.rss.client.send.size.limit16mThe max data size sent to shuffle server
spark.rss.client.read.buffer.size32mThe max data size read from storage
spark.rss.client.send.threadPool.size10The thread size for send shuffle data to shuffle server

MapReduce Client

Property NameDefaultDescription
mapreduce.rss.coordinator.quorum-Coordinator quorum
mapreduce.rss.storage.type-Supports MEMORY_LOCALFILE, MEMORY_HDFS, MEMORY_LOCALFILE_HDFS
mapreduce.rss.client.max.buffer.size3kThe max buffer size in map side
mapreduce.rss.client.read.buffer.size32mThe max data size read from storage

LICENSE

Firestorm is under the Apache License Version 2.0. See the LICENSE file for details.

Contributing

For more information about contributing issues or pull requests, see Firestorm Contributing Guide.

Support

We provide free support for users using this project. If you want to join user wechat group for further help and collaboration. You can scan the following QR code or search wechatID xinghuojihua_01, add our assistant on wechat, and remark Firestorm. The assistant will help you join our wechat group.

Wechat