10.4 C
Canberra
Tuesday, July 1, 2025

RocksDB 101: Optimizing stateful streaming in Apache Spark with Amazon EMR and AWS Glue


Actual-time streaming knowledge processing is a strategic crucial that immediately impacts enterprise competitiveness. Organizations face mounting strain to course of large knowledge streams instantaneously—from detecting fraudulent transactions and delivering personalised buyer experiences to optimizing complicated provide chains and responding to market dynamics milliseconds forward of rivals.

Apache Spark Structured Streaming addresses these essential enterprise challenges by way of its stateful processing capabilities, enabling purposes to keep up and replace intermediate outcomes throughout a number of knowledge streams or time home windows. RocksDB was launched in Apache Spark 3.2, providing a extra environment friendly various to the default HDFS-based in-memory retailer. RocksDB excels in stateful streaming in eventualities that require dealing with massive portions of state knowledge. It delivers optimum efficiency advantages, significantly in decreasing Java digital machine (JVM) reminiscence strain and rubbish assortment (GC) overhead.

This publish explores RocksDB’s key options and demonstrates its implementation utilizing Spark on Amazon EMR and AWS Glue, offering you with the data you want to scale your real-time knowledge processing capabilities.

RocksDB state retailer overview

Spark Structured Streaming processes fall into two classes:

  • Stateful: Requires monitoring intermediate outcomes throughout micro-batches (for instance, when working aggregations and de-duplication).
  • Stateless: Processes every batch independently.

A state retailer is required by stateful purposes that monitor intermediate question outcomes. That is important for computations that rely upon steady occasions and alter outcomes primarily based on every batch of enter, or on mixture knowledge over time, together with late arriving knowledge. By default, Spark provides a state retailer that retains states in JVM reminiscence, which is performant and enough for many common streaming circumstances. Nevertheless, when you have numerous stateful operations in a streaming utility—corresponding to, streaming aggregation, streaming dropDuplicates, stream-stream joins, and so forth—the default in-memory state retailer may face out-of-memory (OOM) points due to a big JVM reminiscence footprint or frequent GC pauses, leading to degraded efficiency.

Benefits of RocksDB over in-memory state retailer

RocksDB addresses the challenges of an in-memory state retailer by way of off-heap reminiscence administration and environment friendly checkpointing.

  • Off-heap reminiscence administration: RocksDB shops state knowledge in OS-managed off-heap reminiscence, decreasing GC strain. Whereas off-heap reminiscence nonetheless consumes machine reminiscence, it doesn’t occupy area within the JVM. As an alternative, its core reminiscence buildings, corresponding to block cache or memTables, allocate immediately from the working system, bypassing the JVM heap. This method makes RocksDB an optimum selection for memory-intensive purposes.
  • Environment friendly checkpointing: RocksDB routinely saves state modifications to checkpoint places, corresponding to Amazon Easy Storage Service (Amazon S3) paths or native directories, serving to to make sure full fault tolerance. When interacting with S3, RocksDB is designed to enhance checkpointing effectivity; it does this by way of incremental updates and compaction to cut back the quantity of knowledge transferred to S3 throughout checkpoints, and by persisting fewer massive state information in comparison with the various small information of the default state retailer, decreasing S3 API calls and latency.

Implementation issues

RocksDB operates as a local C++ library embedded throughout the Spark executor, utilizing off-heap reminiscence. Whereas it doesn’t fall underneath JVM GC management, it nonetheless impacts total executor reminiscence utilization from the YARN or OS perspective. RocksDB’s off-heap reminiscence utilization may exceed YARN container limits with out triggering container termination, doubtlessly resulting in OOM points. It is best to contemplate the next approaches to handle Spark’s reminiscence:

Alter the Spark executor reminiscence dimension

Enhance spark.executor.memoryOverheadorspark.executor.memoryOverheadFactor to depart extra room for off-heap utilization. The next instance units half (4 GB) of spark.executor.reminiscence (8 GB) because the reminiscence overhead dimension.

# Whole executor reminiscence = 8GB (heap) + 4GB (overhead) = 12GB
spark-submit 
. . . . . . . .
--conf spark.executor.reminiscence=8g          # JVM Heap
--conf spark.executor.memoryOverhead=4g  # Off-heap allocation (RocksDB + different native)
. . . . . . . .

For Amazon EMR on Amazon Elastic Compute Cloud (Amazon EC2), enabling YARN reminiscence management with the next strict container reminiscence enforcement by way of polling methodology preempts containers to keep away from node-wide OOM failures:

yarn.nodemanager.useful resource.reminiscence.enforced = false
yarn.nodemanager.elastic-memory-control.enabled = false
yarn.nodemanager.pmem-check-enabled = true 
or 
yarn.nodemanager.vmem-check-enabled = true

Off-heap reminiscence management

Use RocksDB-specific settings to configure reminiscence utilization. Extra particulars may be discovered within the Finest practices and issues part.

Get began with RocksDB on Amazon EMR and AWS Glue

To activate the state retailer RocksDB in Spark, configure your utility with the next setting:

spark.sql.streaming.stateStore.providerClass=org.apache.spark.sql.execution.streaming.state.RocksDBStateStoreProvider

Within the following sections, we discover making a pattern Spark Structured Streaming job with RocksDB enabled working on Amazon EMR and AWS Glue respectively.

RocksDB on Amazon EMR

Amazon EMR variations 6.6.0 and later assist RocksDB, together with Amazon EMR on EC2, Amazon EMR serverless and Amazon EMR on Amazon Elastic Kubernetes Service (Amazon EKS). On this case, we use Amazon EMR on EC2 for example.

Use the next steps to run a pattern streaming job with RocksDB enabled.

  1. Add the next pattern script to s3:///script/sample_script.py
from pyspark.sql import SparkSession
from pyspark.sql.capabilities import explode, cut up, col, expr
import random

# Listing of phrases
phrases = ["apple", "banana", "orange", "grape", "melon", 
         "peach", "berry", "mango", "kiwi", "lemon"]

# Create random strings from phrases
def generate_random_string():
    return " ".be part of(random.selections(phrases, okay=5)) 
    
    
# Create Spark Session
spark = SparkSession 
    .builder 
    .appName("StreamingWordCount") 
    .config("spark.sql.streaming.stateStore.providerClass","org.apache.spark.sql.execution.streaming.state.RocksDBStateStoreProvider") 
    .getOrCreate()


# Register UDF
spark.udf.register("random_string", generate_random_string)

# Create streaming knowledge
raw_stream = spark.readStream 
    .format("charge") 
    .possibility("rowsPerSecond", 1) 
    .load() 
    .withColumn("phrases", expr("random_string()"))

# Execute phrase counts
wordCounts = raw_stream.choose(explode(cut up(raw_stream.phrases, " ")).alias("phrase")).groupby("phrase").rely()

# Output the outcomes
question = wordCounts 
    .writeStream 
    .outputMode("full") 
    .format("console") 
    .begin()

question.awaitTermination()

  1. On the AWS Administration Console for Amazon EMR, select Create Cluster
  2. For Identify and purposes – required, choose the most recent Amazon EMR launch.
  3. For Steps, select Add. For Kind, choose Spark utility.
  4. For Identify, enter GettingStartedWithRocksDB and s3:///script/sample_script.py because the Software location.
  5. Select Save step.
  6. For different settings, select the suitable settings primarily based in your use case.
  7. Select Create cluster to start out the streaming utility through Amazon EMR step.

RocksDB on AWS Glue

AWS Glue 4.0 and later variations assist RocksDB. Use the next steps to run the pattern job with RocksDB enabled on AWS Glue.

  1. On the AWS Glue console, within the navigation pane, select ETL jobs.
  2. Select Script editor and Create script.
  3. For the job title, enter GettingStartedWithRocksDB.
  4. Copy the script from the earlier instance and paste it on the Script tab.
  5. On Job particulars tab, for Kind, choose Spark Streaming.
  6. Select Save, after which select Run to start out the streaming job on AWS Glue.

Walkthrough particulars

Let’s dive deep into the script to grasp tips on how to run a easy stateful Spark utility with RocksDB utilizing the next instance pySpark code.

  1. First, arrange RocksDB as your state retailer by configuring the supplier class:
spark = SparkSession 
    .builder 
    .appName("StreamingWordCount") 
    .config("spark.sql.streaming.stateStore.providerClass","org.apache.spark.sql.execution.streaming.state.RocksDBStateStoreProvider") 
    .getOrCreate()

  1. To simulate streaming knowledge, create an information stream utilizing the charge supply kind. It generates one file per second, containing 5 random fruit names from a pre-defined record.
# Listing of phrases
phrases = ["apple", "banana", "orange", "grape", "melon", 
         "peach", "berry", "mango", "kiwi", "lemon"]

# Create random strings from phrases
def generate_random_string():
    return " ".be part of(random.selections(phrases, okay=5))
# Register UDF
spark.udf.register("random_string", generate_random_string)

# Create streaming knowledge
raw_stream = spark.readStream 
    .format("charge") 
    .possibility("rowsPerSecond", 1) 
    .load() 
    .withColumn("phrases", expr("random_string()"))

  1. Create a phrase counting operation on the incoming stream. This can be a stateful operation as a result of it maintains working counts between processing intervals, that’s, earlier counts have to be saved to calculate the following new totals.
# Break up raw_stream into phrases and counts them
wordCounts = raw_stream.choose(explode(cut up(raw_stream.phrases, " ")).alias("phrase")).groupby("phrase").rely()

  1. Lastly, output the phrase rely totals to the console:
# Output the outcomes
question = wordCounts 
    .writeStream 
    .outputMode("full") 
    .format("console") 
    .begin()

Enter knowledge

In the identical pattern code, check knowledge (raw_stream) is generated at a charge of one-row-per-second, as proven within the following instance:

+-----------------------+-----+--------------------------------+
|timestamp              |worth|phrases                           |
+-----------------------+-----+--------------------------------+
|2025-04-18 07:05:57.204|125  |berry peach orange banana banana|
+-----------------------+-----+--------------------------------+

Output outcome

The streaming job produces the next leads to the output logs. It demonstrates how Spark Structured Streaming maintains and updates the state throughout a number of micro-batches:

  • Batch 0: Begins with an empty state
  • Batch 1: Processes a number of enter information, leading to preliminary counts for each one of many 10 fruits (for instance, banana seems 8 instances)
  • Batch 2: Working totals primarily based on new occurrences from the following set of information are added to the counts (for instance,  banana will increase from 8 to fifteen, indicating 7 new occurrences).

-------------------------------------------
Batch: 0
-------------------------------------------
+----+-----+
|phrase|rely|
+----+-----+
+----+-----+

-------------------------------------------
Batch: 1
-------------------------------------------
+------+-----+
|  phrase|rely|
+------+-----+
|banana|    8|
|orange|    4|
| apple|    3|
| berry|    5|
| lemon|    7|
|  kiwi|    6|
| melon|    8|
| peach|    8|
| mango|    7|
| grape|    9|
+------+-----+

-------------------------------------------
Batch: 2
-------------------------------------------
+------+-----+
|  phrase|rely|
+------+-----+
|banana|   15|
|orange|    8|
| apple|    7|
| berry|   11|
| lemon|   12|
|  kiwi|   11|
| melon|   16|
| peach|   15|
| mango|   12|
| grape|   13|
+------+-----+

State retailer logs

RocksDB generates detailed logs through the job run, like the next:

INFO    2025-04-18T07:52:28,378 83933   org.apache.spark.sql.execution.streaming.MicroBatchExecution    [stream execution thread for [id = xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx, runId = xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx]] 60  Streaming question made progress: {
  "id": "xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx",
  "runId": "xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx",
  "title": null,
  "timestamp": "2025-04-18T07:52:27.642Z",
  "batchId": 39,
  "numInputRows": 1,
  "inputRowsPerSecond": 100.0,
  "processedRowsPerSecond": 1.3623978201634879,
  "durationMs": {
    "addBatch": 648,
    "commitOffsets": 39,
    "getBatch": 0,
    "latestOffset": 0,
    "queryPlanning": 10,
    "triggerExecution": 734,
    "walCommit": 35
  },
  "stateOperators": [
    {
      "operatorName": "stateStoreSave",
      "numRowsTotal": 10,
      "numRowsUpdated": 4,
      "allUpdatesTimeMs": 18,
      "numRowsRemoved": 0,
      "allRemovalsTimeMs": 0,
      "commitTimeMs": 3629,
      "memoryUsedBytes": 174179,
      "numRowsDroppedByWatermark": 0,
      "numShufflePartitions": 36,
      "numStateStoreInstances": 36,
      "customMetrics": {
        "rocksdbBytesCopied": 5009,
        "rocksdbCommitCheckpointLatency": 533,
        "rocksdbCommitCompactLatency": 0,
        "rocksdbCommitFileSyncLatencyMs": 2991,
        "rocksdbCommitFlushLatency": 44,
        "rocksdbCommitPauseLatency": 0,
        "rocksdbCommitWriteBatchLatency": 0,
        "rocksdbFilesCopied": 4,
        "rocksdbFilesReused": 24,
        "rocksdbGetCount": 8,
        "rocksdbGetLatency": 0,
        "rocksdbPinnedBlocksMemoryUsage": 3168,
        "rocksdbPutCount": 4,
        "rocksdbPutLatency": 0,
        "rocksdbReadBlockCacheHitCount": 8,
        "rocksdbReadBlockCacheMissCount": 0,
        "rocksdbSstFileSize": 35035,
        "rocksdbTotalBytesRead": 136,
        "rocksdbTotalBytesReadByCompaction": 0,
        "rocksdbTotalBytesReadThroughIterator": 0,
        "rocksdbTotalBytesWritten": 228,
        "rocksdbTotalBytesWrittenByCompaction": 0,
        "rocksdbTotalBytesWrittenByFlush": 5653,
        "rocksdbTotalCompactionLatencyMs": 0,
        "rocksdbWriterStallLatencyMs": 0,
        "rocksdbZipFileBytesUncompressed": 266452
      }
    }
  ],
  "sources": [
    {
      "description": "RateStreamV2[rowsPerSecond=1, rampUpTimeSeconds=0, numPartitions=default",
      "startOffset": 63,
      "endOffset": 64,
      "latestOffset": 64,
      "numInputRows": 1,
      "inputRowsPerSecond": 100.0,
      "processedRowsPerSecond": 1.3623978201634879
    }
  ],
  "sink": {
    "description": "org.apache.spark.sql.execution.streaming.ConsoleTable$@2cf39784",
    "numOutputRows": 10
  }
}

In Amazon EMR on EC2, these logs can be found on the node the place the YARN ApplicationMaster container is working. They are often discovered at/var/log/hadoop-yarn/containers///stderr.

As for AWS Glue, yow will discover the RocksDB metrics in Amazon CloudWatch, underneath the log group /aws-glue/jobs/error.

RocksDB metrics

The metrics from the previous logs present insights on RocksDB standing. The followings are some instance metrics you may discover helpful when investigating streaming job points:

  • rocksdbCommitCheckpointLatency: Time spent writing checkpoints to native storage
  • rocksdbCommitCompactLatency: Length of checkpoint compaction operations throughout checkpoint commits
  • rocksdbSstFileSize: Present dimension of SST information in RocksDB.

Deep dive into RocksDB key ideas

To higher perceive the state metrics proven within the logs, we deep dive into RocksDB’s key ideas: MemTable, sorted string desk (SST) file, and checkpoints. Moreover, we offer some ideas for finest practices and fine-tuning.

Excessive stage structure

RocksDB is an area, non-distributed persistent key-value retailer embedded in Spark executors. It permits scalable state administration for streaming workloads, backed by Spark’s checkpointing for fault tolerance. As proven within the previous determine, RocksDB shops knowledge in reminiscence and likewise on disk. RocksDB’s means to spill knowledge over to disk is what permits Spark Structured Streaming to deal with state knowledge that exceeds the accessible reminiscence.

Reminiscence:

  • Write buffers (MemTables): Designated reminiscence to buffer writes earlier than flushing onto disk
  • Block cache (learn buffer): Reduces question time by caching outcomes from disk

Disk:

  • SST information: Sorted String Desk saved as SST file format for quick entry

MemTable: Saved off-heap

MemTable, proven within the previous determine, is an in-memory retailer the place knowledge is first written off-heap, earlier than being flushed to disk as an SST file. RocksDB caches the most recent two batches of knowledge (sizzling knowledge) in MemTable to cut back streaming course of latency. By default, RocksDB solely has two MemTables—one is energetic and the opposite is read-only. If in case you have enough reminiscence, the configuration spark.sql.streaming.stateStore.rocksdb.maxWriteBufferNumber may be elevated to have greater than two MemTables. Amongst these MemTables, there may be at all times one energetic desk, and the remaining are read-only MemTables used as write buffers.

SST information: Saved on Spark executor’s native disk

SST information are block-based tables saved on the Spark executor’s native disk. When the in-memory state knowledge can not match right into a MemTable (outlined by a Spark configuration writeBufferSizeMB), the energetic desk is marked as immutable, saving it because the SST file format, which switches it to a read-only MemTable whereas asynchronously flushing it to native disks. Whereas flushing, the immutable MemTable can nonetheless be learn, in order that the newest state knowledge is out there with minimal learn latency.

Studying from RocksDB follows the sequence demonstrated by the previous diagram:

  1. Learn from the energetic MemTable.
  2. If not discovered, iterate by way of read-only MemTables within the order of latest to oldest.
  3. If not discovered, learn from BlockCache (learn buffer).
  4. If misses, load index (one index per SST) from disk into BlockCache. Search for key from index and if hits, load knowledge block onto BlockCache and return outcome.

SST information are saved on executors’ native directories underneath the trail of spark.native.dir (default: /tmp) or yarn.nodemanager.local-dirs:

  • Amazon EMR on EC2 – ${yarn.nodemanager.local-dirs}/usercache/hadoop/appcache///
  • Amazon EMR Serverless, Amazon EMR on EKS, AWS Glue${spark.native.dir}//

Moreover, by utilizing utility logs, you may monitor the MemTable flush and SST file add standing underneath the file path:

  • Amazon EMR on EC2/var/log/hadoop-yarn/containers///stderr
  • Amazon EMR on EKS –/var/log/spark/person/-/stderr

The next is an instance command to test the SST file standing in an executor log from Amazon EMR on EKS:

cat /var/log/spark/person/-/stderr/present | grep outdated

or

kubectl logs --namespace emr -c spark-kubernetes-executor | grep outdated

The next screenshot is an instance of the output of both command.

You should use the next examples to test if the MemTable information had been deleted and flushed out to SST:

cat /var/log/spark/person/-/stderr/present | grep deletes

or

kubectl logs --namespace emr -c spark-kubernetes-executor | grep deletes

The next screenshot is an instance of the output of both command.

Checkpoints: Saved on the executor’s native disk or in an S3 bucket

To deal with fault tolerance and fail over from the final dedicated level, RocksDB helps checkpoints. The checkpoint information are often saved on the executor’s disk or in an S3 bucket, together with snapshot and delta or changelog knowledge information.

Beginning with Amazon EMR 7.0 and AWS Glue5.0, RocksDB state retailer offers a brand new characteristic known as changelog checkpointing to improve checkpoint efficiency. when the changelog is enabled (disabled by default) utilizing the setting spark.sql.streaming.stateStore.rocksdb.changelogCheckpointing.enabled, RocksDB writes smaller change logs to the storage location (the native disk by default) as an alternative of ceaselessly persisting massive snapshot knowledge. Word that snapshots are nonetheless created however much less ceaselessly, as proven within the following screenshot.

Right here’s an instance of a checkpoint location path when overridden to an S3 bucket: s3:////state/0/spark_parition_ID/state_version_ID.zip

Finest practices and issues

This part outlines key methods for fine-tuning RocksDB efficiency and avoiding frequent pitfalls.

1. Reminiscence administration for RocksDB

To stop OOM errors on Spark executors, you may configure RocksDB’s reminiscence utilization at both the node stage or occasion stage:

  • Node stage (advisable): Implement a world off-heap reminiscence restrict per executor. On this context, every executor is handled as a RocksDB node. If an executor processes N partitions of a stateful operator, it’ll have N variety of RocksDB situations on a single executor.
  • Occasion-level: Advantageous-tune particular person RocksDB situations.

Node-level reminiscence management per executor

Beginning with Amazon EMR 7.0 and AWS Glue 5.0 (Spark 3.5), a essential Spark configuration, boundedMemoryUsage, was launched (by way of SPARK-43311) to implement a world reminiscence cap at a single executor stage that’s shared by a number of RocksDB situations. This prevents RocksDB from consuming unbounded off-heap reminiscence, which may result in OOM errors or executor termination by useful resource managers corresponding to YARN or Kubernetes.

The next instance exhibits the node-level configuration:

 # Sure whole reminiscence utilization per executor 
 "spark.sql.streaming.stateStore.rocksdb.boundedMemoryUsage": "true"
 # Set a static whole reminiscence dimension per executor
 "spark.sql.streaming.stateStore.rocksdb.maxMemoryUsageMB": "500"
 # For read-heavy workloads, cut up reminiscence allocation between write buffers (30%) and block cache (70%) 
 "spark.sql.streaming.stateStore.rocksdb.writeBufferCacheRatio": "0.3"

A single RocksDB occasion stage management

For granular reminiscence administration, you may configure particular person RocksDB situations utilizing the next settings:

# Management MemTable (write buffer) dimension and rely
"spark.sql.streaming.stateStore.rocksdb.writeBufferSizeMB": "64"
"spark.sql.streaming.stateStore.rocksdb.maxWriteBufferNumber": "4"

  • writeBufferSizeMB (default: 64, recommended: 64 – 128): Controls the most dimension of a single MemTable in RocksDB, affecting reminiscence utilization and write throughput. This setting is out there in Spark3.5 – [SPARK-42819] and later. It determines the dimensions of the reminiscence buffer earlier than state knowledge is flushed to disk. Bigger buffer sizes can enhance write efficiency by decreasing SST flush frequency however will enhance the executor’s reminiscence utilization. Adjusting this parameter is essential for optimizing reminiscence utilization and write throughput.
  • maxWriteBufferNumber (default: 2, recommended: 3 – 4): Units the overall variety of energetic and immutable MemTables.

For read-heavy workloads, prioritize the next block cache tuning over write buffers to cut back disk I/O. You possibly can configure SST block dimension and caching as follows:

"spark.sql.streaming.stateStore.rocksdb.blockSizeKB": "64"
"spark.sql.streaming.stateStore.rocksdb.blockCacheSizeMB": "128"

  •  blockSizeKB (default: 4, recommended: 64–128): When an energetic MemTable is full, it turns into a read-only memTable. From there, new writes proceed to build up in a brand new desk. The read-only MemTable is flushed into SST information on the disk. The information in SST information is roughly chunked into fixed-sized blocks (default is 4 KB). Every block, in flip, retains a number of knowledge entries. When writing knowledge to SST information, you may compress or encode knowledge effectively inside a block, which frequently leads to a smaller knowledge dimension in contrast with its uncooked format.

For workloads with a small state dimension (corresponding to lower than 10 GB), the default block dimension is often enough. For a big state (corresponding to greater than 50 GB), growing the block dimension can enhance compression effectivity and sequential learn efficiency however enhance CPU overhead.

  • blockCacheSizeMB (default: 8, recommended: 64–512, massive state: greater than 1024): When retrieving knowledge from SST information, RocksDB offers a cache layer (block cache) to enhance the learn efficiency. It first locates the information block the place the goal file may reside, then caches the block to reminiscence, and at last searches that file throughout the cached block. To keep away from frequent reads of the identical block, the block cache can be utilized to maintain the loaded blocks in reminiscence.

2. Clear up state knowledge at checkpoint

To assist be sure that your state file sizes and storage prices stay underneath management when checkpoint efficiency turns into a priority, use the next Spark configurations to regulate cleanup frequency, retention limits, and checkpoint file varieties:

# clear up RocksDB state each 30 seconds
"spark.sql.streaming.stateStore.maintenanceInterval":"30s"
# retain solely the final 50 state variations  
"spark.sql.streaming.minBatchesToRetain":"50"
# use changelog as an alternative of snapshots
"spark.sql.streaming.stateStore.rocksdb.changelogCheckpointing.enabled":"true"

  • maintenanceInterval (default: 60 seconds): Retaining a state for an extended time frame may help cut back upkeep price and background IO. Nevertheless, longer intervals enhance file itemizing time, as a result of state shops typically scan each retained file.
  • minBatchesToRetain (default: 100, recommended: 10–50): Limits the variety of state variations retained at checkpoint places. Decreasing this quantity leads to fewer information being persevered and reduces storage utilization.
  • changelogCheckpointing (default: false, recommended: true): Historically, RocksDB snapshots and uploads incremental SST information to checkpoint. To keep away from this price, changelog checkpointing was launched in Amazon EMR7.0+ and AWS Glue 5.0, which write solely state modifications because the final checkpoint.

To trace an SST file’s retention standing, you may search RocksDBFileManager entries within the executor logs. Contemplate the next logs in Amazon EMR on EKS for example. The output (proven within the screenshot) exhibits that 4 SST information underneath model 102 had been uploaded to an S3 checkpoint location, and that an outdated changelog state file with model 97 was cleaned up.

cat /var/log/spark/person/-/stderr/ present | grep RocksDBFileManager

or

kubectl logs  -n emr -c spark-kubernetes-executor | grep RocksDBFileManager

3. Optimize native disk utilization

RocksDB consumes native disk area when producing SST information at every Spark executor. Whereas disk utilization doesn’t scale linearly, RocksDB can accumulate storage over time primarily based on state knowledge dimension. When working streaming jobs, if native accessible disk area will get inadequate, No area left on gadget errors can happen.

To optimize disk utilization by RocksDB, modify the next Spark configurations:

# compact state information throughout commit (default:false)
"spark.sql.streaming.stateStore.rocksdb.compactOnCommit": "true"
# variety of delta SST information earlier than turns into a consolidated snapshot file(default:10)
"spark.sql.streaming.stateStore.minDeltasForSnapshot": "5" 

Infrastructure changes can additional mitigate the disk subject:

For Amazon EMR:

For AWS Glue:

  • Use AWS Glue G.2X or bigger employee varieties to keep away from the restricted disk capability of G.1X staff.
  • Schedule common upkeep home windows at optimum timing to unlock disk area primarily based on workload wants.

Conclusion

On this publish, we explored RocksDB because the new state retailer implementation in Apache Spark Structured Streaming, accessible on Amazon EMR and AWS Glue. RocksDB provides benefits over the default HDFS-backed in-memory state retailer, significantly for purposes coping with large-scale stateful operations. RocksDB helps forestall JVM reminiscence strain and rubbish assortment points frequent with the default state retailer.

The implementation is simple, requiring minimal configuration modifications, although you must pay cautious consideration to reminiscence and disk area administration for optimum efficiency. Whereas RocksDB isn’t assured to cut back job latency, it offers a strong resolution for dealing with large-scale stateful operations in Spark Structured Streaming purposes.

We encourage you to judge RocksDB to your use circumstances, significantly if you happen to’re experiencing reminiscence strain points with the default state retailer or must deal with massive quantities of state knowledge in your streaming purposes.


Concerning the authors

Melody Yang is a Principal Analytics Architect for Amazon EMR at AWS. She is an skilled analytics chief working with AWS clients to offer finest observe steerage and technical recommendation so as to help their success in knowledge transformation. Her areas of pursuits are open-source frameworks and automation, knowledge engineering and DataOps.

Dai Ozaki is a Cloud Assist Engineer on the AWS Huge Knowledge Assist workforce. He’s captivated with serving to clients construct knowledge lakes utilizing ETL workloads. In his spare time, he enjoys taking part in desk tennis.

Noritaka Sekiyama is a Principal Huge Knowledge Architect with Amazon Internet Companies (AWS) Analytics providers. He’s chargeable for constructing software program artifacts to assist clients. In his spare time, he enjoys biking on his street bike.

Amir Shenavandeh is a Senior Analytics Specialist Options Architect and Amazon EMR subject material skilled at Amazon Internet Companies. He helps clients with architectural steerage and optimisation. He leverages his expertise to assist folks convey their concepts to life, specializing in distributed processing and large knowledge architectures.

Xi Yang is a Senior Hadoop System Engineer and Amazon EMR subject material skilled at Amazon Internet Companies. He’s captivated with serving to clients resolve difficult points within the Huge Knowledge space.

Related Articles

LEAVE A REPLY

Please enter your comment!
Please enter your name here

[td_block_social_counter facebook="tagdiv" twitter="tagdivofficial" youtube="tagdiv" style="style8 td-social-boxed td-social-font-icons" tdc_css="eyJhbGwiOnsibWFyZ2luLWJvdHRvbSI6IjM4IiwiZGlzcGxheSI6IiJ9LCJwb3J0cmFpdCI6eyJtYXJnaW4tYm90dG9tIjoiMzAiLCJkaXNwbGF5IjoiIn0sInBvcnRyYWl0X21heF93aWR0aCI6MTAxOCwicG9ydHJhaXRfbWluX3dpZHRoIjo3Njh9" custom_title="Stay Connected" block_template_id="td_block_template_8" f_header_font_family="712" f_header_font_transform="uppercase" f_header_font_weight="500" f_header_font_size="17" border_color="#dd3333"]
- Advertisement -spot_img

Latest Articles