Databases at CERN blog - Powering particle physics https://db-blog.web.cern.ch/ en Building an Apache Spark Performance Lab: Tools and Techniques for Spark Optimization https://db-blog.web.cern.ch/node/195 <span>Building an Apache Spark Performance Lab: Tools and Techniques for Spark Optimization</span> <div class="field field--name-body field--type-text-with-summary field--label-above"> <div class="field--label"><b>Blog article:</b></div> <div class="field--item"><p>Apache Spark is renowned for its speed and efficiency in handling large-scale data processing. However, optimizing Spark to achieve maximum performance requires a precise understanding of its inner workings. This blog post will guide you through establishing a Spark Performance Lab with essential tools and techniques aimed at enhancing Spark performance through detailed metrics analysis.</p> <h2 style="text-align: left;"><span style="color: red;">Why a Spark Performance Lab</span></h2> <p>The purpose of a Spark Performance Lab isn't just to measure the elapsed time of your Spark jobs but to understand the underlying performance metrics deeply. By using these metrics, you can create models that explain what's happening within Spark's execution and identify areas for improvement. Here are some key reasons to set up a Spark Performance Lab:</p> <ul style="text-align: left;"><li><b>Hands-on learning and testing:</b> A controlled lab setting allows for safer experimentation with Spark configurations and tuning and also experimenting and understanding the monitoring tools and Spark-generated metrics.</li> <li><b>Load and scale:</b> Our lab uses a workload generator, running TPCDS queries. This is a well-known set of complex queries that is representative of OLAP workloads, and that can easily be scaled up for testing from GBs to 100s of TBs.</li> <li><b>Improving your toolkit:</b> Having a toolbox is invaluable, however you need to practice and understand their output in a sandbox environment before moving to production.</li> <li><b>Get value from the Spark metric system:</b> Instead of focusing solely on how long a job takes, use detailed metrics to understand the performance and spot inefficiencies.<br />  </li> </ul><h2 style="text-align: left;"><span style="color: red;">Tools and Components</span></h2> <p>In our Spark Performance Lab, several key tools and components form the backbone of our testing and monitoring environment:</p> <ul style="text-align: left;"><li>Workload generator:  <ul><li>We use a custom tool, <b><a href="https://github.com/LucaCanali/Miscellaneous/tree/master/Performance_Testing/TPCDS_PySpark">TPCDS_PySpark</a>,</b> to generate a consistent set of queries (TPCDS benchmark), creating a reliable testing framework.</li> </ul></li> <li>Spark instrumentation:  <ul><li>Spark’s built-in <b><a href="https://spark.apache.org/docs/latest/web-ui.html">Web UI</a> </b>for initial metrics and job visualization.</li> </ul></li> <li>Custom tools: <ul><li><b><a href="https://github.com/LucaCanali/sparkMeasure">SparkMeasure</a>:</b> Use this for detailed performance metrics collection.</li> <li><b><a href="https://github.com/cerndb/spark-dashboard">Spark-Dashboard</a>:</b> Use this to monitor Spark jobs and visualize key performance metrics.</li> </ul></li> </ul><div>Additional tools for Performance Measurement include:</div> <div> <ul style="text-align: left;"><li><a href="https://github.com/LucaCanali/Miscellaneous/blob/master/Spark_Notes/Tools_Spark_Linux_FlameGraph.md">Flame Graphs for Spark</a> and <a href="https://github.com/LucaCanali/Miscellaneous/blob/master/Spark_Notes/Tools_Spark_Pyroscope_FlameGraph.md">Grafana Pyroscope with Spark</a></li> <li>Tools for <a href="https://github.com/LucaCanali/Miscellaneous/blob/master/Spark_Notes/Tools_Linux_OS_CPU_Disk_Network.md">OS performance monitoring</a></li> </ul><p> </p> </div> <div> <h2><span style="color: red;">Demos</span></h2> <p>These quick demos and tutorials will show you how to use the tools in this Spark Performance Lab. You can follow along and get the same results on your own, which will help you start learning and exploring.</p> <ul style="text-align: left;"><li><b>SparkMeasure </b>- metrics collection <ul><li><a data-original-attrs="{&quot;data-original-href&quot;:&quot;https://www.youtube.com/watch?v=NEA1kkFcZWs&quot;}" href="https://www.youtube.com/watch?v=NEA1kkFcZWs" rel="nofollow"><img alt="Watch the video" data-entity-type="" data-entity-uuid="" data-original-attrs="{&quot;data-canonical-src&quot;:&quot;https://www.youtube.com/s/desktop/050e6796/img/favicon_32x32.png&quot;,&quot;style&quot;:&quot;&quot;}" src="https://camo.githubusercontent.com/e8e4794e202e57bbf63e681b48d0dc3144427f0ee4d2baa3afb18b2812091569/68747470733a2f2f7777772e796f75747562652e636f6d2f732f6465736b746f702f30353065363739362f696d672f66617669636f6e5f33327833322e706e67" style="cursor: move; max-width: 100%;" /> Watch sparkMeasure's getting started demo and tutorial</a></li> </ul></li> <li><b>TPCDS_PySpark </b>- workload generator <ul><li><a href="https://www.youtube.com/watch?v=1EjLCamuRl0" rel="nofollow"><img alt="Watch the video" data-canonical-src="https://www.youtube.com/s/desktop/050e6796/img/favicon_32x32.png" data-entity-type="" data-entity-uuid="" src="https://camo.githubusercontent.com/e8e4794e202e57bbf63e681b48d0dc3144427f0ee4d2baa3afb18b2812091569/68747470733a2f2f7777772e796f75747562652e636f6d2f732f6465736b746f702f30353065363739362f696d672f66617669636f6e5f33327833322e706e67" style="max-width: 100%;" /> Watch TPCDS-PySpark demo and tutorial</a> <ul></ul></li> </ul></li> <li><b>Spark-Dashboard </b>- real-time dashboards <ul><li><a href="https://www.youtube.com/watch?v=sLjAyDwpg80" rel="nofollow"><img alt="Watch the video" data-canonical-src="https://www.youtube.com/s/desktop/050e6796/img/favicon_32x32.png" data-entity-type="" data-entity-uuid="" src="https://camo.githubusercontent.com/e8e4794e202e57bbf63e681b48d0dc3144427f0ee4d2baa3afb18b2812091569/68747470733a2f2f7777772e796f75747562652e636f6d2f732f6465736b746f702f30353065363739362f696d672f66617669636f6e5f33327833322e706e67" style="max-width: 100%;" /> Watch Spark-Dashboard demo and tutorial</a></li> </ul></li> </ul><div> </div> <div> <div class="separator" style="clear: both; text-align: center;"><a href="https://blogger.googleusercontent.com/img/b/R29vZ2xl/AVvXsEiRO_L4QLjGz_0vsIrqa6lZVO3lvUAS0dV3WfLsZRRk8HzEkJauJPYuNEOKiFmsGhSUI9Hvj-7fR6AMwgYBaGI45Sw2vX4qssJQSAGpRxdY-F8XxpFNu7EuM0EdwMfSLxyu4s7CRQKN3dDkjEggK6Ld4zo4tv5YZG9bfuyV7O1e82l6Insu-btK19hyjFI/s936/Snapshot_dashboard_TPCDS10000.png" style="margin-left: 1em; margin-right: 1em;"><img alt="" border="0" data-entity-type="" data-entity-uuid="" data-original-height="393" data-original-width="936" height="336" src="https://blogger.googleusercontent.com/img/b/R29vZ2xl/AVvXsEiRO_L4QLjGz_0vsIrqa6lZVO3lvUAS0dV3WfLsZRRk8HzEkJauJPYuNEOKiFmsGhSUI9Hvj-7fR6AMwgYBaGI45Sw2vX4qssJQSAGpRxdY-F8XxpFNu7EuM0EdwMfSLxyu4s7CRQKN3dDkjEggK6Ld4zo4tv5YZG9bfuyV7O1e82l6Insu-btK19hyjFI/s936/Snapshot_dashboard_TPCDS10000.png" width="800" /></a></div> <p><b>Figure 1:</b> Snapshot from the Spark-Dashboard, visualizing the number of active tasks vs. measurement time, taken when running TPCDS scale 10,000 GB on a YARN cluster</p> <div> </div> </div> <h2 style="text-align: left;"><span style="color: red;">How to Make the Best of Spark Metrics System</span></h2> <div> <div>Understanding and utilizing Spark's metrics system is crucial for optimization:</div> <div> <ul style="text-align: left;"><li><b>Importance of Metrics:</b> Metrics provide insights beyond simple timing, revealing details about task execution, resource utilization, and bottlenecks.<br />  </li> <li><b>Execution Time is Not Enough:</b> Measuring the execution time of a job, like how long it takes to run, is useful but it doesn’t show the whole picture. Say the job ran in 10 seconds. It's crucial to understand why it took 10 seconds instead of 100 seconds or just 1 second. What was slowing things down? Was it the CPU, data input/output, or something else, like data shuffling? This helps us identify the <b>root causes</b> of performance issues.<br />  </li> <li><b>Key Metrics to Collect:</b> <ul><li><b>Executor Run Time:</b> Total time executors spend processing tasks.</li> <li><b>Executor CPU Time:</b> Direct CPU time consumed by tasks.</li> <li><b>JVM GC Time:</b> Time spent in garbage collection, affecting performance.</li> <li><b>Shuffle and I/O Metrics</b>: Critical for understanding data movement and disk interactions.</li> <li><b>Memory Metrics</b>: Key for performance and troubleshooting Out Of Memory errors<br />  </li> </ul></li> <li><b>Metrics Analysis, what to look for:</b> <ul><li><b>Look for bottlenecks:</b> are there resources that are the bottleneck? Are the jobs running mostly on CPU or waiting for I/O or spending a lot of time on Garbage Collection?</li> <li><b><a href="https://www.brendangregg.com/usemethod.html">USE method</a></b>: Utilization Saturation and Errors (USE) Method is a methodology for analyzing the performance of any system.  <ul><li>The tools described here can help you to measure and understand Utilization and Saturation.</li> </ul></li> <li><b>Can your job use a significant fraction of the available CPU cores? </b> <ul><li>Examine the measurement of  the actual number of active tasks vs. time.</li> <li>Figure 1 shows the number of active tasks measured while running TPCDS 10TB on a YARN cluster, with 256 cores allocated. The graph shows spikes and troughs.</li> </ul></li> <li> <ul><li>Understand the root causes of the troughs using metrics and monitoring data. The reasons can be many: resource allocation, partition skew, straggler tasks, stage boundaries, etc.</li> </ul></li> <li><b>Which tool should I use?</b> <ul><li>Start with using the Spark Web UI</li> <li>Instrument your jobs with sparkMesure. This is recommended early in the application development, testing, and for Continuous Integration (CI) pipelines.</li> <li>Observe your Spark application execution profile with Spark-Dashboard.</li> <li>Use available tools with OS metrics too. See also Spark-Dashboard extended instrumentation: it collects and visualizes OS metrics (from cgroup statistics) like network stats, etc</li> </ul></li> <li><b>Drill down:</b> <ul><li>An example of <a href="https://github.com/LucaCanali/Miscellaneous/tree/master/Performance_Testing/TPCDS_PySpark#tpcds-at-scale-10000g-and-analysis">Spark metrics analysis for TPCDS run at scale 10 TB</a></li> </ul></li> </ul></li> </ul><div> <ul style="text-align: left;"><li><b>Documentation:</b>  <ul><li>For those interested in delving deeper into Spark instrumentation and metrics, the <a href="https://spark.apache.org/docs/latest/monitoring.html">Spark documentation</a> offers a comprehensive guide.</li> <li>SparkMeasure: This tool captures metrics directly from Spark’s instrumentation via the Listener Bus. For a detailed understanding of how it operates, refer to the <a href="https://github.com/LucaCanali/sparkMeasure?tab=readme-ov-file#architecture-diagram">SparkMeasure architecture</a>. It specifically gathers data from Spark's Task Metrics System, which you can <a href="https://spark.apache.org/docs/latest/monitoring.html#executor-task-metrics">explore further here</a>.</li> <li>Spark-Dashboard: This application aggregates metrics that Spark exposes through the Dropwizard metrics library (see <a href="https://github.com/LucaCanali/Miscellaneous/tree/master/Spark_Dashboard#understanding-the-architecture">Spark-Dashboard architecture</a>). A complete <a href="https://spark.apache.org/docs/latest/monitoring.html#list-of-available-metrics-providers">list of the metrics can be found here</a>.</li> </ul></li> </ul></div> </div> <div> </div> <div> <div class="separator" style="clear: both; text-align: center;"><a href="https://raw.githubusercontent.com/LucaCanali/Miscellaneous/master/Spark_Dashboard/images/Spark_MetricsSystem_Grafana_Dashboard_V2.0.png" imageanchor="1" style="margin-left: 1em; margin-right: 1em;"><img alt="" border="0" data-entity-type="" data-entity-uuid="" data-original-height="450" data-original-width="800" height="450" src="https://raw.githubusercontent.com/LucaCanali/Miscellaneous/master/Spark_Dashboard/images/Spark_MetricsSystem_Grafana_Dashboard_V2.0.png" width="800" /></a></div> <p><b>Figure 2:</b> This technical drawing outlines the integrated monitoring pipeline for Apache Spark implemented by Spark-Dashboard using open-source components. The flow of the diagram illustrates the Spark metrics source and the components used to store and visualize the metrics.</p></div> <div>  </div> </div> </div> <h2 style="text-align: left;"><span style="color: red;">Lessons Learned and Conclusions</span></h2> <p>From setting up and running a Spark Performance Lab, here are some key takeaways:</p> <ul style="text-align: left;"><li><b>Collect, analyze and visualize metrics:</b> Go beyond just measuring jobs' execution times to troubleshoot and fine-tune Spark performance effectively.</li> <li><b>Use the Right Tools:</b> Familiarize yourself with tools for performance measurement and monitoring.</li> <li><b>Start Small, Scale Up:</b> Begin with smaller datasets and configurations, then gradually scale to test larger, more complex scenarios.</li> <li><b>Tuning is an Iterative Process:</b> Experiment with different configurations, parallelism levels, and data partitioning strategies to find the best setup for your workload.</li> </ul><p>Establishing a <b>Spark Performance Lab </b>is a fundamental step for any data engineer aiming to master Spark's performance aspects. By integrating tools like <b>Web UI, TPCDS_PySpark, sparkMeasure, and Spark-Dashboard</b>, developers and data engineers can gain unprecedented insights into Spark operations and optimizations.</p> <p>Explore this lab setup to turn theory into expertise in managing and optimizing Apache Spark. <b>Learn by doing and experimentation!</b></p> <div> </div> <div><b>Acknowledgements: </b>A special acknowledgment goes out to the teams behind the CERN data analytics, monitoring, and web notebook services, as well as the dedicated members of the ATLAS database group.</div> <div> </div> <div> <h2 style="text-align: left;"><span style="color: red;">Resources</span></h2> <div>To get started with the tools mentioned in this blog:</div> <div> <ul style="text-align: left;"><li><a href="https://github.com/LucaCanali/Miscellaneous/tree/master/Performance_Testing/TPCDS_PySpark">TPCDS_PySpark</a></li> <li><a href="https://github.com/LucaCanali/sparkMeasure">SparkMeasure</a></li> <li><a href="https://github.com/cerndb/spark-dashboard">Spark-Dashboard</a> and <a href="https://github.com/LucaCanali/Miscellaneous/tree/master/Spark_Dashboard">Dashboard Notes</a></li> <li><a href="https://github.com/LucaCanali/Miscellaneous/blob/master/Spark_Notes/Tools_Spark_Linux_FlameGraph.md">Flame Graphs for Spark</a> and <a href="https://github.com/LucaCanali/Miscellaneous/blob/master/Spark_Notes/Tools_Spark_Pyroscope_FlameGraph.md">Grafana Pyroscope with Spark</a></li> <li>Tools for <a href="https://github.com/LucaCanali/Miscellaneous/blob/master/Spark_Notes/Tools_Linux_OS_CPU_Disk_Network.md">OS performance monitoring</a></li> </ul></div> <div> </div> </div> </div> </div> <span><a title="View user profile." href="/users/luca-canali" lang="" about="/users/luca-canali" typeof="schema:Person" property="schema:name" datatype="">canali</a></span> <span><time datetime="2024-04-26T14:58:43+02:00" title="Friday, April 26, 2024 - 14:58">Fri, 04/26/2024 - 14:58</time> </span> <div class="field field--name-field-tags field--type-entity-reference field--label-above"> <div class="field--label"><b>Tags</b></div> <div class="field--items"> <div class="field--item"><a href="/tags/apache-spark" hreflang="en">Apache Spark</a></div> <div class="field--item"><a href="/tags/troubleshooting" hreflang="en">Troubleshooting</a></div> <div class="field--item"><a href="/tags/performance" hreflang="en">Performance</a></div> <div class="field--item"><a href="/tags/tools" hreflang="en">Tools</a></div> </div> </div> <section> <h2>Add new comment</h2> <drupal-render-placeholder callback="comment.lazy_builders:renderForm" arguments="0=node&amp;1=195&amp;2=comment_node_blog_post&amp;3=comment_node_blog_post" token="-Td3Hcqf7apEtlDH-YoByh0R7kTYTf2RllxHsWZJCLs"></drupal-render-placeholder> </section> Fri, 26 Apr 2024 12:58:43 +0000 canali 195 at https://db-blog.web.cern.ch Enhancing Apache Spark and Parquet Efficiency: A Deep Dive into Column Indexes and Bloom Filters https://db-blog.web.cern.ch/node/194 <span>Enhancing Apache Spark and Parquet Efficiency: A Deep Dive into Column Indexes and Bloom Filters</span> <div class="field field--name-body field--type-text-with-summary field--label-above"> <div class="field--label"><b>Blog article:</b></div> <div class="field--item"><p><span face="Söhne, ui-sans-serif, system-ui, -apple-system, &quot;Segoe UI&quot;, Roboto, Ubuntu, Cantarell, &quot;Noto Sans&quot;, sans-serif, &quot;Helvetica Neue&quot;, Arial, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;, &quot;Segoe UI Symbol&quot;, &quot;Noto Color Emoji&quot;" style="color: #374151; font-size: 16px; white-space-collapse: preserve;">In the ever-evolving landscape of big data, Apache Spark and Apache Parquet continue to introduce game-changing features. Their latest updates have brought forward significant enhancements, including column indexes, bloom filters. This blog post delves into these new features, exploring their applications and benefits. </span><span style="color: #374151; font-size: 16px; white-space-collapse: preserve;">This post is based on the extended notes at:</span></p> <ul style="text-align: left;"><li><a href="https://github.com/LucaCanali/Miscellaneous/blob/master/Spark_Notes/Spark_Parquet.md"><span style="font-size: medium;">Note on getting the best out of Apache Parquet using Apache Spark</span></a></li> <li><a href="https://github.com/LucaCanali/Miscellaneous/blob/master/Spark_Notes/Tools_Parquet_Diagnostics.md"><span style="font-size: medium;">Note on tools for Parquet diagnostics</span></a></li> </ul><h3 dir="auto" style="background-color: white; box-sizing: border-box; color: #1f2328; font-family: -apple-system, BlinkMacSystemFont, &quot;Segoe UI&quot;, &quot;Noto Sans&quot;, Helvetica, Arial, sans-serif, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;; font-size: 1.25em; font-weight: var(--base-text-weight-semibold, 600); line-height: 1.25; margin-bottom: 16px; margin-top: 24px;" tabindex="-1"> Key Advantages of Parquet in Spark</h3> <h2 style="--tw-border-spacing-x: 0; --tw-border-spacing-y: 0; --tw-ring-color: rgba(69,89,164,.5); --tw-ring-offset-color: #fff; --tw-ring-offset-shadow: 0 0 transparent; --tw-ring-offset-width: 0px; --tw-ring-shadow: 0 0 transparent; --tw-rotate: 0; --tw-scale-x: 1; --tw-scale-y: 1; --tw-scroll-snap-strictness: proximity; --tw-shadow-colored: 0 0 transparent; --tw-shadow: 0 0 transparent; --tw-skew-x: 0; --tw-skew-y: 0; --tw-translate-x: 0; --tw-translate-y: 0; border: 0px solid rgb(217, 217, 227); box-sizing: border-box; color: var(--tw-prose-headings); font-family: Söhne, ui-sans-serif, system-ui, -apple-system, &quot;Segoe UI&quot;, Roboto, Ubuntu, Cantarell, &quot;Noto Sans&quot;, sans-serif, &quot;Helvetica Neue&quot;, Arial, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;, &quot;Segoe UI Symbol&quot;, &quot;Noto Color Emoji&quot;; line-height: 1.33333; margin: 2rem 0px 1rem; white-space-collapse: preserve;"><span color="inherit" style="--tw-border-spacing-x: 0; --tw-border-spacing-y: 0; --tw-ring-color: rgba(69,89,164,.5); --tw-ring-offset-color: #fff; --tw-ring-offset-shadow: 0 0 transparent; --tw-ring-offset-width: 0px; --tw-ring-shadow: 0 0 transparent; --tw-rotate: 0; --tw-scale-x: 1; --tw-scale-y: 1; --tw-scroll-snap-strictness: proximity; --tw-shadow-colored: 0 0 transparent; --tw-shadow: 0 0 transparent; --tw-skew-x: 0; --tw-skew-y: 0; --tw-translate-x: 0; --tw-translate-y: 0; border: 0px solid rgb(217, 217, 227); box-sizing: border-box;"></span></h2> <p dir="auto" style="background-color: white; box-sizing: border-box; color: #1f2328; font-family: -apple-system, BlinkMacSystemFont, &quot;Segoe UI&quot;, &quot;Noto Sans&quot;, Helvetica, Arial, sans-serif, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;; font-size: 16px; font-weight: 400; margin-bottom: 16px; margin-top: 0px; white-space-collapse: collapse;"><span color="inherit" style="--tw-border-spacing-x: 0; --tw-border-spacing-y: 0; --tw-ring-color: rgba(69,89,164,.5); --tw-ring-offset-color: #fff; --tw-ring-offset-shadow: 0 0 transparent; --tw-ring-offset-width: 0px; --tw-ring-shadow: 0 0 transparent; --tw-rotate: 0; --tw-scale-x: 1; --tw-scale-y: 1; --tw-scroll-snap-strictness: proximity; --tw-shadow-colored: 0 0 transparent; --tw-shadow: 0 0 transparent; --tw-skew-x: 0; --tw-skew-y: 0; --tw-translate-x: 0; --tw-translate-y: 0; border: 0px solid rgb(217, 217, 227); box-sizing: border-box;">This is not an introductory article, however here is a quick recap of why you may want to spend time learning more about Apache Parquet and Spark. Parquet is a columnar storage file format optimized for use with data processing frameworks like Apache Spark. It offers efficient data compression and encoding schemes.</span></p> <ul dir="auto" style="background-color: white; box-sizing: border-box; color: #1f2328; font-family: -apple-system, BlinkMacSystemFont, &quot;Segoe UI&quot;, &quot;Noto Sans&quot;, Helvetica, Arial, sans-serif, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;; font-size: 16px; font-weight: 400; margin-bottom: 16px; margin-top: 0px; padding-left: 2em; white-space-collapse: collapse;"><li style="box-sizing: border-box;"><span color="inherit" style="--tw-border-spacing-x: 0; --tw-border-spacing-y: 0; --tw-ring-color: rgba(69,89,164,.5); --tw-ring-offset-color: #fff; --tw-ring-offset-shadow: 0 0 transparent; --tw-ring-offset-width: 0px; --tw-ring-shadow: 0 0 transparent; --tw-rotate: 0; --tw-scale-x: 1; --tw-scale-y: 1; --tw-scroll-snap-strictness: proximity; --tw-shadow-colored: 0 0 transparent; --tw-shadow: 0 0 transparent; --tw-skew-x: 0; --tw-skew-y: 0; --tw-translate-x: 0; --tw-translate-y: 0; border: 0px solid rgb(217, 217, 227); box-sizing: border-box;">Parquet is a columnar format enabling efficient data storage and retrieval</span></li> <li style="box-sizing: border-box; margin-top: 0.25em;"><span color="inherit" style="--tw-border-spacing-x: 0; --tw-border-spacing-y: 0; --tw-ring-color: rgba(69,89,164,.5); --tw-ring-offset-color: #fff; --tw-ring-offset-shadow: 0 0 transparent; --tw-ring-offset-width: 0px; --tw-ring-shadow: 0 0 transparent; --tw-rotate: 0; --tw-scale-x: 1; --tw-scale-y: 1; --tw-scroll-snap-strictness: proximity; --tw-shadow-colored: 0 0 transparent; --tw-shadow: 0 0 transparent; --tw-skew-x: 0; --tw-skew-y: 0; --tw-translate-x: 0; --tw-translate-y: 0; border: 0px solid rgb(217, 217, 227); box-sizing: border-box;">It supports compression and encoding</span></li> <li style="box-sizing: border-box; margin-top: 0.25em;"><span color="inherit" style="--tw-border-spacing-x: 0; --tw-border-spacing-y: 0; --tw-ring-color: rgba(69,89,164,.5); --tw-ring-offset-color: #fff; --tw-ring-offset-shadow: 0 0 transparent; --tw-ring-offset-width: 0px; --tw-ring-shadow: 0 0 transparent; --tw-rotate: 0; --tw-scale-x: 1; --tw-scale-y: 1; --tw-scroll-snap-strictness: proximity; --tw-shadow-colored: 0 0 transparent; --tw-shadow: 0 0 transparent; --tw-skew-x: 0; --tw-skew-y: 0; --tw-translate-x: 0; --tw-translate-y: 0; border: 0px solid rgb(217, 217, 227); box-sizing: border-box;">Optimizations in Spark for Parquet include:</span><br /><ul dir="auto" style="box-sizing: border-box; margin-bottom: 0px; margin-top: 0px; padding-left: 2em;"><li style="box-sizing: border-box;"><span color="inherit" style="--tw-border-spacing-x: 0; --tw-border-spacing-y: 0; --tw-ring-color: rgba(69,89,164,.5); --tw-ring-offset-color: #fff; --tw-ring-offset-shadow: 0 0 transparent; --tw-ring-offset-width: 0px; --tw-ring-shadow: 0 0 transparent; --tw-rotate: 0; --tw-scale-x: 1; --tw-scale-y: 1; --tw-scroll-snap-strictness: proximity; --tw-shadow-colored: 0 0 transparent; --tw-shadow: 0 0 transparent; --tw-skew-x: 0; --tw-skew-y: 0; --tw-translate-x: 0; --tw-translate-y: 0; border: 0px solid rgb(217, 217, 227); box-sizing: border-box;">Vectorized Parquet reader</span></li> <li style="box-sizing: border-box; margin-top: 0.25em;"><span color="inherit" style="--tw-border-spacing-x: 0; --tw-border-spacing-y: 0; --tw-ring-color: rgba(69,89,164,.5); --tw-ring-offset-color: #fff; --tw-ring-offset-shadow: 0 0 transparent; --tw-ring-offset-width: 0px; --tw-ring-shadow: 0 0 transparent; --tw-rotate: 0; --tw-scale-x: 1; --tw-scale-y: 1; --tw-scroll-snap-strictness: proximity; --tw-shadow-colored: 0 0 transparent; --tw-shadow: 0 0 transparent; --tw-skew-x: 0; --tw-skew-y: 0; --tw-translate-x: 0; --tw-translate-y: 0; border: 0px solid rgb(217, 217, 227); box-sizing: border-box;">Filter push down capabilities</span></li> <li style="box-sizing: border-box; margin-top: 0.25em;"><span color="inherit" style="--tw-border-spacing-x: 0; --tw-border-spacing-y: 0; --tw-ring-color: rgba(69,89,164,.5); --tw-ring-offset-color: #fff; --tw-ring-offset-shadow: 0 0 transparent; --tw-ring-offset-width: 0px; --tw-ring-shadow: 0 0 transparent; --tw-rotate: 0; --tw-scale-x: 1; --tw-scale-y: 1; --tw-scroll-snap-strictness: proximity; --tw-shadow-colored: 0 0 transparent; --tw-shadow: 0 0 transparent; --tw-skew-x: 0; --tw-skew-y: 0; --tw-translate-x: 0; --tw-translate-y: 0; border: 0px solid rgb(217, 217, 227); box-sizing: border-box;">Enhanced support for partitioning and handling large files</span></li> </ul></li> </ul><h2 data-original-attrs="{&quot;style&quot;:&quot;--tw-border-spacing-x: 0; --tw-border-spacing-y: 0; --tw-ring-color: rgba(69,89,164,.5); --tw-ring-offset-color: #fff; --tw-ring-offset-shadow: 0 0 transparent; --tw-ring-offset-width: 0px; --tw-ring-shadow: 0 0 transparent; --tw-rotate: 0; --tw-scale-x: 1; --tw-scale-y: 1; --tw-scroll-snap-strictness: proximity; --tw-shadow-colored: 0 0 transparent; --tw-shadow: 0 0 transparent; --tw-skew-x: 0; --tw-skew-y: 0; --tw-translate-x: 0; --tw-translate-y: 0; color: var(--tw-prose-headings);&quot;}" style="border:0px solid #d9d9e3; text-align:left; margin-top:32px; margin-bottom:16px"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><span style="box-sizing:border-box"><span style="line-height:1.33333"></span></span></span></span></span></span></span></span></span></span></h2> <div><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;--tw-border-spacing-x: 0; --tw-border-spacing-y: 0; --tw-ring-color: rgba(69,89,164,.5); --tw-ring-offset-color: #fff; --tw-ring-offset-shadow: 0 0 transparent; --tw-ring-offset-width: 0px; --tw-ring-shadow: 0 0 transparent; --tw-rotate: 0; --tw-scale-x: 1; --tw-scale-y: 1; --tw-scroll-snap-strictness: proximity; --tw-shadow-colored: 0 0 transparent; --tw-shadow: 0 0 transparent; --tw-skew-x: 0; --tw-skew-y: 0; --tw-translate-x: 0; --tw-translate-y: 0;&quot;,&quot;color&quot;:&quot;inherit&quot;}"><font style="border:0px solid #d9d9e3; box-sizing:border-box"><font color="#1f2328"><font face="-apple-system, BlinkMacSystemFont, Segoe UI, Noto Sans, Helvetica, Arial, sans-serif, Apple Color Emoji, Segoe UI Emoji"><span style="font-size:16px"><span style="font-weight:400">Another key aspect of Parquet with Spark that are important to know for the following is:</span></span></font></font></font></font></font></div> <div> <ul><li><font data-keep-original-tag="false"><font color="#1f2328"><font face="-apple-system, BlinkMacSystemFont, Segoe UI, Noto Sans, Helvetica, Arial, sans-serif, Apple Color Emoji, Segoe UI Emoji"><span style="font-size:16px"><span style="font-weight:400"></span></span></font></font></font>Row Group Organization: Parquet files consist of one or more 'row groups,' typically sized around 128 MB, although this is adjustable.</li> <li> <p>Parallel Processing Capabilities: Both Spark and other engines can process Parquet data in parallel, leveraging the row group level or the file level for enhanced efficiency.</p> </li> <li> <p>Row Group Statistics: Each row group holds vital statistics like minimum and maximum values, counts, and the number of nulls. These stats enable the 'skip data' feature when filters are applied, essentially serving as a zone map to optimize query performance.</p> <p> <font data-keep-original-tag="false"><font color="#1f2328"><font face="-apple-system, BlinkMacSystemFont, Segoe UI, Noto Sans, Helvetica, Arial, sans-serif, Apple Color Emoji, Segoe UI Emoji"><span style="font-size:16px"><span style="font-weight:400"></span></span></font></font></font></p></li> </ul></div> <p style="box-sizing: border-box; margin-top: 0.25em;"><span color="inherit" style="--tw-border-spacing-x: 0; --tw-border-spacing-y: 0; --tw-ring-color: rgba(69,89,164,.5); --tw-ring-offset-color: #fff; --tw-ring-offset-shadow: 0 0 transparent; --tw-ring-offset-width: 0px; --tw-ring-shadow: 0 0 transparent; --tw-rotate: 0; --tw-scale-x: 1; --tw-scale-y: 1; --tw-scroll-snap-strictness: proximity; --tw-shadow-colored: 0 0 transparent; --tw-shadow: 0 0 transparent; --tw-skew-x: 0; --tw-skew-y: 0; --tw-translate-x: 0; --tw-translate-y: 0; border: 0px solid rgb(217, 217, 227); box-sizing: border-box;"><span style="box-sizing: border-box; font-weight: var(--base-text-weight-semibold, 600);">ORC:</span> For a comparison of Apache Parquet with another popular data format, Apache ORC, refer to <a href="https://github.com/LucaCanali/Miscellaneous/blob/master/Spark_Notes/Spark_ORC_vs_Parquet.md" style="background-color: transparent; box-sizing: border-box; text-underline-offset: 0.2rem;">Parquet-ORC Comparison</a>.</span><br />  </p> <h2 style="--tw-border-spacing-x: 0; --tw-border-spacing-y: 0; --tw-ring-color: rgba(69,89,164,.5); --tw-ring-offset-color: #fff; --tw-ring-offset-shadow: 0 0 transparent; --tw-ring-offset-width: 0px; --tw-ring-shadow: 0 0 transparent; --tw-rotate: 0; --tw-scale-x: 1; --tw-scale-y: 1; --tw-scroll-snap-strictness: proximity; --tw-shadow-colored: 0 0 transparent; --tw-shadow: 0 0 transparent; --tw-skew-x: 0; --tw-skew-y: 0; --tw-translate-x: 0; --tw-translate-y: 0; border: 0px solid rgb(217, 217, 227); box-sizing: border-box; color: var(--tw-prose-headings); font-family: Söhne, ui-sans-serif, system-ui, -apple-system, &quot;Segoe UI&quot;, Roboto, Ubuntu, Cantarell, &quot;Noto Sans&quot;, sans-serif, &quot;Helvetica Neue&quot;, Arial, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;, &quot;Segoe UI Symbol&quot;, &quot;Noto Color Emoji&quot;; line-height: 1.33333; margin: 2rem 0px 1rem; white-space-collapse: preserve;"><span style="--tw-border-spacing-x: 0; --tw-border-spacing-y: 0; --tw-ring-color: rgba(69,89,164,.5); --tw-ring-offset-color: #fff; --tw-ring-offset-shadow: 0 0 transparent; --tw-ring-offset-width: 0px; --tw-ring-shadow: 0 0 transparent; --tw-rotate: 0; --tw-scale-x: 1; --tw-scale-y: 1; --tw-scroll-snap-strictness: proximity; --tw-shadow-colored: 0 0 transparent; --tw-shadow: 0 0 transparent; --tw-skew-x: 0; --tw-skew-y: 0; --tw-translate-x: 0; --tw-translate-y: 0; border: 0px solid rgb(217, 217, 227); box-sizing: border-box; color: red;">Understanding Column Indexes and Bloom Filters in Parquet</span></h2> <h3 style="--tw-border-spacing-x: 0; --tw-border-spacing-y: 0; --tw-ring-color: rgba(69,89,164,.5); --tw-ring-offset-color: #fff; --tw-ring-offset-shadow: 0 0 transparent; --tw-ring-offset-width: 0px; --tw-ring-shadow: 0 0 transparent; --tw-rotate: 0; --tw-scale-x: 1; --tw-scale-y: 1; --tw-scroll-snap-strictness: proximity; --tw-shadow-colored: 0 0 transparent; --tw-shadow: 0 0 transparent; --tw-skew-x: 0; --tw-skew-y: 0; --tw-translate-x: 0; --tw-translate-y: 0; border: 0px solid rgb(217, 217, 227); box-sizing: border-box; color: var(--tw-prose-headings); font-family: Söhne, ui-sans-serif, system-ui, -apple-system, &quot;Segoe UI&quot;, Roboto, Ubuntu, Cantarell, &quot;Noto Sans&quot;, sans-serif, &quot;Helvetica Neue&quot;, Arial, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;, &quot;Segoe UI Symbol&quot;, &quot;Noto Color Emoji&quot;; font-size: 1.25em; line-height: 1.6; margin: 1rem 0px 0.5rem; white-space-collapse: preserve;"><span color="inherit" style="--tw-border-spacing-x: 0; --tw-border-spacing-y: 0; --tw-ring-color: rgba(69,89,164,.5); --tw-ring-offset-color: #fff; --tw-ring-offset-shadow: 0 0 transparent; --tw-ring-offset-width: 0px; --tw-ring-shadow: 0 0 transparent; --tw-rotate: 0; --tw-scale-x: 1; --tw-scale-y: 1; --tw-scroll-snap-strictness: proximity; --tw-shadow-colored: 0 0 transparent; --tw-shadow: 0 0 transparent; --tw-skew-x: 0; --tw-skew-y: 0; --tw-translate-x: 0; --tw-translate-y: 0; border: 0px solid rgb(217, 217, 227); box-sizing: border-box;">Column Indexes: Enhancing Query Efficiency</span></h3> <p style="--tw-border-spacing-x: 0; --tw-border-spacing-y: 0; --tw-ring-color: rgba(69,89,164,.5); --tw-ring-offset-color: #fff; --tw-ring-offset-shadow: 0 0 transparent; --tw-ring-offset-width: 0px; --tw-ring-shadow: 0 0 transparent; --tw-rotate: 0; --tw-scale-x: 1; --tw-scale-y: 1; --tw-scroll-snap-strictness: proximity; --tw-shadow-colored: 0 0 transparent; --tw-shadow: 0 0 transparent; --tw-skew-x: 0; --tw-skew-y: 0; --tw-translate-x: 0; --tw-translate-y: 0; border: 0px solid rgb(217, 217, 227); box-sizing: border-box; color: #374151; font-family: Söhne, ui-sans-serif, system-ui, -apple-system, &quot;Segoe UI&quot;, Roboto, Ubuntu, Cantarell, &quot;Noto Sans&quot;, sans-serif, &quot;Helvetica Neue&quot;, Arial, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;, &quot;Segoe UI Symbol&quot;, &quot;Noto Color Emoji&quot;; font-size: 16px; margin: 0px 0px 1.25em; white-space-collapse: preserve;">Column indexes, introduced in Parquet 1.11 and utilized in Spark 3.2.0 and higher, offer a fine-grained approach to data filtering. These indexes store min and max values at the Parquet page level, allowing Spark to efficiently execute filter predicates at a much finer granularity than the default 128 MB row group size. Particularly effective for sorted data, column indexes can drastically reduce the data read from disk, improving query performance.</p> <h3 style="--tw-border-spacing-x: 0; --tw-border-spacing-y: 0; --tw-ring-color: rgba(69,89,164,.5); --tw-ring-offset-color: #fff; --tw-ring-offset-shadow: 0 0 transparent; --tw-ring-offset-width: 0px; --tw-ring-shadow: 0 0 transparent; --tw-rotate: 0; --tw-scale-x: 1; --tw-scale-y: 1; --tw-scroll-snap-strictness: proximity; --tw-shadow-colored: 0 0 transparent; --tw-shadow: 0 0 transparent; --tw-skew-x: 0; --tw-skew-y: 0; --tw-translate-x: 0; --tw-translate-y: 0; border: 0px solid rgb(217, 217, 227); box-sizing: border-box; color: var(--tw-prose-headings); font-family: Söhne, ui-sans-serif, system-ui, -apple-system, &quot;Segoe UI&quot;, Roboto, Ubuntu, Cantarell, &quot;Noto Sans&quot;, sans-serif, &quot;Helvetica Neue&quot;, Arial, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;, &quot;Segoe UI Symbol&quot;, &quot;Noto Color Emoji&quot;; font-size: 1.25em; line-height: 1.6; margin: 1rem 0px 0.5rem; white-space-collapse: preserve;"><span color="inherit" style="--tw-border-spacing-x: 0; --tw-border-spacing-y: 0; --tw-ring-color: rgba(69,89,164,.5); --tw-ring-offset-color: #fff; --tw-ring-offset-shadow: 0 0 transparent; --tw-ring-offset-width: 0px; --tw-ring-shadow: 0 0 transparent; --tw-rotate: 0; --tw-scale-x: 1; --tw-scale-y: 1; --tw-scroll-snap-strictness: proximity; --tw-shadow-colored: 0 0 transparent; --tw-shadow: 0 0 transparent; --tw-skew-x: 0; --tw-skew-y: 0; --tw-translate-x: 0; --tw-translate-y: 0; border: 0px solid rgb(217, 217, 227); box-sizing: border-box;">Bloom Filters: A Leap in Filter Operations</span></h3> <p style="--tw-border-spacing-x: 0; --tw-border-spacing-y: 0; --tw-ring-color: rgba(69,89,164,.5); --tw-ring-offset-color: #fff; --tw-ring-offset-shadow: 0 0 transparent; --tw-ring-offset-width: 0px; --tw-ring-shadow: 0 0 transparent; --tw-rotate: 0; --tw-scale-x: 1; --tw-scale-y: 1; --tw-scroll-snap-strictness: proximity; --tw-shadow-colored: 0 0 transparent; --tw-shadow: 0 0 transparent; --tw-skew-x: 0; --tw-skew-y: 0; --tw-translate-x: 0; --tw-translate-y: 0; border: 0px solid rgb(217, 217, 227); box-sizing: border-box; color: #374151; font-family: Söhne, ui-sans-serif, system-ui, -apple-system, &quot;Segoe UI&quot;, Roboto, Ubuntu, Cantarell, &quot;Noto Sans&quot;, sans-serif, &quot;Helvetica Neue&quot;, Arial, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;, &quot;Segoe UI Symbol&quot;, &quot;Noto Color Emoji&quot;; font-size: 16px; margin: 0px 0px 1.25em; white-space-collapse: preserve;">Parquet 1.12 (utilized by Spark 3.2.0 and higher) introduced Bloom filters, a probabilistic data structure that efficiently determines whether an element is in a set. They are particularly useful for columns with high cardinality and scenarios where filter operations are based on values likely to be absent from the dataset. Using bloom filters can lead to significant improvements in read performance.</p> <p style="--tw-border-spacing-x: 0; --tw-border-spacing-y: 0; --tw-ring-color: rgba(69,89,164,.5); --tw-ring-offset-color: #fff; --tw-ring-offset-shadow: 0 0 transparent; --tw-ring-offset-width: 0px; --tw-ring-shadow: 0 0 transparent; --tw-rotate: 0; --tw-scale-x: 1; --tw-scale-y: 1; --tw-scroll-snap-strictness: proximity; --tw-shadow-colored: 0 0 transparent; --tw-shadow: 0 0 transparent; --tw-skew-x: 0; --tw-skew-y: 0; --tw-translate-x: 0; --tw-translate-y: 0; border: 0px solid rgb(217, 217, 227); box-sizing: border-box; color: #374151; font-family: Söhne, ui-sans-serif, system-ui, -apple-system, &quot;Segoe UI&quot;, Roboto, Ubuntu, Cantarell, &quot;Noto Sans&quot;, sans-serif, &quot;Helvetica Neue&quot;, Arial, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;, &quot;Segoe UI Symbol&quot;, &quot;Noto Color Emoji&quot;; font-size: 16px; margin: 0px 0px 1.25em; white-space-collapse: preserve;"> </p> <h2 dir="auto" style="background-color: white; border-bottom: 1px solid var(--borderColor-muted, var(--color-border-muted)); box-sizing: border-box; font-family: -apple-system, BlinkMacSystemFont, &quot;Segoe UI&quot;, &quot;Noto Sans&quot;, Helvetica, Arial, sans-serif, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;; font-weight: var(--base-text-weight-semibold, 600); line-height: 1.25; margin-bottom: 16px; margin-top: 24px; padding-bottom: 0.3em;" tabindex="-1"><span style="color: red;">Example: Spark using Parquet column indexes</span></h2> <p dir="auto" style="background-color: white; box-sizing: border-box; color: #1f2328; font-family: -apple-system, BlinkMacSystemFont, &quot;Segoe UI&quot;, &quot;Noto Sans&quot;, Helvetica, Arial, sans-serif, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;; font-size: 16px; margin-bottom: 16px; margin-top: 0px;"><span style="box-sizing: border-box; font-weight: var(--base-text-weight-semibold, 600);">Test dataset and preparation</span></p> <ul dir="auto" style="background-color: white; box-sizing: border-box; color: #1f2328; font-family: -apple-system, BlinkMacSystemFont, &quot;Segoe UI&quot;, &quot;Noto Sans&quot;, Helvetica, Arial, sans-serif, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;; font-size: 16px; margin-bottom: 16px; margin-top: 0px; padding-left: 2em;"><li style="box-sizing: border-box;">The Parquet test file used below <code style="background-color: var(--bgColor-neutral-muted, var(--color-neutral-muted)); border-radius: 6px; box-sizing: border-box; font-family: ui-monospace, SFMono-Regular, &quot;SF Mono&quot;, Menlo, Consolas, &quot;Liberation Mono&quot;, monospace; font-size: 13.6px; margin: 0px; padding: 0.2em 0.4em; white-space-collapse: break-spaces;">parquet112file_sorted</code> is extracted from the TPCDS benchmark table <a href="https://github.com/apache/spark/blob/master/sql/core/src/test/scala/org/apache/spark/sql/TPCDSSchema.scala#L162" style="background-color: transparent; box-sizing: border-box; text-underline-offset: 0.2rem;">web_sales</a></li> <li style="box-sizing: border-box; margin-top: 0.25em;">the table (parquet file) contains data sorted on the column ws_sold_time_sk</li> <li style="box-sizing: border-box; margin-top: 0.25em;">it's important that the data is sorted, this groups together values in the filter column "ws_sold_time_sk", if the values are scattered the column index min-max statistics will have a wide range and will not be able to help with skipping data</li> <li style="box-sizing: border-box; margin-top: 0.25em;">the sorted dataset has been created using <code style="background-color: var(--bgColor-neutral-muted, var(--color-neutral-muted)); border-radius: 6px; box-sizing: border-box; font-family: ui-monospace, SFMono-Regular, &quot;SF Mono&quot;, Menlo, Consolas, &quot;Liberation Mono&quot;, monospace; font-size: 13.6px; margin: 0px; padding: 0.2em 0.4em; white-space-collapse: break-spaces;">spark.read.parquet("path + "web_sales_piece.parquet").sort("ws_sold_time_sk").coalesce(1).write.parquet(path + "web_sales_piece_sorted_ws_sold_time_sk.parquet")</code></li> <li style="box-sizing: border-box; margin-top: 0.25em;"><span style="box-sizing: border-box; font-weight: var(--base-text-weight-semibold, 600);">Download the test data:</span><br /><ul dir="auto" style="box-sizing: border-box; margin-bottom: 0px; margin-top: 0px; padding-left: 2em;"><li style="box-sizing: border-box;">Retrieve the test data using <code style="background-color: var(--bgColor-neutral-muted, var(--color-neutral-muted)); border-radius: 6px; box-sizing: border-box; font-family: ui-monospace, SFMono-Regular, &quot;SF Mono&quot;, Menlo, Consolas, &quot;Liberation Mono&quot;, monospace; font-size: 13.6px; margin: 0px; padding: 0.2em 0.4em; white-space-collapse: break-spaces;">wget</code>, a web browser, or any method of your choosing</li> <li style="box-sizing: border-box; margin-top: 0.25em;"><a href="https://sparkdltrigger.web.cern.ch/sparkdltrigger/Parquet_Tests/web_sales_piece.parquet" rel="nofollow" style="background-color: transparent; box-sizing: border-box; text-underline-offset: 0.2rem;">web_sales_piece.parquet</a></li> <li style="box-sizing: border-box; margin-top: 0.25em;"><a href="https://sparkdltrigger.web.cern.ch/sparkdltrigger/Parquet_Tests/web_sales_piece_sorted_ws_sold_time_sk.parquet" rel="nofollow" style="background-color: transparent; box-sizing: border-box; text-underline-offset: 0.2rem;">web_sales_piece_sorted_ws_sold_time_sk.parquet</a></li> </ul></li> </ul><p dir="auto" style="background-color: white; box-sizing: border-box; color: #1f2328; font-family: -apple-system, BlinkMacSystemFont, &quot;Segoe UI&quot;, &quot;Noto Sans&quot;, Helvetica, Arial, sans-serif, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;; font-size: 16px; margin-bottom: 16px; margin-top: 0px;"><span style="box-sizing: border-box; font-weight: var(--base-text-weight-semibold, 600);">Run the tests</span></p> <ol dir="auto" style="background-color: white; box-sizing: border-box; color: #1f2328; font-family: -apple-system, BlinkMacSystemFont, &quot;Segoe UI&quot;, &quot;Noto Sans&quot;, Helvetica, Arial, sans-serif, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;; font-size: 16px; margin-bottom: 16px; margin-top: 0px; padding-left: 2em;"><li style="box-sizing: border-box;"><span style="box-sizing: border-box; font-weight: var(--base-text-weight-semibold, 600);">Fast</span> (reads only 20k rows):<br style="box-sizing: border-box;" /><br /> Spark will read the Parquet using a filter and makes use of <span style="box-sizing: border-box; font-weight: var(--base-text-weight-semibold, 600);">column and offset indexes</span>:</li> </ol><div class="snippet-clipboard-content notranslate position-relative overflow-auto" style="box-sizing: border-box; color: #1f2328; display: flex; font-family: -apple-system, BlinkMacSystemFont, &quot;Segoe UI&quot;, &quot;Noto Sans&quot;, Helvetica, Arial, sans-serif, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;; font-size: 16px; justify-content: space-between; margin-bottom: 16px; overflow: auto; position: relative;"> <pre class="notranslate" style="border-radius: 6px; box-sizing: border-box; color: var(--fgColor-default, var(--color-fg-default)); font-family: ui-monospace, SFMono-Regular, &quot;SF Mono&quot;, Menlo, Consolas, &quot;Liberation Mono&quot;, monospace; font-size: 13.6px; line-height: 1.45; margin-bottom: 0px; margin-top: 0px; overflow-wrap: normal; overflow: auto; padding: 16px;"> <code style="background-attachment: initial; background-clip: initial; background-image: initial; background-origin: initial; background-position: initial; background-repeat: initial; background-size: initial; border-radius: 6px; border: 0px; box-sizing: border-box; display: inline; font-family: ui-monospace, SFMono-Regular, &quot;SF Mono&quot;, Menlo, Consolas, &quot;Liberation Mono&quot;, monospace; line-height: inherit; margin: 0px; overflow-wrap: normal; overflow: visible; padding: 0px; word-break: normal;">bin/spark-shell val path = "./" val df = spark.read.parquet(path + "web_sales_piece_sorted_ws_sold_time_sk.parquet") // Read the file using a filter, this will use column and offset indexes val q1 = df.filter("ws_sold_time_sk=28801") val plan = q1.queryExecution.executedPlan q1.collect // Use Spark metrics to see how many rows were processed // This is also available for the WebUI in graphical form val metrics = plan.collectLeaves().head.metrics metrics("numOutputRows").value res: Long = 20000 </code></pre><div class="zeroclipboard-container" style="animation: auto ease 0s 1 normal none running none; box-sizing: border-box;"> </div> </div> <p dir="auto" style="background-color: white; box-sizing: border-box; color: #1f2328; font-family: -apple-system, BlinkMacSystemFont, &quot;Segoe UI&quot;, &quot;Noto Sans&quot;, Helvetica, Arial, sans-serif, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;; font-size: 16px; margin-bottom: 16px; margin-top: 0px;">The result shows that only 20000 rows were processed, this corresponds to processing just a few pages, as opposed to reading and processing the entire file. This is made possible by the use of the min-max value statistics in the column index for column ws_sold_time_sk.<br style="box-sizing: border-box;" /><br /> Column indexes are created by default in Spark version 3.2.x and higher.</p> <p dir="auto" style="background-color: white; box-sizing: border-box; color: #1f2328; font-family: -apple-system, BlinkMacSystemFont, &quot;Segoe UI&quot;, &quot;Noto Sans&quot;, Helvetica, Arial, sans-serif, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;; font-size: 16px; margin-bottom: 16px; margin-top: 0px;"> </p> <ol dir="auto" start="2" style="background-color: white; box-sizing: border-box; color: #1f2328; font-family: -apple-system, BlinkMacSystemFont, &quot;Segoe UI&quot;, &quot;Noto Sans&quot;, Helvetica, Arial, sans-serif, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;; font-size: 16px; margin-bottom: 16px; margin-top: 0px; padding-left: 2em;"><li style="box-sizing: border-box;"><span style="box-sizing: border-box; font-weight: var(--base-text-weight-semibold, 600);">Slow</span> (reads 2M rows):<br style="box-sizing: border-box;" /><br /> Same as above, but this time we disable the use of column indexes.<br style="box-sizing: border-box;" /><br /> Note this is also what happens if you use Spark versions prior to Spark 3.2.0 (notably Spark 2.x) to read the file.</li> </ol><div class="snippet-clipboard-content notranslate position-relative overflow-auto" style="box-sizing: border-box; color: #1f2328; display: flex; font-family: -apple-system, BlinkMacSystemFont, &quot;Segoe UI&quot;, &quot;Noto Sans&quot;, Helvetica, Arial, sans-serif, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;; font-size: 16px; justify-content: space-between; margin-bottom: 16px; overflow: auto; position: relative;"> <pre class="notranslate" style="border-radius: 6px; box-sizing: border-box; color: var(--fgColor-default, var(--color-fg-default)); font-family: ui-monospace, SFMono-Regular, &quot;SF Mono&quot;, Menlo, Consolas, &quot;Liberation Mono&quot;, monospace; font-size: 13.6px; line-height: 1.45; margin-bottom: 0px; margin-top: 0px; overflow-wrap: normal; overflow: auto; padding: 16px;"> <code style="background-attachment: initial; background-clip: initial; background-image: initial; background-origin: initial; background-position: initial; background-repeat: initial; background-size: initial; border-radius: 6px; border: 0px; box-sizing: border-box; display: inline; font-family: ui-monospace, SFMono-Regular, &quot;SF Mono&quot;, Menlo, Consolas, &quot;Liberation Mono&quot;, monospace; line-height: inherit; margin: 0px; overflow-wrap: normal; overflow: visible; padding: 0px; word-break: normal;">bin/spark-shell val path = "./" // disable the use of column indexes for testing purposes val df = spark.read.option("parquet.filter.columnindex.enabled","false").parquet(path + "web_sales_piece_sorted_ws_sold_time_sk.parquet") val q1 = df.filter("ws_sold_time_sk=28801") val plan = q1.queryExecution.executedPlan q1.collect // Use Spark metrics to see how many rows were processed val metrics = plan.collectLeaves().head.metrics metrics("numOutputRows").value res: Long = 2088626 </code></pre><div class="zeroclipboard-container" style="animation: auto ease 0s 1 normal none running none; box-sizing: border-box;"> </div> </div> <p dir="auto" style="background-color: white; box-sizing: border-box; color: #1f2328; font-family: -apple-system, BlinkMacSystemFont, &quot;Segoe UI&quot;, &quot;Noto Sans&quot;, Helvetica, Arial, sans-serif, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;; font-size: 16px; margin-bottom: 16px; margin-top: 0px;">The result is that all the rows in the row group (2088626 rows in the example) were read as Spark could not push the filter down to the Parquet page level. This example runs more slowly than the example below and in general performs more work (uses more CPU cycles and reads more data from the filesystem).</p> <h3 dir="auto" style="background-color: white; box-sizing: border-box; color: #1f2328; font-family: -apple-system, BlinkMacSystemFont, &quot;Segoe UI&quot;, &quot;Noto Sans&quot;, Helvetica, Arial, sans-serif, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;; font-size: 1.25em; font-weight: var(--base-text-weight-semibold, 600); line-height: 1.25; margin-bottom: 16px; margin-top: 24px;" tabindex="-1"><span style="font-size: 1.25em; font-weight: var(--base-text-weight-semibold, 600);"></span></h3> <h3 dir="auto" style="background-color: white; box-sizing: border-box; font-family: -apple-system, BlinkMacSystemFont, &quot;Segoe UI&quot;, &quot;Noto Sans&quot;, Helvetica, Arial, sans-serif, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;; font-size: 1.25em; font-weight: var(--base-text-weight-semibold, 600); line-height: 1.25; margin-bottom: 16px; margin-top: 24px;" tabindex="-1"><span style="color: red; font-size: 1.25em; font-weight: var(--base-text-weight-semibold, 600);">Diagnostics and Internals of Column and Offset Indexes</span></h3> <p dir="auto" style="background-color: white; box-sizing: border-box; color: #1f2328; font-family: -apple-system, BlinkMacSystemFont, &quot;Segoe UI&quot;, &quot;Noto Sans&quot;, Helvetica, Arial, sans-serif, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;; font-size: 16px; margin-bottom: 16px; margin-top: 0px;">Column indexes in Parquet are key structures designed to optimize filter performance during data reads. They are particularly effective for managing and querying large datasets.</p> <h4 style="background-color: white; box-sizing: border-box; color: #1f2328; font-family: -apple-system, BlinkMacSystemFont, &quot;Segoe UI&quot;, &quot;Noto Sans&quot;, Helvetica, Arial, sans-serif, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;; font-size: 1.25em; line-height: 1.25; margin-bottom: 16px; margin-top: 24px; text-align: left;"><span style="font-size: 1.25em; font-weight: var(--base-text-weight-semibold, 600);">Key Aspects of Column Indexes:</span></h4> <ul dir="auto" style="background-color: white; box-sizing: border-box; color: #1f2328; font-family: -apple-system, BlinkMacSystemFont, &quot;Segoe UI&quot;, &quot;Noto Sans&quot;, Helvetica, Arial, sans-serif, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;; font-size: 16px; margin-bottom: 16px; margin-top: 0px; padding-left: 2em;"><li style="box-sizing: border-box;"><span style="box-sizing: border-box; font-weight: var(--base-text-weight-semibold, 600);">Purpose and Functionality:</span> Column indexes offer statistical data (minimum and maximum values) at the page level, facilitating efficient filter evaluation and optimization.</li> <li style="box-sizing: border-box; margin-top: 0.25em;"><span style="box-sizing: border-box; font-weight: var(--base-text-weight-semibold, 600);">Default Activation:</span> By default, column indexes are enabled to ensure optimal query performance.</li> <li style="box-sizing: border-box; margin-top: 0.25em;"><span style="box-sizing: border-box; font-weight: var(--base-text-weight-semibold, 600);">Granularity Insights:</span> While column indexes provide page-level statistics, similar statistics are also available at the row group level. Typically, a row group is approximately 128MB, contrasting with pages usually around 1MB.</li> <li style="box-sizing: border-box; margin-top: 0.25em;"><span style="box-sizing: border-box; font-weight: var(--base-text-weight-semibold, 600);">Customization Options:</span> Both rowgroup and page sizes are configurable, offering flexibility to tailor data organization. For further details, see <a href="https://github.com/LucaCanali/Miscellaneous/blob/master/Spark_Notes/Spark_Parquet.md#parquet-configuration-options" style="background-color: transparent; box-sizing: border-box; text-underline-offset: 0.2rem;">Parquet Configuration Options</a>.</li> </ul><h3 dir="auto" style="background-color: white; box-sizing: border-box; color: #1f2328; font-family: -apple-system, BlinkMacSystemFont, &quot;Segoe UI&quot;, &quot;Noto Sans&quot;, Helvetica, Arial, sans-serif, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;; font-size: 1.25em; font-weight: var(--base-text-weight-semibold, 600); line-height: 1.25; margin-bottom: 16px; margin-top: 24px;" tabindex="-1"><span style="font-size: 1.25em; font-weight: var(--base-text-weight-semibold, 600);">Complementary Role of Offset Indexes:</span></h3> <ul dir="auto" style="background-color: white; box-sizing: border-box; color: #1f2328; font-family: -apple-system, BlinkMacSystemFont, &quot;Segoe UI&quot;, &quot;Noto Sans&quot;, Helvetica, Arial, sans-serif, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;; font-size: 16px; margin-bottom: 16px; margin-top: 0px; padding-left: 2em;"><li style="box-sizing: border-box;"><span style="box-sizing: border-box; font-weight: var(--base-text-weight-semibold, 600);">Association with Column Indexes:</span> Offset indexes work in tandem with column indexes and are stored in the file's footer in Parquet versions 1.11 and above.</li> <li style="box-sizing: border-box; margin-top: 0.25em;"><span style="box-sizing: border-box; font-weight: var(--base-text-weight-semibold, 600);">Scanning Efficiency:</span> A key benefit of these indexes is their role in data scanning. When filters are not applied in Parquet file scans, the footers with column index data can be efficiently skipped, enhancing the scanning process.</li> </ul><h3 dir="auto" style="background-color: white; box-sizing: border-box; color: #1f2328; font-family: -apple-system, BlinkMacSystemFont, &quot;Segoe UI&quot;, &quot;Noto Sans&quot;, Helvetica, Arial, sans-serif, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;; font-size: 1.25em; font-weight: var(--base-text-weight-semibold, 600); line-height: 1.25; margin-bottom: 16px; margin-top: 24px;" tabindex="-1"><span style="font-size: 1.25em; font-weight: var(--base-text-weight-semibold, 600);">Additional Resources:</span></h3> <p dir="auto" style="background-color: white; box-sizing: border-box; color: #1f2328; font-family: -apple-system, BlinkMacSystemFont, &quot;Segoe UI&quot;, &quot;Noto Sans&quot;, Helvetica, Arial, sans-serif, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;; font-size: 16px; margin-bottom: 16px; margin-top: 0px;">For an in-depth explanation of column and offset indexes in Parquet, consider reading this <a href="https://github.com/apache/parquet-format/blob/master/PageIndex.md" style="background-color: transparent; box-sizing: border-box; text-underline-offset: 0.2rem;">detailed description</a>.</p> <p dir="auto" style="background-color: white; box-sizing: border-box; color: #1f2328; font-family: -apple-system, BlinkMacSystemFont, &quot;Segoe UI&quot;, &quot;Noto Sans&quot;, Helvetica, Arial, sans-serif, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;; font-size: 16px; margin-bottom: 16px; margin-top: 0px;">The integration of column and offset indexes significantly improves Parquet's capability in efficiently handling large-scale data, especially in scenarios involving filtered reads. Proper understanding and utilization of these indexes can lead to marked performance improvements in data processing workflows.</p> <h3 dir="auto" style="background-color: white; box-sizing: border-box; color: #1f2328; font-family: -apple-system, BlinkMacSystemFont, &quot;Segoe UI&quot;, &quot;Noto Sans&quot;, Helvetica, Arial, sans-serif, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;; font-size: 1.25em; font-weight: var(--base-text-weight-semibold, 600); line-height: 1.25; margin-bottom: 16px; margin-top: 24px;" tabindex="-1"><span style="font-size: 1.25em; font-weight: var(--base-text-weight-semibold, 600);"></span></h3> <h3 dir="auto" style="background-color: white; box-sizing: border-box; font-family: -apple-system, BlinkMacSystemFont, &quot;Segoe UI&quot;, &quot;Noto Sans&quot;, Helvetica, Arial, sans-serif, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;; font-size: 1.25em; font-weight: var(--base-text-weight-semibold, 600); line-height: 1.25; margin-bottom: 16px; margin-top: 24px;" tabindex="-1"><span style="color: red; font-size: 1.25em; font-weight: var(--base-text-weight-semibold, 600);">Tools to drill down on column index metadata in Parquet files</span></h3> <ul dir="auto" style="background-color: white; box-sizing: border-box; color: #1f2328; font-family: -apple-system, BlinkMacSystemFont, &quot;Segoe UI&quot;, &quot;Noto Sans&quot;, Helvetica, Arial, sans-serif, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;; font-size: 16px; margin-bottom: 16px; margin-top: 0px; padding-left: 2em;"><li style="box-sizing: border-box;"> <p dir="auto" style="box-sizing: border-box; margin-bottom: 16px; margin-top: 16px;"><span style="box-sizing: border-box; font-weight: var(--base-text-weight-semibold, 600);">parquet-cli</span></p> <ul dir="auto" style="box-sizing: border-box; margin-bottom: 0px; margin-top: 0px; padding-left: 2em;"><li style="box-sizing: border-box;">example: <code style="background-color: var(--bgColor-neutral-muted, var(--color-neutral-muted)); border-radius: 6px; box-sizing: border-box; font-family: ui-monospace, SFMono-Regular, &quot;SF Mono&quot;, Menlo, Consolas, &quot;Liberation Mono&quot;, monospace; font-size: 13.6px; margin: 0px; padding: 0.2em 0.4em; white-space-collapse: break-spaces;">hadoop jar target/parquet-cli-1.13.1-runtime.jar org.apache.parquet.cli.Main column-index -c ws_sold_time_sk &lt;path&gt;/my_parquetfile</code></li> <li style="box-sizing: border-box; margin-top: 0.25em;">More details on how to use parquet-cli at <a href="https://github.com/LucaCanali/Miscellaneous/blob/master/Spark_Notes/Tools_Parquet_Diagnostics.md" style="background-color: transparent; box-sizing: border-box; text-underline-offset: 0.2rem;">Tools for Parquet Diagnostics</a></li> </ul></li> <li style="box-sizing: border-box; margin-top: 0.25em;"> <p dir="auto" style="box-sizing: border-box; margin-bottom: 16px; margin-top: 16px;">Example with the <span style="box-sizing: border-box; font-weight: var(--base-text-weight-semibold, 600);">Java API</span> from Spark-shell</p> <div class="snippet-clipboard-content notranslate position-relative overflow-auto" style="background-color: var(--bgColor-muted, var(--color-canvas-subtle)); box-sizing: border-box; display: flex; justify-content: space-between; margin-bottom: 16px; overflow: auto; position: relative;"> <pre class="notranslate" style="background-color: var(--bgColor-muted, var(--color-canvas-subtle)); border-radius: 6px; box-sizing: border-box; color: var(--fgColor-default, var(--color-fg-default)); font-family: ui-monospace, SFMono-Regular, &quot;SF Mono&quot;, Menlo, Consolas, &quot;Liberation Mono&quot;, monospace; font-size: 13.6px; line-height: 1.45; margin-bottom: 0px; margin-top: 0px; overflow-wrap: normal; overflow: auto; padding: 16px;"> <code style="background: transparent; border-radius: 6px; border: 0px; box-sizing: border-box; display: inline; font-family: ui-monospace, SFMono-Regular, &quot;SF Mono&quot;, Menlo, Consolas, &quot;Liberation Mono&quot;, monospace; line-height: inherit; margin: 0px; overflow-wrap: normal; overflow: visible; padding: 0px; word-break: normal;">// customize with the file path and name val fullPathUri = java.net.URI.create("&lt;path&gt;/myParquetFile") // crate a Hadoop input file and opens it with ParquetFileReader val in = org.apache.parquet.hadoop.util.HadoopInputFile.fromPath(new org.apache.hadoop.fs.Path(fullPathUri), spark.sessionState.newHadoopConf()) val pf = org.apache.parquet.hadoop.ParquetFileReader.open(in) // Get the Parquet file version pf.getFooter.getFileMetaData.getCreatedBy // columns index val columnIndex = pf.readColumnIndex(columns.get(0)) columnIndex.toString.foreach(print) // offset index pf.readOffsetIndex(columns.get(0)) print(pf.readOffsetIndex(columns.get(0))) </code></pre><div class="zeroclipboard-container" style="animation: auto ease 0s 1 normal none running none; box-sizing: border-box;"> </div> </div> </li> </ul><p dir="auto" style="background-color: white; box-sizing: border-box; color: #1f2328; font-family: -apple-system, BlinkMacSystemFont, &quot;Segoe UI&quot;, &quot;Noto Sans&quot;, Helvetica, Arial, sans-serif, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;; font-size: 16px; margin-bottom: 16px; margin-top: 0px;">The output on a column that is sorted looks like:</p> <div class="snippet-clipboard-content notranslate position-relative overflow-auto" style="box-sizing: border-box; color: #1f2328; display: flex; font-family: -apple-system, BlinkMacSystemFont, &quot;Segoe UI&quot;, &quot;Noto Sans&quot;, Helvetica, Arial, sans-serif, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;; font-size: 16px; justify-content: space-between; margin-bottom: 16px; overflow: auto; position: relative;"> <pre class="notranslate" style="border-radius: 6px; box-sizing: border-box; color: var(--fgColor-default, var(--color-fg-default)); font-family: ui-monospace, SFMono-Regular, &quot;SF Mono&quot;, Menlo, Consolas, &quot;Liberation Mono&quot;, monospace; font-size: 13.6px; line-height: 1.45; margin-bottom: 0px; margin-top: 0px; overflow-wrap: normal; overflow: auto; padding: 16px;"> <code style="background-attachment: initial; background-clip: initial; background-image: initial; background-origin: initial; background-position: initial; background-repeat: initial; background-size: initial; border-radius: 6px; border: 0px; box-sizing: border-box; display: inline; font-family: ui-monospace, SFMono-Regular, &quot;SF Mono&quot;, Menlo, Consolas, &quot;Liberation Mono&quot;, monospace; line-height: inherit; margin: 0px; overflow-wrap: normal; overflow: visible; padding: 0px; word-break: normal;">row-group 0: column index for column ws_sold_time_sk: Boudary order: ASCENDING null count min max page-0 45 29 12320 page-1 0 12320 19782 page-2 0 19782 26385 page-3 0 26385 31758 page-4 0 31758 36234 page-5 0 36234 40492 page-6 0 40492 44417 page-7 0 44417 47596 page-8 0 47596 52972 page-9 0 52972 58388 page-10 0 58388 62482 page-11 0 62482 65804 page-12 0 65804 68647 page-13 0 68647 71299 page-14 0 71303 74231 page-15 0 74231 77978 page-16 0 77978 85712 page-17 0 85712 86399 offset index for column ws_sold_time_sk: offset compressed size first row index page-0 94906 4759 0 page-1 99665 4601 20000 page-2 104266 4549 40000 page-3 108815 4415 60000 page-4 113230 4343 80000 page-5 117573 4345 100000 page-6 121918 4205 120000 page-7 126123 3968 140000 page-8 130091 4316 160000 page-9 134407 4370 180000 page-10 138777 4175 200000 page-11 142952 4012 220000 page-12 146964 3878 240000 page-13 150842 3759 260000 page-14 154601 3888 280000 page-15 158489 4048 300000 page-16 162537 4444 320000 page-17 166981 200 340000 </code> </pre><p> </p> <div class="zeroclipboard-container" style="animation: auto ease 0s 1 normal none running none; box-sizing: border-box;"> </div> </div> <h2 dir="auto" style="background-color: white; border-bottom: 1px solid var(--borderColor-muted, var(--color-border-muted)); box-sizing: border-box; color: #1f2328; font-family: -apple-system, BlinkMacSystemFont, &quot;Segoe UI&quot;, &quot;Noto Sans&quot;, Helvetica, Arial, sans-serif, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;; font-weight: var(--base-text-weight-semibold, 600); line-height: 1.25; margin-bottom: 16px; margin-top: 24px; padding-bottom: 0.3em;" tabindex="-1"><span style="color: red;">Bloom filters in Parquet</span></h2> <p dir="auto" style="background-color: white; box-sizing: border-box; color: #1f2328; font-family: -apple-system, BlinkMacSystemFont, &quot;Segoe UI&quot;, &quot;Noto Sans&quot;, Helvetica, Arial, sans-serif, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;; font-size: 16px; margin-bottom: 16px; margin-top: 0px;">With the release of Parquet 1.12, there's now the capability to generate and store Bloom filters within the file footer's metadata. This addition significantly enhances query performance for specific filtering operations. Bloom filters are especially advantageous in the following scenarios:</p> <p dir="auto" style="background-color: white; box-sizing: border-box; color: #1f2328; font-family: -apple-system, BlinkMacSystemFont, &quot;Segoe UI&quot;, &quot;Noto Sans&quot;, Helvetica, Arial, sans-serif, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;; font-size: 16px; margin-bottom: 16px; margin-top: 0px;"><span style="box-sizing: border-box; font-weight: var(--base-text-weight-semibold, 600);">High Cardinality Columns:</span> They effectively address the limitations inherent in using Parquet dictionaries for columns with a vast range of unique values.</p> <p dir="auto" style="background-color: white; box-sizing: border-box; color: #1f2328; font-family: -apple-system, BlinkMacSystemFont, &quot;Segoe UI&quot;, &quot;Noto Sans&quot;, Helvetica, Arial, sans-serif, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;; font-size: 16px; margin-bottom: 16px; margin-top: 0px;"><span style="box-sizing: border-box; font-weight: var(--base-text-weight-semibold, 600);">Absent Value Filtering:</span> Bloom filters are highly efficient for queries that filter based on values likely to be missing from the table or DataFrame. This efficiency stems from the characteristic of Bloom filters where false positives (erroneously concluding that a non-existent value is present) are possible, but false negatives (failing to identify an existing value) do not occur.</p> <p dir="auto" style="background-color: white; box-sizing: border-box; color: #1f2328; font-family: -apple-system, BlinkMacSystemFont, &quot;Segoe UI&quot;, &quot;Noto Sans&quot;, Helvetica, Arial, sans-serif, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;; font-size: 16px; margin-bottom: 16px; margin-top: 0px;">For a comprehensive understanding and technical details of implementing Bloom filters in Apache Parquet, refer to the <a href="https://github.com/apache/parquet-format/blob/master/BloomFilter.md" style="background-color: transparent; box-sizing: border-box; text-underline-offset: 0.2rem;">official documentation on bloom filters in Apache Parquet</a></p> <p dir="auto" style="background-color: white; box-sizing: border-box; color: #1f2328; font-family: -apple-system, BlinkMacSystemFont, &quot;Segoe UI&quot;, &quot;Noto Sans&quot;, Helvetica, Arial, sans-serif, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;; font-size: 16px; margin-bottom: 16px; margin-top: 0px;"><span style="font-size: 1.25em; font-weight: var(--base-text-weight-semibold, 600);"></span></p> <p dir="auto" style="background-color: white; box-sizing: border-box; color: #1f2328; font-family: -apple-system, BlinkMacSystemFont, &quot;Segoe UI&quot;, &quot;Noto Sans&quot;, Helvetica, Arial, sans-serif, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;; font-size: 16px; margin-bottom: 16px; margin-top: 0px;"><span style="font-size: 1.25em; font-weight: var(--base-text-weight-semibold, 600);">Configuration</span></p> <p dir="auto" style="background-color: white; box-sizing: border-box; color: #1f2328; font-family: -apple-system, BlinkMacSystemFont, &quot;Segoe UI&quot;, &quot;Noto Sans&quot;, Helvetica, Arial, sans-serif, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;; font-size: 16px; margin-bottom: 16px; margin-top: 0px;">Important configurations for writing bloom filters in Parquet files are:</p> <div class="snippet-clipboard-content notranslate position-relative overflow-auto" style="box-sizing: border-box; color: #1f2328; display: flex; font-family: -apple-system, BlinkMacSystemFont, &quot;Segoe UI&quot;, &quot;Noto Sans&quot;, Helvetica, Arial, sans-serif, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;; font-size: 16px; justify-content: space-between; margin-bottom: 16px; overflow: auto; position: relative;"> <pre class="notranslate" style="border-radius: 6px; box-sizing: border-box; color: var(--fgColor-default, var(--color-fg-default)); font-family: ui-monospace, SFMono-Regular, &quot;SF Mono&quot;, Menlo, Consolas, &quot;Liberation Mono&quot;, monospace; font-size: 13.6px; line-height: 1.45; margin-bottom: 0px; margin-top: 0px; overflow-wrap: normal; overflow: auto; padding: 16px;"> <code style="background-attachment: initial; background-clip: initial; background-image: initial; background-origin: initial; background-position: initial; background-repeat: initial; background-size: initial; border-radius: 6px; border: 0px; box-sizing: border-box; display: inline; font-family: ui-monospace, SFMono-Regular, &quot;SF Mono&quot;, Menlo, Consolas, &quot;Liberation Mono&quot;, monospace; line-height: inherit; margin: 0px; overflow-wrap: normal; overflow: visible; padding: 0px; word-break: normal;">.option("parquet.bloom.filter.enabled","true") // write bloom filters for all columns, default is false .option("parquet.bloom.filter.enabled#column_name", "true") // write bloom filter for the given column .option("parquet.bloom.filter.expected.ndv#column_name", num_values) // tuning for bloom filters, ndv = number of distinct values .option("parquet.bloom.filter.max.bytes", 1024*1024) // The maximum number of bytes for a bloom filter bitset, default 1 MB </code></pre><div class="zeroclipboard-container" style="animation: auto ease 0s 1 normal none running none; box-sizing: border-box;"> </div> </div> <h3 dir="auto" style="background-color: white; box-sizing: border-box; color: #1f2328; font-family: -apple-system, BlinkMacSystemFont, &quot;Segoe UI&quot;, &quot;Noto Sans&quot;, Helvetica, Arial, sans-serif, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;; font-size: 1.25em; font-weight: var(--base-text-weight-semibold, 600); line-height: 1.25; margin-bottom: 16px; margin-top: 24px;" tabindex="-1"><span style="font-size: 1.25em; font-weight: var(--base-text-weight-semibold, 600);"></span></h3> <h3 dir="auto" style="background-color: white; box-sizing: border-box; font-family: -apple-system, BlinkMacSystemFont, &quot;Segoe UI&quot;, &quot;Noto Sans&quot;, Helvetica, Arial, sans-serif, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;; font-size: 1.25em; font-weight: var(--base-text-weight-semibold, 600); line-height: 1.25; margin-bottom: 16px; margin-top: 24px;" tabindex="-1"><span style="color: red; font-size: 1.25em; font-weight: var(--base-text-weight-semibold, 600);">Write Parquet files with Bloom filters</span></h3> <p dir="auto" style="background-color: white; box-sizing: border-box; color: #1f2328; font-family: -apple-system, BlinkMacSystemFont, &quot;Segoe UI&quot;, &quot;Noto Sans&quot;, Helvetica, Arial, sans-serif, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;; font-size: 16px; margin-bottom: 16px; margin-top: 0px;">This is an example of how to read a Parquet file without bloom filter (for example because it had been created with an older version of Spark/Parquet) and add the bloom filter, with additional tuning of the bloom filter parameters for one of the columns:</p> <div class="snippet-clipboard-content notranslate position-relative overflow-auto" style="box-sizing: border-box; color: #1f2328; display: flex; font-family: -apple-system, BlinkMacSystemFont, &quot;Segoe UI&quot;, &quot;Noto Sans&quot;, Helvetica, Arial, sans-serif, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;; font-size: 16px; justify-content: space-between; margin-bottom: 16px; overflow: auto; position: relative;"> <pre class="notranslate" style="border-radius: 6px; box-sizing: border-box; color: var(--fgColor-default, var(--color-fg-default)); font-family: ui-monospace, SFMono-Regular, &quot;SF Mono&quot;, Menlo, Consolas, &quot;Liberation Mono&quot;, monospace; font-size: 13.6px; line-height: 1.45; margin-bottom: 0px; margin-top: 0px; overflow-wrap: normal; overflow: auto; padding: 16px;"> <code style="background-attachment: initial; background-clip: initial; background-image: initial; background-origin: initial; background-position: initial; background-repeat: initial; background-size: initial; border-radius: 6px; border: 0px; box-sizing: border-box; display: inline; font-family: ui-monospace, SFMono-Regular, &quot;SF Mono&quot;, Menlo, Consolas, &quot;Liberation Mono&quot;, monospace; line-height: inherit; margin: 0px; overflow-wrap: normal; overflow: visible; padding: 0px; word-break: normal;">val df = spark.read.parquet("&lt;path&gt;/web_sales") df.coalesce(1).write.option("parquet.bloom.filter.enabled","true").option("parquet.bloom.filter.expected.ndv#ws_sold_time_sk", 25000).parquet("&lt;myfilepath") </code></pre><div class="zeroclipboard-container" style="animation: auto ease 0s 1 normal none running none; box-sizing: border-box;"> </div> </div> <h3 dir="auto" style="background-color: white; box-sizing: border-box; color: #1f2328; font-family: -apple-system, BlinkMacSystemFont, &quot;Segoe UI&quot;, &quot;Noto Sans&quot;, Helvetica, Arial, sans-serif, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;; font-size: 1.25em; font-weight: var(--base-text-weight-semibold, 600); line-height: 1.25; margin-bottom: 16px; margin-top: 24px;" tabindex="-1"><span style="font-size: 1.25em; font-weight: var(--base-text-weight-semibold, 600);"></span></h3> <h3 dir="auto" style="background-color: white; box-sizing: border-box; font-family: -apple-system, BlinkMacSystemFont, &quot;Segoe UI&quot;, &quot;Noto Sans&quot;, Helvetica, Arial, sans-serif, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;; font-size: 1.25em; font-weight: var(--base-text-weight-semibold, 600); line-height: 1.25; margin-bottom: 16px; margin-top: 24px;" tabindex="-1"><span style="color: red; font-size: 1.25em; font-weight: var(--base-text-weight-semibold, 600);">Example: Checking I/O Performance in Parquet: With and Without Bloom Filters</span></h3> <p dir="auto" style="background-color: white; box-sizing: border-box; color: #1f2328; font-family: -apple-system, BlinkMacSystemFont, &quot;Segoe UI&quot;, &quot;Noto Sans&quot;, Helvetica, Arial, sans-serif, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;; font-size: 16px; margin-bottom: 16px; margin-top: 0px;">Understanding the impact of using bloom filters on I/O performance during Parquet file reads can be important for optimizing data processing. This example outlines the steps to compare I/O performance when reading Parquet files, both with and without the utilization of bloom filters.</p> <p dir="auto" style="background-color: white; box-sizing: border-box; color: #1f2328; font-family: -apple-system, BlinkMacSystemFont, &quot;Segoe UI&quot;, &quot;Noto Sans&quot;, Helvetica, Arial, sans-serif, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;; font-size: 16px; margin-bottom: 16px; margin-top: 0px;"><span style="box-sizing: border-box; font-weight: var(--base-text-weight-semibold, 600);">This example uses Parquet bloom filters to improve Spark read performance</span></p> <p dir="auto" style="background-color: white; box-sizing: border-box; color: #1f2328; font-family: -apple-system, BlinkMacSystemFont, &quot;Segoe UI&quot;, &quot;Noto Sans&quot;, Helvetica, Arial, sans-serif, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;; font-size: 16px; margin-bottom: 16px; margin-top: 0px;"><span style="box-sizing: border-box; font-weight: var(--base-text-weight-semibold, 600);">1. Prepare the test table</span></p> <div class="snippet-clipboard-content notranslate position-relative overflow-auto" style="box-sizing: border-box; color: #1f2328; display: flex; font-family: -apple-system, BlinkMacSystemFont, &quot;Segoe UI&quot;, &quot;Noto Sans&quot;, Helvetica, Arial, sans-serif, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;; font-size: 16px; justify-content: space-between; margin-bottom: 16px; overflow: auto; position: relative;"> <pre class="notranslate" style="border-radius: 6px; box-sizing: border-box; color: var(--fgColor-default, var(--color-fg-default)); font-family: ui-monospace, SFMono-Regular, &quot;SF Mono&quot;, Menlo, Consolas, &quot;Liberation Mono&quot;, monospace; font-size: 13.6px; line-height: 1.45; margin-bottom: 0px; margin-top: 0px; overflow-wrap: normal; overflow: auto; padding: 16px;"> <code style="background-attachment: initial; background-clip: initial; background-image: initial; background-origin: initial; background-position: initial; background-repeat: initial; background-size: initial; border-radius: 6px; border: 0px; box-sizing: border-box; display: inline; font-family: ui-monospace, SFMono-Regular, &quot;SF Mono&quot;, Menlo, Consolas, &quot;Liberation Mono&quot;, monospace; line-height: inherit; margin: 0px; overflow-wrap: normal; overflow: visible; padding: 0px; word-break: normal;"> bin/spark-shell val numDistinctVals=1e6.toInt val df=sql(s"select id, int(random()*100*$numDistinctVals) randomval from range($numDistinctVals)") val path = "./" // Write the test DataFrame into a Parquet file with a Bloom filter df.coalesce(1).write.mode("overwrite").option("parquet.bloom.filter.enabled","true").option("parquet.bloom.filter.enabled#randomval", "true").option("parquet.bloom.filter.expected.ndv#randomval", numDistinctVals).parquet(path + "spark320_test_bloomfilter") // Write the same DataFrame in Parquet, but this time without Bloom filters df.coalesce(1).write.mode("overwrite").option("parquet.bloom.filter.enabled","false").parquet(path + "spark320_test_bloomfilter_nofilter") // use the OS (ls -l) to compare the size of the files with bloom filter and without // in my test (Spark 3.5.0, Parquet 1.13.1) it was 10107275 with bloom filter and 8010077 without :quit </code></pre><div class="zeroclipboard-container" style="animation: auto ease 0s 1 normal none running none; box-sizing: border-box;"> </div> </div> <p dir="auto" style="background-color: white; box-sizing: border-box; color: #1f2328; font-family: -apple-system, BlinkMacSystemFont, &quot;Segoe UI&quot;, &quot;Noto Sans&quot;, Helvetica, Arial, sans-serif, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;; font-size: 16px; margin-bottom: 16px; margin-top: 0px;"><span style="box-sizing: border-box; font-weight: var(--base-text-weight-semibold, 600);">2. Read data using the Bloom filter, for improved performance</span></p> <div class="snippet-clipboard-content notranslate position-relative overflow-auto" style="box-sizing: border-box; color: #1f2328; display: flex; font-family: -apple-system, BlinkMacSystemFont, &quot;Segoe UI&quot;, &quot;Noto Sans&quot;, Helvetica, Arial, sans-serif, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;; font-size: 16px; justify-content: space-between; margin-bottom: 16px; overflow: auto; position: relative;"> <pre class="notranslate" style="border-radius: 6px; box-sizing: border-box; color: var(--fgColor-default, var(--color-fg-default)); font-family: ui-monospace, SFMono-Regular, &quot;SF Mono&quot;, Menlo, Consolas, &quot;Liberation Mono&quot;, monospace; font-size: 13.6px; line-height: 1.45; margin-bottom: 0px; margin-top: 0px; overflow-wrap: normal; overflow: auto; padding: 16px;"> <code style="background-attachment: initial; background-clip: initial; background-image: initial; background-origin: initial; background-position: initial; background-repeat: initial; background-size: initial; border-radius: 6px; border: 0px; box-sizing: border-box; display: inline; font-family: ui-monospace, SFMono-Regular, &quot;SF Mono&quot;, Menlo, Consolas, &quot;Liberation Mono&quot;, monospace; line-height: inherit; margin: 0px; overflow-wrap: normal; overflow: visible; padding: 0px; word-break: normal;">bin/spark-shell val path = "./" val df =spark.read.option("parquet.filter.bloom.enabled","true").parquet(path + "spark320_test_bloomfilter") val q1 = df.filter("randomval=1000000") // filter for a value that is not in the file q1.collect // print I/O metrics org.apache.hadoop.fs.FileSystem.printStatistics() // Output FileSystem org.apache.hadoop.fs.RawLocalFileSystem: 1091611 bytes read, ... :quit </code></pre><div class="zeroclipboard-container" style="animation: auto ease 0s 1 normal none running none; box-sizing: border-box;"> </div> </div> <p dir="auto" style="background-color: white; box-sizing: border-box; color: #1f2328; font-family: -apple-system, BlinkMacSystemFont, &quot;Segoe UI&quot;, &quot;Noto Sans&quot;, Helvetica, Arial, sans-serif, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;; font-size: 16px; margin-bottom: 16px; margin-top: 0px;"><span style="box-sizing: border-box; font-weight: var(--base-text-weight-semibold, 600);">3. Read disabling the Bloom filter (this will read more data from the filesystem and have worse performance)</span></p> <div class="snippet-clipboard-content notranslate position-relative overflow-auto" style="box-sizing: border-box; color: #1f2328; display: flex; font-family: -apple-system, BlinkMacSystemFont, &quot;Segoe UI&quot;, &quot;Noto Sans&quot;, Helvetica, Arial, sans-serif, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;; font-size: 16px; justify-content: space-between; margin-bottom: 16px; overflow: auto; position: relative;"> <pre class="notranslate" style="border-radius: 6px; box-sizing: border-box; color: var(--fgColor-default, var(--color-fg-default)); font-family: ui-monospace, SFMono-Regular, &quot;SF Mono&quot;, Menlo, Consolas, &quot;Liberation Mono&quot;, monospace; font-size: 13.6px; line-height: 1.45; margin-bottom: 0px; margin-top: 0px; overflow-wrap: normal; overflow: auto; padding: 16px;"> <code style="background-attachment: initial; background-clip: initial; background-image: initial; background-origin: initial; background-position: initial; background-repeat: initial; background-size: initial; border-radius: 6px; border: 0px; box-sizing: border-box; display: inline; font-family: ui-monospace, SFMono-Regular, &quot;SF Mono&quot;, Menlo, Consolas, &quot;Liberation Mono&quot;, monospace; line-height: inherit; margin: 0px; overflow-wrap: normal; overflow: visible; padding: 0px; word-break: normal;">bin/spark-shell val path = "./" val df =spark.read.option("parquet.filter.bloom.enabled","false").parquet(path + "spark320_test_bloomfilter") val q1 = df.filter("randomval=1000000") // filter for a value that is not in the file q1.collect // print I/O metrics org.apache.hadoop.fs.FileSystem.printStatistics() // Output FileSystem org.apache.hadoop.fs.RawLocalFileSystem: 8299656 bytes read, ... </code></pre><div class="zeroclipboard-container" style="animation: auto ease 0s 1 normal none running none; box-sizing: border-box;"> </div> </div> <h3 dir="auto" style="background-color: white; box-sizing: border-box; color: #1f2328; font-family: -apple-system, BlinkMacSystemFont, &quot;Segoe UI&quot;, &quot;Noto Sans&quot;, Helvetica, Arial, sans-serif, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;; font-size: 1.25em; font-weight: var(--base-text-weight-semibold, 600); line-height: 1.25; margin-bottom: 16px; margin-top: 24px;" tabindex="-1"><span style="font-size: 1.25em; font-weight: var(--base-text-weight-semibold, 600);"></span></h3> <h3 dir="auto" style="background-color: white; box-sizing: border-box; font-family: -apple-system, BlinkMacSystemFont, &quot;Segoe UI&quot;, &quot;Noto Sans&quot;, Helvetica, Arial, sans-serif, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;; font-size: 1.25em; font-weight: var(--base-text-weight-semibold, 600); line-height: 1.25; margin-bottom: 16px; margin-top: 24px;" tabindex="-1"><span style="color: red; font-size: 1.25em; font-weight: var(--base-text-weight-semibold, 600);">Reading Parquet Bloom Filter Metadata with Apache Parquet Java API</span></h3> <p dir="auto" style="background-color: white; box-sizing: border-box; color: #1f2328; font-family: -apple-system, BlinkMacSystemFont, &quot;Segoe UI&quot;, &quot;Noto Sans&quot;, Helvetica, Arial, sans-serif, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;; font-size: 16px; margin-bottom: 16px; margin-top: 0px;">To extract metadata about the bloom filter from a Parquet file using the Apache Parquet Java API in spark-shell, follow these steps:<br />  </p> <ol dir="auto" style="background-color: white; box-sizing: border-box; color: #1f2328; font-family: -apple-system, BlinkMacSystemFont, &quot;Segoe UI&quot;, &quot;Noto Sans&quot;, Helvetica, Arial, sans-serif, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;; font-size: 16px; margin-bottom: 16px; margin-top: 0px; padding-left: 2em;"><li dir="ltr" style="box-sizing: border-box;">Initialize the File Path: define the full path of your Parquet file<br /><div class="snippet-clipboard-content notranslate position-relative overflow-auto" style="background-color: var(--bgColor-muted, var(--color-canvas-subtle)); box-sizing: border-box; display: flex; justify-content: space-between; margin-bottom: 16px; overflow: auto; position: relative;"> <pre class="notranslate" style="background-color: var(--bgColor-muted, var(--color-canvas-subtle)); border-radius: 6px; box-sizing: border-box; color: var(--fgColor-default, var(--color-fg-default)); font-family: ui-monospace, SFMono-Regular, &quot;SF Mono&quot;, Menlo, Consolas, &quot;Liberation Mono&quot;, monospace; font-size: 13.6px; line-height: 1.45; margin-bottom: 0px; margin-top: 0px; overflow-wrap: normal; overflow: auto; padding: 16px;"> bin/spark-shell<code style="background: transparent; border-radius: 6px; border: 0px; box-sizing: border-box; display: inline; font-family: ui-monospace, SFMono-Regular, &quot;SF Mono&quot;, Menlo, Consolas, &quot;Liberation Mono&quot;, monospace; line-height: inherit; margin: 0px; overflow-wrap: normal; overflow: visible; padding: 0px; word-break: normal;"> </code>val fullPathUri = java.net.URI.create("&lt;my_file_path&gt;")</pre></div> </li> <li dir="ltr" style="box-sizing: border-box; margin-top: 0.25em;">Create Input File: utilize HadoopInputFile to create an input file from the specified path<br /><div class="snippet-clipboard-content notranslate position-relative overflow-auto" style="background-color: var(--bgColor-muted, var(--color-canvas-subtle)); box-sizing: border-box; display: flex; justify-content: space-between; margin-bottom: 16px; overflow: auto; position: relative;"> <pre class="notranslate" style="background-color: var(--bgColor-muted, var(--color-canvas-subtle)); border-radius: 6px; box-sizing: border-box; color: var(--fgColor-default, var(--color-fg-default)); font-family: ui-monospace, SFMono-Regular, &quot;SF Mono&quot;, Menlo, Consolas, &quot;Liberation Mono&quot;, monospace; font-size: 13.6px; line-height: 1.45; margin-bottom: 0px; margin-top: 0px; overflow-wrap: normal; overflow: auto; padding: 16px;"> <code style="background: transparent; border-radius: 6px; border: 0px; box-sizing: border-box; display: inline; font-family: ui-monospace, SFMono-Regular, &quot;SF Mono&quot;, Menlo, Consolas, &quot;Liberation Mono&quot;, monospace; line-height: inherit; margin: 0px; overflow-wrap: normal; overflow: visible; padding: 0px; word-break: normal;">val in = org.apache.parquet.hadoop.util.HadoopInputFile.fromPath( new org.apache.hadoop.fs.Path(fullPathUri), spark.sessionState.newHadoopConf() ) </code></pre><div class="zeroclipboard-container" style="animation: auto ease 0s 1 normal none running none; box-sizing: border-box;"> </div> </div> </li> <li dir="ltr" style="box-sizing: border-box; margin-top: 0.25em;">Open Parquet File Reader: open the Parquet file reader for the input file<br /><div class="snippet-clipboard-content notranslate position-relative overflow-auto" style="background-color: var(--bgColor-muted, var(--color-canvas-subtle)); box-sizing: border-box; display: flex; justify-content: space-between; margin-bottom: 16px; overflow: auto; position: relative;"> <pre class="notranslate" style="background-color: var(--bgColor-muted, var(--color-canvas-subtle)); border-radius: 6px; box-sizing: border-box; color: var(--fgColor-default, var(--color-fg-default)); font-family: ui-monospace, SFMono-Regular, &quot;SF Mono&quot;, Menlo, Consolas, &quot;Liberation Mono&quot;, monospace; font-size: 13.6px; line-height: 1.45; margin-bottom: 0px; margin-top: 0px; overflow-wrap: normal; overflow: auto; padding: 16px;"> <code style="background: transparent; border-radius: 6px; border: 0px; box-sizing: border-box; display: inline; font-family: ui-monospace, SFMono-Regular, &quot;SF Mono&quot;, Menlo, Consolas, &quot;Liberation Mono&quot;, monospace; line-height: inherit; margin: 0px; overflow-wrap: normal; overflow: visible; padding: 0px; word-break: normal;">val pf = org.apache.parquet.hadoop.ParquetFileReader.open(in) </code></pre><div class="zeroclipboard-container" style="animation: auto ease 0s 1 normal none running none; box-sizing: border-box;"> </div> </div> </li> <li dir="ltr" style="box-sizing: border-box; margin-top: 0.25em;">Retrieve Blocks and Columns: extract the blocks from the file footer and then get the columns from the first block<br /><div class="snippet-clipboard-content notranslate position-relative overflow-auto" style="background-color: var(--bgColor-muted, var(--color-canvas-subtle)); box-sizing: border-box; display: flex; justify-content: space-between; margin-bottom: 16px; overflow: auto; position: relative;"> <pre class="notranslate" style="background-color: var(--bgColor-muted, var(--color-canvas-subtle)); border-radius: 6px; box-sizing: border-box; color: var(--fgColor-default, var(--color-fg-default)); font-family: ui-monospace, SFMono-Regular, &quot;SF Mono&quot;, Menlo, Consolas, &quot;Liberation Mono&quot;, monospace; font-size: 13.6px; line-height: 1.45; margin-bottom: 0px; margin-top: 0px; overflow-wrap: normal; overflow: auto; padding: 16px;"> <code style="background: transparent; border-radius: 6px; border: 0px; box-sizing: border-box; display: inline; font-family: ui-monospace, SFMono-Regular, &quot;SF Mono&quot;, Menlo, Consolas, &quot;Liberation Mono&quot;, monospace; line-height: inherit; margin: 0px; overflow-wrap: normal; overflow: visible; padding: 0px; word-break: normal;">val blocks = pf.getFooter.getBlocks val columns = blocks.get(0).getColumns </code></pre><div class="zeroclipboard-container" style="animation: auto ease 0s 1 normal none running none; box-sizing: border-box;"> </div> </div> </li> <li dir="ltr" style="box-sizing: border-box; margin-top: 0.25em;">Read Bloom Filter: finally, read the bloom filter from the first column<br /><div class="snippet-clipboard-content notranslate position-relative overflow-auto" style="background-color: var(--bgColor-muted, var(--color-canvas-subtle)); box-sizing: border-box; display: flex; justify-content: space-between; margin-bottom: 16px; overflow: auto; position: relative;"> <pre class="notranslate" style="background-color: var(--bgColor-muted, var(--color-canvas-subtle)); border-radius: 6px; box-sizing: border-box; color: var(--fgColor-default, var(--color-fg-default)); font-family: ui-monospace, SFMono-Regular, &quot;SF Mono&quot;, Menlo, Consolas, &quot;Liberation Mono&quot;, monospace; font-size: 13.6px; line-height: 1.45; margin-bottom: 0px; margin-top: 0px; overflow-wrap: normal; overflow: auto; padding: 16px;"> val bloomFilter = pf.readBloomFilter(columns.get(0)) bloomFilter.getBitsetSize</pre><div class="zeroclipboard-container" style="animation: auto ease 0s 1 normal none running none; box-sizing: border-box;"> </div> </div> </li> </ol><p dir="auto" style="background-color: white; box-sizing: border-box; color: #1f2328; font-family: -apple-system, BlinkMacSystemFont, &quot;Segoe UI&quot;, &quot;Noto Sans&quot;, Helvetica, Arial, sans-serif, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;; font-size: 16px; margin-bottom: 16px; margin-top: 0px;">By following these steps, you can successfully read the bloom filter metadata from a Parquet file using the Java API in the spark-shell environment.</p> <p dir="auto" style="background-color: white; box-sizing: border-box; color: #1f2328; font-family: -apple-system, BlinkMacSystemFont, &quot;Segoe UI&quot;, &quot;Noto Sans&quot;, Helvetica, Arial, sans-serif, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;; font-size: 16px; margin-bottom: 16px; margin-top: 0px;"> </p> <h2 style="background-color: white; box-sizing: border-box; font-family: -apple-system, BlinkMacSystemFont, &quot;Segoe UI&quot;, &quot;Noto Sans&quot;, Helvetica, Arial, sans-serif, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;; font-weight: var(--base-text-weight-semibold, 600); line-height: 1.25; margin-bottom: 16px; margin-top: 24px; text-align: left;"><span style="color: red; font-size: large;"><span style="color: red; font-size: 1.25em; font-weight: var(--base-text-weight-semibold, 600);">Discovering Parquet Version</span></span></h2> <p dir="auto" style="background-color: white; box-sizing: border-box; color: #1f2328; font-family: -apple-system, BlinkMacSystemFont, &quot;Segoe UI&quot;, &quot;Noto Sans&quot;, Helvetica, Arial, sans-serif, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;; font-size: 16px; margin-bottom: 16px; margin-top: 0px;">The Parquet file format is constantly evolving, incorporating additional metadata to support emerging features. Each Parquet file embeds the version information within its metadata, reflecting the Parquet version used during its creation.</p> <p dir="auto" style="background-color: white; box-sizing: border-box; color: #1f2328; font-family: -apple-system, BlinkMacSystemFont, &quot;Segoe UI&quot;, &quot;Noto Sans&quot;, Helvetica, Arial, sans-serif, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;; font-size: 16px; margin-bottom: 16px; margin-top: 0px;"><span style="box-sizing: border-box; font-weight: var(--base-text-weight-semibold, 600);">Importance of Version Awareness:</span></p> <p dir="auto" style="background-color: white; box-sizing: border-box; color: #1f2328; font-family: -apple-system, BlinkMacSystemFont, &quot;Segoe UI&quot;, &quot;Noto Sans&quot;, Helvetica, Arial, sans-serif, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;; font-size: 16px; margin-bottom: 16px; margin-top: 0px;"><span style="box-sizing: border-box; font-weight: var(--base-text-weight-semibold, 600);">Compatibility Considerations:</span> When working with Parquet files generated by older versions of Spark and its corresponding Parquet library, it's important to be aware that certain newer features may not be supported. For instance, column indexes, which are available in the Spark DataFrame Parquet writer from version 3.2.0, might not be present in files created with older versions.</p> <p dir="auto" style="background-color: white; box-sizing: border-box; color: #1f2328; font-family: -apple-system, BlinkMacSystemFont, &quot;Segoe UI&quot;, &quot;Noto Sans&quot;, Helvetica, Arial, sans-serif, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;; font-size: 16px; margin-bottom: 16px; margin-top: 0px;"><span style="box-sizing: border-box; font-weight: var(--base-text-weight-semibold, 600);">Upgrading for Enhanced Features:</span> Upon upgrading your Spark version, it's beneficial to also update the metadata in existing Parquet files. This update allows you to utilize the latest features introduced in newer versions of Parquet.</p> <h4 dir="auto" style="background-color: white; box-sizing: border-box; color: #1f2328; font-family: -apple-system, BlinkMacSystemFont, &quot;Segoe UI&quot;, &quot;Noto Sans&quot;, Helvetica, Arial, sans-serif, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;; font-size: 16px; font-weight: var(--base-text-weight-semibold, 600); line-height: 1.25; margin-bottom: 16px; margin-top: 24px;" tabindex="-1"><a aria-hidden="true" class="anchor" href="https://github.com/LucaCanali/Miscellaneous/blob/master/Spark_Notes/Spark_Parquet.md#checking-the-parquet-file-version" id="user-content-checking-the-parquet-file-version" style="background-color: transparent; box-sizing: border-box; float: left; line-height: 1; margin-left: -20px; padding-right: 4px; position: absolute; text-decoration-line: none; text-underline-offset: 0.2rem;" tabindex="-1"> <svg aria-hidden="true" class="octicon octicon-link" height="16" version="1.1" viewbox="0 0 16 16" width="16"></svg><p></p></a></h4> <h4 dir="auto" style="background-color: white; box-sizing: border-box; color: #1f2328; font-family: -apple-system, BlinkMacSystemFont, &quot;Segoe UI&quot;, &quot;Noto Sans&quot;, Helvetica, Arial, sans-serif, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;; font-size: 16px; font-weight: var(--base-text-weight-semibold, 600); line-height: 1.25; margin-bottom: 16px; margin-top: 24px;" tabindex="-1">Checking the Parquet File Version:</h4> <p dir="auto" style="background-color: white; box-sizing: border-box; color: #1f2328; font-family: -apple-system, BlinkMacSystemFont, &quot;Segoe UI&quot;, &quot;Noto Sans&quot;, Helvetica, Arial, sans-serif, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;; font-size: 16px; margin-bottom: 16px; margin-top: 0px;">The following sections will guide you on how to check the Parquet version used in your files, ensuring that you can effectively manage and upgrade your Parquet datasets. This format provides a structured and detailed approach to understanding and managing Parquet file versions, emphasizing the importance of version compatibility and the process of upgrading.</p> <ul dir="auto" style="background-color: white; box-sizing: border-box; color: #1f2328; font-family: -apple-system, BlinkMacSystemFont, &quot;Segoe UI&quot;, &quot;Noto Sans&quot;, Helvetica, Arial, sans-serif, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;; font-size: 16px; margin-bottom: 16px; margin-top: 0px; padding-left: 2em;"><li style="box-sizing: border-box;"> <p dir="auto" style="box-sizing: border-box; margin-bottom: 16px; margin-top: 16px;">Details at <a href="https://github.com/LucaCanali/Miscellaneous/blob/master/Spark_Notes/Tools_Parquet_Diagnostics.md" style="background-color: transparent; box-sizing: border-box; text-underline-offset: 0.2rem;"><span style="box-sizing: border-box; font-weight: var(--base-text-weight-semibold, 600);">Tools for Parquet Diagnostics</span></a></p> </li> <li style="box-sizing: border-box; margin-top: 0.25em;"> <p dir="auto" style="box-sizing: border-box; margin-bottom: 16px; margin-top: 16px;"><span style="box-sizing: border-box; font-weight: var(--base-text-weight-semibold, 600);">parquet-cli</span></p> <ul dir="auto" style="box-sizing: border-box; margin-bottom: 0px; margin-top: 0px; padding-left: 2em;"><li style="box-sizing: border-box;">example: <code style="background-color: var(--bgColor-neutral-muted, var(--color-neutral-muted)); border-radius: 6px; box-sizing: border-box; font-family: ui-monospace, SFMono-Regular, &quot;SF Mono&quot;, Menlo, Consolas, &quot;Liberation Mono&quot;, monospace; font-size: 13.6px; margin: 0px; padding: 0.2em 0.4em; white-space-collapse: break-spaces;">hadoop jar parquet-cli/target/parquet-cli-1.13.1-runtime.jar org.apache.parquet.cli.Main meta &lt;path&gt;/myParquetFile</code></li> </ul></li> <li style="box-sizing: border-box; margin-top: 0.25em;"> <p dir="auto" style="box-sizing: border-box; margin-bottom: 16px; margin-top: 16px;"><span style="box-sizing: border-box; font-weight: var(--base-text-weight-semibold, 600);">Hadoop API</span> ...</p> <ul dir="auto" style="box-sizing: border-box; margin-bottom: 0px; margin-top: 0px; padding-left: 2em;"><li style="box-sizing: border-box;">example of using Hadoop API from the spark-shell CLI</li> </ul><div class="snippet-clipboard-content notranslate position-relative overflow-auto" style="background-color: var(--bgColor-muted, var(--color-canvas-subtle)); box-sizing: border-box; display: flex; justify-content: space-between; margin-bottom: 16px; overflow: auto; position: relative;"> <pre class="notranslate" style="background-color: var(--bgColor-muted, var(--color-canvas-subtle)); border-radius: 6px; box-sizing: border-box; color: var(--fgColor-default, var(--color-fg-default)); font-family: ui-monospace, SFMono-Regular, &quot;SF Mono&quot;, Menlo, Consolas, &quot;Liberation Mono&quot;, monospace; font-size: 13.6px; line-height: 1.45; margin-bottom: 0px; margin-top: 0px; overflow-wrap: normal; overflow: auto; padding: 16px;"> <code style="background: transparent; border-radius: 6px; border: 0px; box-sizing: border-box; display: inline; font-family: ui-monospace, SFMono-Regular, &quot;SF Mono&quot;, Menlo, Consolas, &quot;Liberation Mono&quot;, monospace; line-height: inherit; margin: 0px; overflow-wrap: normal; overflow: visible; padding: 0px; word-break: normal;">// customize with the file path and name val fullPathUri = java.net.URI.create("&lt;path&gt;/myParquetFile") // crate a Hadoop input file and opens it with ParquetFileReader val in = org.apache.parquet.hadoop.util.HadoopInputFile.fromPath(new org.apache.hadoop.fs.Path(fullPathUri), spark.sessionState.newHadoopConf()) val pf = org.apache.parquet.hadoop.ParquetFileReader.open(in) // Get the Parquet file version pf.getFooter.getFileMetaData.getCreatedBy // Info on file metadata print(pf.getFileMetaData) print(pf.getRowGroups) </code></pre><div class="zeroclipboard-container" style="animation: auto ease 0s 1 normal none running none; box-sizing: border-box;"> </div> </div> </li> </ul><ul><li style="box-sizing: border-box; margin-top: 0.25em;"> <p dir="auto" style="box-sizing: border-box; margin-bottom: 16px; margin-top: 16px;"><span style="box-sizing: border-box; font-weight: var(--base-text-weight-semibold, 600);">Spark extension library</span><br style="box-sizing: border-box;" /><br /> The <a href="https://github.com/G-Research/spark-extension/blob/master/PARQUET.md" style="background-color: transparent; box-sizing: border-box; text-underline-offset: 0.2rem;">spark-extension library</a> allows to query Parquet metadata using Apache Spark.<br style="box-sizing: border-box;" /><br /> Example:</p> <div class="snippet-clipboard-content notranslate position-relative overflow-auto" style="background-color: var(--bgColor-muted, var(--color-canvas-subtle)); box-sizing: border-box; display: flex; justify-content: space-between; margin-bottom: 16px; overflow: auto; position: relative;"> <pre class="notranslate" style="background-color: var(--bgColor-muted, var(--color-canvas-subtle)); border-radius: 6px; box-sizing: border-box; color: var(--fgColor-default, var(--color-fg-default)); font-family: ui-monospace, SFMono-Regular, &quot;SF Mono&quot;, Menlo, Consolas, &quot;Liberation Mono&quot;, monospace; font-size: 13.6px; line-height: 1.45; margin-bottom: 0px; margin-top: 0px; overflow-wrap: normal; overflow: auto; padding: 16px;"> <code style="background: transparent; border-radius: 6px; border: 0px; box-sizing: border-box; display: inline; font-family: ui-monospace, SFMono-Regular, &quot;SF Mono&quot;, Menlo, Consolas, &quot;Liberation Mono&quot;, monospace; line-height: inherit; margin: 0px; overflow-wrap: normal; overflow: visible; padding: 0px; word-break: normal;">bin/spark-shell --packages uk.co.gresearch.spark:spark-extension_2.12:2.11.0-3.5 import uk.co.gresearch.spark.parquet._ spark.read.parquetMetadata("...path..").show() spark.read.parquetBlockColumns(...path..").show() </code></pre><div class="zeroclipboard-container" style="animation: auto ease 0s 1 normal none running none; box-sizing: border-box;"> </div> </div> </li> </ul><h3 dir="auto" style="background-color: white; box-sizing: border-box; font-family: -apple-system, BlinkMacSystemFont, &quot;Segoe UI&quot;, &quot;Noto Sans&quot;, Helvetica, Arial, sans-serif, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;; font-size: 1.25em; font-weight: var(--base-text-weight-semibold, 600); line-height: 1.25; margin-bottom: 16px; margin-top: 24px;" tabindex="-1"><span style="font-size: 1.25em; font-weight: var(--base-text-weight-semibold, 600);"><span style="color: red;">Updating Parquet File Versions</span></span></h3> <p dir="auto" style="background-color: white; box-sizing: border-box; color: #1f2328; font-family: -apple-system, BlinkMacSystemFont, &quot;Segoe UI&quot;, &quot;Noto Sans&quot;, Helvetica, Arial, sans-serif, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;; font-size: 16px; margin-bottom: 16px; margin-top: 0px;">Upgrading your Parquet files to a newer version can be achieved by copying them using a more recent version of Spark. This section covers the steps to convert your Parquet files to an updated version.</p> <p dir="auto" style="background-color: white; box-sizing: border-box; color: #1f2328; font-family: -apple-system, BlinkMacSystemFont, &quot;Segoe UI&quot;, &quot;Noto Sans&quot;, Helvetica, Arial, sans-serif, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;; font-size: 16px; margin-bottom: 16px; margin-top: 0px;">Conversion Method:</p> <p dir="auto" style="background-color: white; box-sizing: border-box; color: #1f2328; font-family: -apple-system, BlinkMacSystemFont, &quot;Segoe UI&quot;, &quot;Noto Sans&quot;, Helvetica, Arial, sans-serif, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;; font-size: 16px; margin-bottom: 16px; margin-top: 0px;"><span style="box-sizing: border-box; font-weight: var(--base-text-weight-semibold, 600);">Using Recent Spark Versions:</span> To update Parquet files, read them with a newer version of Spark and then save them again. This process effectively updates the files to the Parquet version used by that Spark release.<br style="box-sizing: border-box;" /><br /> For instance, using Spark 3.5.0 will allow you to write files in Parquet version 1.13.1.</p> <p dir="auto" style="background-color: white; box-sizing: border-box; color: #1f2328; font-family: -apple-system, BlinkMacSystemFont, &quot;Segoe UI&quot;, &quot;Noto Sans&quot;, Helvetica, Arial, sans-serif, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;; font-size: 16px; margin-bottom: 16px; margin-top: 0px;"><span style="box-sizing: border-box; font-weight: var(--base-text-weight-semibold, 600);">Approach Note:</span> This method is somewhat brute-force as there isn't a direct mechanism solely for upgrading Parquet metadata.</p> <p dir="auto" style="background-color: white; box-sizing: border-box; color: #1f2328; font-family: -apple-system, BlinkMacSystemFont, &quot;Segoe UI&quot;, &quot;Noto Sans&quot;, Helvetica, Arial, sans-serif, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;; font-size: 16px; margin-bottom: 16px; margin-top: 0px;"><span style="box-sizing: border-box; font-weight: var(--base-text-weight-semibold, 600);">Practical Example:</span> Copying and converting Parquet version by reading and re-writing, applied to the TPCDS benchmark:</p> <div class="snippet-clipboard-content notranslate position-relative overflow-auto" style="-webkit-text-stroke-width: 0px; background-color: var(--bgColor-muted, var(--color-canvas-subtle)); box-sizing: border-box; color: #1f2328; display: flex; font-family: -apple-system, BlinkMacSystemFont, &quot;Segoe UI&quot;, &quot;Noto Sans&quot;, Helvetica, Arial, sans-serif, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;; font-size: 16px; font-style: normal; font-variant-caps: normal; font-variant-ligatures: normal; font-weight: 400; justify-content: space-between; letter-spacing: normal; margin-bottom: 16px; orphans: 2; overflow: auto; position: relative; text-align: start; text-decoration-color: initial; text-decoration-style: initial; text-decoration-thickness: initial; text-indent: 0px; text-transform: none; white-space: normal; widows: 2; word-spacing: 0px;"> <pre class="notranslate" style="background-color: var(--bgColor-muted, var(--color-canvas-subtle)); border-radius: 6px; box-sizing: border-box; color: var(--fgColor-default, var(--color-fg-default)); font-family: ui-monospace, SFMono-Regular, &quot;SF Mono&quot;, Menlo, Consolas, &quot;Liberation Mono&quot;, monospace; font-size: 13.6px; line-height: 1.45; margin-bottom: 0px; margin-top: 0px; overflow-wrap: normal; overflow: auto; padding: 16px;"> <code style="background: transparent; border-radius: 6px; border: 0px; box-sizing: border-box; display: inline; font-family: ui-monospace, SFMono-Regular, &quot;SF Mono&quot;, Menlo, Consolas, &quot;Liberation Mono&quot;, monospace; font-size: 13.6px; line-height: inherit; margin: 0px; overflow-wrap: normal; overflow: visible; padding: 0px; white-space: pre; word-break: normal;">bin/spark-shell --master yarn --driver-memory 4g --executor-memory 50g --executor-cores 10 --num-executors 20 --conf spark.sql.shuffle.partitions=400 val inpath="/project/spark/TPCDS/tpcds_1500_parquet_1.10.1/" val outpath="/project/spark/TPCDS/tpcds_1500_parquet_1.13.1/" val compression_type="snappy" // may experiment with "zstd" // we need to do this in two separate groups: partitioned and non-partitioned tables // copy the **partitioned tables** of the TPCDS benchmark // compact each directory into 1 file with repartition val tables_partition=List(("catalog_returns","cr_returned_date_sk"), ("catalog_sales","cs_sold_date_sk"), ("inventory","inv_date_sk"), ("store_returns","sr_returned_date_sk"), ("store_sales","ss_sold_date_sk"), ("web_returns","wr_returned_date_sk"), ("web_sales","ws_sold_date_sk")) for (t &lt;- tables_partition) { println(s"Copying partitioned table $t") spark.read.parquet(inpath + t._1).repartition(col(t._2)).write.partitionBy(t._2).mode("overwrite").option("compression", compression_type).parquet(outpath + t._1) } // copy non-partitioned tables of the TPCDS benchmark // compact each directory into 1 file with repartition val tables_nopartition=List("call_center","catalog_page","customer","customer_address","customer_demographics","date_dim","household_demographics","income_band","item","promotion","reason","ship_mode","store","time_dim","warehouse","web_page","web_site") for (t &lt;- tables_nopartition) { println(s"Copying table $t") spark.read.parquet(inpath + t).coalesce(1).write.mode("overwrite").option("compression", compression_type).parquet(outpath + t) } </code></pre><div class="zeroclipboard-container" style="animation: auto ease 0s 1 normal none running none; box-sizing: border-box; display: block;"> </div> </div> <h2 style="--tw-border-spacing-x: 0; --tw-border-spacing-y: 0; --tw-ring-color: rgba(69,89,164,.5); --tw-ring-offset-color: #fff; --tw-ring-offset-shadow: 0 0 transparent; --tw-ring-offset-width: 0px; --tw-ring-shadow: 0 0 transparent; --tw-rotate: 0; --tw-scale-x: 1; --tw-scale-y: 1; --tw-scroll-snap-strictness: proximity; --tw-shadow-colored: 0 0 transparent; --tw-shadow: 0 0 transparent; --tw-skew-x: 0; --tw-skew-y: 0; --tw-translate-x: 0; --tw-translate-y: 0; border: 0px solid rgb(217, 217, 227); box-sizing: border-box; color: var(--tw-prose-headings); font-family: Söhne, ui-sans-serif, system-ui, -apple-system, &quot;Segoe UI&quot;, Roboto, Ubuntu, Cantarell, &quot;Noto Sans&quot;, sans-serif, &quot;Helvetica Neue&quot;, Arial, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;, &quot;Segoe UI Symbol&quot;, &quot;Noto Color Emoji&quot;; line-height: 1.33333; margin: 2rem 0px 1rem; white-space-collapse: preserve;"><span style="--tw-border-spacing-x: 0; --tw-border-spacing-y: 0; --tw-ring-color: rgba(69,89,164,.5); --tw-ring-offset-color: #fff; --tw-ring-offset-shadow: 0 0 transparent; --tw-ring-offset-width: 0px; --tw-ring-shadow: 0 0 transparent; --tw-rotate: 0; --tw-scale-x: 1; --tw-scale-y: 1; --tw-scroll-snap-strictness: proximity; --tw-shadow-colored: 0 0 transparent; --tw-shadow: 0 0 transparent; --tw-skew-x: 0; --tw-skew-y: 0; --tw-translate-x: 0; --tw-translate-y: 0; border: 0px solid rgb(217, 217, 227); box-sizing: border-box; color: red;">Conclusions</span></h2> <p style="--tw-border-spacing-x: 0; --tw-border-spacing-y: 0; --tw-ring-color: rgba(69,89,164,.5); --tw-ring-offset-color: #fff; --tw-ring-offset-shadow: 0 0 transparent; --tw-ring-offset-width: 0px; --tw-ring-shadow: 0 0 transparent; --tw-rotate: 0; --tw-scale-x: 1; --tw-scale-y: 1; --tw-scroll-snap-strictness: proximity; --tw-shadow-colored: 0 0 transparent; --tw-shadow: 0 0 transparent; --tw-skew-x: 0; --tw-skew-y: 0; --tw-translate-x: 0; --tw-translate-y: 0; border: 0px solid rgb(217, 217, 227); box-sizing: border-box; color: #374151; font-family: Söhne, ui-sans-serif, system-ui, -apple-system, &quot;Segoe UI&quot;, Roboto, Ubuntu, Cantarell, &quot;Noto Sans&quot;, sans-serif, &quot;Helvetica Neue&quot;, Arial, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;, &quot;Segoe UI Symbol&quot;, &quot;Noto Color Emoji&quot;; font-size: 16px; margin: 0px 0px 1.25em; white-space-collapse: preserve;">Apache Spark and Apache Parquet continue to innovate and are constantly upping their game in big data. They've rolled out cool features like column indexes and bloom filters, really pushing the envelope on speed and efficiency. It's a smart move to keep your Spark updated, especially to Spark 3.x or newer, to get the most out of these perks. Also, don’t forget to give your Parquet files a quick refresh to the latest format – the blog post has got you covered with a how-to. Staying on top of these updates is key to keeping your data game strong!</p> <p style="--tw-border-spacing-x: 0; --tw-border-spacing-y: 0; --tw-ring-color: rgba(69,89,164,.5); --tw-ring-offset-color: #fff; --tw-ring-offset-shadow: 0 0 transparent; --tw-ring-offset-width: 0px; --tw-ring-shadow: 0 0 transparent; --tw-rotate: 0; --tw-scale-x: 1; --tw-scale-y: 1; --tw-scroll-snap-strictness: proximity; --tw-shadow-colored: 0 0 transparent; --tw-shadow: 0 0 transparent; --tw-skew-x: 0; --tw-skew-y: 0; --tw-translate-x: 0; --tw-translate-y: 0; border: 0px solid rgb(217, 217, 227); box-sizing: border-box; color: #374151; font-family: Söhne, ui-sans-serif, system-ui, -apple-system, &quot;Segoe UI&quot;, Roboto, Ubuntu, Cantarell, &quot;Noto Sans&quot;, sans-serif, &quot;Helvetica Neue&quot;, Arial, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;, &quot;Segoe UI Symbol&quot;, &quot;Noto Color Emoji&quot;; font-size: 16px; margin: 0px 0px 1.25em; white-space-collapse: preserve;">I extend my deepest gratitude to my colleagues at CERN for their invaluable guidance and support. A special acknowledgment goes out to the teams behind the CERN data analytics, monitoring, and web notebook services, as well as the dedicated members of the ATLAS database team.</p> <p style="--tw-border-spacing-x: 0; --tw-border-spacing-y: 0; --tw-ring-color: rgba(69,89,164,.5); --tw-ring-offset-color: #fff; --tw-ring-offset-shadow: 0 0 transparent; --tw-ring-offset-width: 0px; --tw-ring-shadow: 0 0 transparent; --tw-rotate: 0; --tw-scale-x: 1; --tw-scale-y: 1; --tw-scroll-snap-strictness: proximity; --tw-shadow-colored: 0 0 transparent; --tw-shadow: 0 0 transparent; --tw-skew-x: 0; --tw-skew-y: 0; --tw-translate-x: 0; --tw-translate-y: 0; border: 0px solid rgb(217, 217, 227); box-sizing: border-box; margin: 0px 0px 1.25em;"><span style="color: #374151; font-family: Söhne, ui-sans-serif, system-ui, -apple-system, &quot;Segoe UI&quot;, Roboto, Ubuntu, Cantarell, &quot;Noto Sans&quot;, sans-serif, &quot;Helvetica Neue&quot;, Arial, &quot;Apple Color Emoji&quot;, &quot;Segoe UI Emoji&quot;, &quot;Segoe UI Symbol&quot;, &quot;Noto Color Emoji&quot;; font-size: 16px; white-space-collapse: preserve;">Further details on the topics covered here can be found at:</span></p> <ul><li><a href="https://github.com/LucaCanali/Miscellaneous/blob/master/Spark_Notes/Spark_Parquet.md"><span style="font-size: medium;">Note on getting the best out of Apache Parquet using Apache Spark</span></a></li> <li><a href="https://github.com/LucaCanali/Miscellaneous/blob/master/Spark_Notes/Tools_Parquet_Diagnostics.md"><span style="font-size: medium;">Note on tools for Parquet diagnostics</span></a></li> </ul><p> </p> </div> </div> <span><a title="View user profile." href="/users/luca-canali" lang="" about="/users/luca-canali" typeof="schema:Person" property="schema:name" datatype="">canali</a></span> <span><time datetime="2024-01-30T21:50:14+01:00" title="Tuesday, January 30, 2024 - 21:50">Tue, 01/30/2024 - 21:50</time> </span> <div class="field field--name-field-tags field--type-entity-reference field--label-above"> <div class="field--label"><b>Tags</b></div> <div class="field--items"> <div class="field--item"><a href="/tags/apache-spark" hreflang="en">Apache Spark</a></div> <div class="field--item"><a href="/tags/parquet" hreflang="en">Parquet</a></div> <div class="field--item"><a href="/tags/performance" hreflang="en">Performance</a></div> </div> </div> <section> <h2>Add new comment</h2> <drupal-render-placeholder callback="comment.lazy_builders:renderForm" arguments="0=node&amp;1=194&amp;2=comment_node_blog_post&amp;3=comment_node_blog_post" token="6aQWfiNbmA064WgqRubzQkRkKKGASNy47db0PJynPsE"></drupal-render-placeholder> </section> Tue, 30 Jan 2024 20:50:14 +0000 canali 194 at https://db-blog.web.cern.ch Enhancing Apache Spark Performance with Flame Graphs: A Practical Example Using Grafana Pyroscope https://db-blog.web.cern.ch/node/193 <span>Enhancing Apache Spark Performance with Flame Graphs: A Practical Example Using Grafana Pyroscope</span> <div class="field field--name-body field--type-text-with-summary field--label-above"> <div class="field--label"><b>Blog article:</b></div> <div class="field--item"><p><b><font color="#ff0000" data-keep-original-tag="false" data-original-attrs="{&quot;style&quot;:&quot;&quot;}">TL;DR</font></b><span> </span>Explore a step-by-step example of troubleshooting Apache Spark job performance using flame graph visualization and profiling. Discover the seamless integration of Grafana Pyroscope with Spark for streamlined data collection and visualization.</p> <p> </p> <h3 data-original-attrs="{&quot;style&quot;:&quot;&quot;}" style="text-align: left;"><font color="#ff0000" data-keep-original-tag="false" data-original-attrs="{&quot;style&quot;:&quot;&quot;}">The Puzzle of the Slow Query</font></h3> <p>Set within the framework of data analysis for the ATLAS experiment's Data Control System, our exploration uses data stored in the Parquet format and deploys Apache Spark for queries. The setup: Jupyter notebooks operating on the SWAN service at CERN interfacing with the Hadoop and Spark service.</p> <p><b>The Hiccup:</b> A notably slow query during data analysis where two tables are joined. Running on 32 cores, this query takes 27 minutes—surprisingly long given the amount of data in play.</p> <p><b>The tables involved:</b></p> <ul style="padding: 0px 2.5em; margin: 0.5em 0px; line-height: 1.4; text-align: left;"><li style="padding: 0px; margin: 0px 0px 0.25em; text-indent: 0px;">EVENTHISTORY: A log of events for specific sub-detectors, each row contains a timestamp, the subsystem id and a value</li> <li style="padding: 0px; margin: 0px 0px 0.25em; text-indent: 0px;">LUMINOSITY, a table containing the details of time intervals called "luminosity blocks", see <a href="https://particle.wiki/wiki/Luminosity_block" style="text-decoration: none; color: rgb(34, 136, 187);">Luminosity block - Particle Wiki</a></li> </ul><div> </div> <div><b>Data size:</b></div> <div>EVENTHISTORY is a large table, it can collect millions of data points per day, while LUMINOSITY is a much smaller table (only thousands of points per day). In the test case reported here we used data collected over 1 day, with EVENTHISTORY -&gt; 75M records, and LUMINOSITY -&gt; 2K records.</div> <div> <p> </p> </div> <div><b>The join condition</b><span> </span>between EVENTHISTORY and LUMINOSITY is an expression used to match for events in EVENTHISORY and intervals in LUMINOSITY (note this is not a join based on an equality predicate). This is what the query looks like in SQL:</div> <div> </div> <div> </div> <div> <div><font data-keep-original-tag="false" data-original-attrs="{&quot;style&quot;:&quot;&quot;}" face="courier">spark.sql("""</font></div> <div><font data-keep-original-tag="false" data-original-attrs="{&quot;style&quot;:&quot;&quot;}" face="courier"><b>select l.LUMI_NUMBER, e.ELEMENT_ID, e.VALUE_NUMBER</b></font></div> <div><font data-keep-original-tag="false" data-original-attrs="{&quot;style&quot;:&quot;&quot;}" face="courier"><b>from eventhistory e, luminosity l</b></font></div> <div><b><font data-keep-original-tag="false" data-original-attrs="{&quot;style&quot;:&quot;&quot;}" face="courier">where e.</font><font data-keep-original-tag="false" data-original-attrs="{&quot;style&quot;:&quot;&quot;}" face="courier">ts between l.starttime and l.endtime</font></b></div> <div><font data-keep-original-tag="false" data-original-attrs="{&quot;style&quot;:&quot;&quot;}" face="courier">""")</font></div> </div> <div> </div> <div> </div> <div>An alternative version of the same query written using the DataFrame API:<br />  </div> <div><font data-keep-original-tag="false" data-original-attrs="{&quot;style&quot;:&quot;&quot;}" face="courier">eventhistory_df.join(</font></div> <div> <div><font data-keep-original-tag="false" data-original-attrs="{&quot;style&quot;:&quot;&quot;}" face="courier">    luminosity_df, </font></div> <div><font data-keep-original-tag="false" data-original-attrs="{&quot;style&quot;:&quot;&quot;}" face="courier">    (eventhistory_df.ts &gt;= luminosity_df.starttime) &amp; </font></div> <div><font data-keep-original-tag="false" data-original-attrs="{&quot;style&quot;:&quot;&quot;}" face="courier">    (eventhistory_df.ts &lt;= luminosity_df.endtime)</font></div> <div><font data-keep-original-tag="false" data-original-attrs="{&quot;style&quot;:&quot;&quot;}" face="courier">    ).select(luminosity_df.LUMI_NUMBER,</font></div> <div><font data-keep-original-tag="false" data-original-attrs="{&quot;style&quot;:&quot;&quot;}" face="courier">             eventhistory_df.ELEMENT_ID,</font></div> <div><font data-keep-original-tag="false" data-original-attrs="{&quot;style&quot;:&quot;&quot;}" face="courier">             eventhistory_df.VALUE_NUMBER)</font></div> <div> </div> <div> </div> </div> <h3 data-original-attrs="{&quot;style&quot;:&quot;&quot;}" style="text-align: left;"><font color="#ff0000" data-keep-original-tag="false" data-original-attrs="{&quot;style&quot;:&quot;&quot;}">Cracking the Performance Case</font></h3> <p><b>WebUI:</b><span> </span>The first point of entry for troubleshooting this was the Spark WebUI. We could find there the execution time of the query (27 minutes) and details on the execution plan and SQL metrics under the "SQL/ DataFrame" tab. Figure 1 shows a relevant snippet where we could clearly see that Broadcast nested loop join was used for this.</p> <p> </p> <p><b>Execution Plan:</b><span> </span>The execution plan is the one we wanted for this query, that is the small LUMINOSITY table is broadcasted to all the executors and then joined with each partition of the larger EVENTHISTORY table.</p> <p> </p> <div class="separator" data-original-attrs="{&quot;style&quot;:&quot;&quot;}" style="clear: both; text-align: center;"><a data-original-attrs="{&quot;data-original-href&quot;:&quot;https://blogger.googleusercontent.com/img/b/R29vZ2xl/AVvXsEiQa8CwQ5VGSg2IRxKAdcG3W_2GoanmJzhFnm0rqk7r_-JEyrHoKEMH2BvaFV8eGF-a7MypJK1Fz2xlTOfr56EBJ8rq6YB41N7ApbKW2RIwbzeOEFkHxQ29MqL8F4ny5AqUlMbzB_Xz_kI-lwRUyTgqdaLojnQpQ2ds_M321genry7mStpsb-0fz6MY60E/s600/Figure_WebUI_broadcastjoin.png&quot;,&quot;style&quot;:&quot;&quot;}" href="https://www.blogger.com/blog/post/edit/7003976656201910397/6424794412119323238#" style="margin-left: 1em; margin-right: 1em;"><img alt="" border="0" data-entity-type="" data-entity-uuid="" data-original-height="357" data-original-width="600" height="475" src="https://blogger.googleusercontent.com/img/b/R29vZ2xl/AVvXsEiQa8CwQ5VGSg2IRxKAdcG3W_2GoanmJzhFnm0rqk7r_-JEyrHoKEMH2BvaFV8eGF-a7MypJK1Fz2xlTOfr56EBJ8rq6YB41N7ApbKW2RIwbzeOEFkHxQ29MqL8F4ny5AqUlMbzB_Xz_kI-lwRUyTgqdaLojnQpQ2ds_M321genry7mStpsb-0fz6MY60E/s600/Figure_WebUI_broadcastjoin.png" style="cursor: move;" width="800" /></a></div> <p><b>Figure 1:</b><span> </span>This shows a relevant snippet of the execution graph from the Spark WebUI. The slow query discussed in this post runs using broadcast nested loops join. This means that the small table is broadcasted to all the nodes and then joined to each partition of the larger table.</p> <p> </p> <p><b>CPU utilization measured with Spark Dashboard</b></p> <p>Spark Dashboard instrumentation provides a way to collect and visualize Spark execution metrics. This makes it easy to plot the CPU used during the SQL execution. From there we could see that  the workload was CPU-bound</p> <p> </p> <h3 data-original-attrs="{&quot;style&quot;:&quot;&quot;}" style="text-align: left;"><font color="#ff0000" data-keep-original-tag="false" data-original-attrs="{&quot;style&quot;:&quot;&quot;}">The Clue: Profiling with Flame Graphs and Pyroscope</font></h3> <p>Stack profiling and Flame Graphs visualization are powerful techniques to investigate CPU-bound workloads. We use it here to find where the CPU cycles are consumed and thus make the query slow.</p> <p>First a little recap of what is stack profiling with flame graph visualization, and what tools we can use to apply it to Apache Spark workloads:</p> <p> </p> <p><b>Stack profiling and <a href="https://www.brendangregg.com/flamegraphs.html">Flame Graphs</a></b><a href="https://www.brendangregg.com/flamegraphs.html"><span> </span></a>visualization provide a powerful technique for troubleshooting CPU-bound workloads. </p> <ul data-original-attrs="{&quot;style&quot;:&quot;&quot;}" style="text-align: left;"><li>Flame Graphs provide information on the "hot methods" consuming CPU</li> <li>Flame Graphs and profiling can also be used to profile time spent waiting (off-cpu) and memory allocation</li> </ul><div> </div> <p><a data-original-attrs="{&quot;data-original-href&quot;:&quot;https://grafana.com/oss/pyroscope/&quot;}" href="https://grafana.com/oss/pyroscope/"><b>Grafana Pyroscope</b><span> </span></a>simplifies data collections and visualization, using agents and a custom WebUI. Key motivations for using it with Spark are:</p> <ul data-original-attrs="{&quot;style&quot;:&quot;&quot;}" style="text-align: left;"><li><strong>Streamlined Data Collection &amp; Visualization</strong>: The<span> </span><a data-original-attrs="{&quot;data-original-href&quot;:&quot;https://grafana.com/oss/pyroscope/&quot;}" href="https://grafana.com/oss/pyroscope/" rel="nofollow">Pyroscope project page</a><span> </span>offers a simplified approach to data gathering and visualization with its custom WebUI and agent integration.</li> <li><strong>Java Integration</strong>: The<span> </span><a data-original-attrs="{&quot;data-original-href&quot;:&quot;https://grafana.com/docs/pyroscope/latest/configure-client/language-sdks/java/&quot;}" href="https://grafana.com/docs/pyroscope/latest/configure-client/language-sdks/java/" rel="nofollow">Pyroscope java agent</a><span> </span>is tailored to work seamlessly with Spark. This integration shines especially when Spark is running on various clusters such as YARN, K8S, or standalone Spark clusters.</li> <li><strong>Correlation with Grafana</strong>: Grafana’s integration with Pyroscope lets you juxtapose metrics with other instruments, including the<span> </span><a data-original-attrs="{&quot;data-original-href&quot;:&quot;https://github.com/cerndb/spark-dashboard&quot;}" href="https://github.com/cerndb/spark-dashboard">Spark metrics dashboard</a>.</li> <li><strong>Proven Underlying Technology</strong>: For Java and Python, the tech essentials for collecting stack profiling data,<span> </span><a data-original-attrs="{&quot;data-original-href&quot;:&quot;https://github.com/LucaCanali/Miscellaneous/blob/master/Spark_Notes/Tools_Spark_Linux_FlameGraph.md&quot;}" href="https://github.com/LucaCanali/Miscellaneous/blob/master/Spark_Notes/Tools_Spark_Linux_FlameGraph.md">async-profiler and py-spy</a>, are time-tested and reliable.</li> <li><strong>Functional &amp; Detailed WebUI</strong>: Pyroscope’s WebUI stands out with features that allow users to:<br /><ul dir="auto"><li>Select specific data periods</li> <li>Store and display data across various measurements</li> <li>Offer functionalities to contrast and differentiate measurements</li> <li>Showcase collected data for all Spark executors, with an option to focus on individual executors or machines</li> </ul></li> <li><strong>Lightweight Data Acquisition</strong>: The Pyroscope java agent is efficient in data gathering. By default, stacks are sampled every 10 milliseconds and uploaded every 10 seconds. We did not observe any measurable  performance or stability impact of the instrumentation.</li> </ul><div> </div> <h3 data-original-attrs="{&quot;style&quot;:&quot;&quot;}" style="text-align: left;"><font color="#ff0000" data-keep-original-tag="false" data-original-attrs="{&quot;style&quot;:&quot;&quot;}">Spark Configuration</font></h3> <div> </div> <div>To use Pyroscope with Spark we used some additional configurations. Note this uses a specialized <a data-original-attrs="{&quot;data-original-href&quot;:&quot;https://github.com/cerndb/SparkPlugins&quot;}" href="https://github.com/cerndb/SparkPlugins">Spark Plugin from this repo</a>. It is also possible to use java agents. The details are at:  </div> <div><a data-original-attrs="{&quot;data-original-href&quot;:&quot;https://github.com/LucaCanali/Miscellaneous/blob/master/Spark_Notes/Tools_Spark_Pyroscope_FlameGraph.md&quot;}" href="https://github.com/LucaCanali/Miscellaneous/blob/master/Spark_Notes/Tools_Spark_Pyroscope_FlameGraph.md">How-to profile Apache Spark jobs using Grafana Pyroscope</a></div> <div> </div> <div>This is how we profiled and visualized the Flame Graph of the query execution:</div> <div> </div> <div>1. Start Pyroscope <ul dir="auto"><li>Download from<span> </span><a data-original-attrs="{&quot;data-original-href&quot;:&quot;https://github.com/grafana/pyroscope/releases&quot;}" href=" https://github.com/grafana/pyroscope/releases">https://github.com/grafana/pyroscope/releases</a></li> <li>CLI start:<span> </span><code>./pyroscope -server.http-listen-port 5040</code></li> <li>Or use docker:<span> </span><code>docker run -it -p 5040:4040 grafana/pyroscope</code></li> <li>Note: customize the port number, I used port 5040 to avoid confusion with the Spark WebUI which defaults to port 4040 too</li> </ul></div> <div>2. Start Spark with custom configuration, as in this example with PySpark:</div> <div> </div> <div> <div><font data-keep-original-tag="false" data-original-attrs="{&quot;style&quot;:&quot;&quot;}" face="courier"># Get the Spark session</font></div> <div><font data-keep-original-tag="false" data-original-attrs="{&quot;style&quot;:&quot;&quot;}" face="courier">from pyspark.sql import SparkSession</font></div> <div><font data-keep-original-tag="false" data-original-attrs="{&quot;style&quot;:&quot;&quot;}" face="courier">spark = (SparkSession.builder.</font></div> <div><font data-keep-original-tag="false" data-original-attrs="{&quot;style&quot;:&quot;&quot;}" face="courier">      appName("DCS analysis").master("yarn")</font></div> <div><font data-keep-original-tag="false" data-original-attrs="{&quot;style&quot;:&quot;&quot;}" face="courier">      .config("spark.jars.packages",</font></div> <div><font data-keep-original-tag="false" data-original-attrs="{&quot;style&quot;:&quot;&quot;}" face="courier">      "ch.cern.sparkmeasure:sparkplugins_2.12:0.3, io.pyroscope:agent:0.12.0")</font></div> <div><font data-keep-original-tag="false" data-original-attrs="{&quot;style&quot;:&quot;&quot;}" face="courier">      .config("spark.plugins", "ch.cern.PyroscopePlugin")</font></div> <div><font data-keep-original-tag="false" data-original-attrs="{&quot;style&quot;:&quot;&quot;}" face="courier">      .config("spark.pyroscope.server", "http://pcitdbgpu1:5040")</font></div> <div><font data-keep-original-tag="false" data-original-attrs="{&quot;style&quot;:&quot;&quot;}" face="courier">      .getOrCreate()</font></div> <div><font data-keep-original-tag="false" data-original-attrs="{&quot;style&quot;:&quot;&quot;}" face="courier">    )</font></div> </div> <div> </div> <div> </div> <div class="separator" data-original-attrs="{&quot;style&quot;:&quot;&quot;}" style="clear: both; text-align: center;"><a data-original-attrs="{&quot;data-original-href&quot;:&quot;https://blogger.googleusercontent.com/img/b/R29vZ2xl/AVvXsEifWxSwmr8OK8Cw-PlBfTcbiMYSwcBa7grtrupqmXt9wdi3kXJZvCsSBrUhauKEQE3RQXlzeFTz9WCr7eG1u_CypaqVQgIW-BwCepLvY216C5bTpoih11-838u_MPvXnaM_iMVSJNYQyTHVCCgiLZp4DvgXZiI6jz-Mp1MrzdatP_xUIn7gVqu-Hqwpue8/s912/Figure_Pyroscope_Spark_Slow_Redacted.png&quot;,&quot;style&quot;:&quot;&quot;}" href="https://www.blogger.com/blog/post/edit/7003976656201910397/6424794412119323238#" style="margin-left: 1em; margin-right: 1em;"><img alt="" border="0" data-entity-type="" data-entity-uuid="" data-original-height="912" data-original-width="830" height="876" src="https://blogger.googleusercontent.com/img/b/R29vZ2xl/AVvXsEifWxSwmr8OK8Cw-PlBfTcbiMYSwcBa7grtrupqmXt9wdi3kXJZvCsSBrUhauKEQE3RQXlzeFTz9WCr7eG1u_CypaqVQgIW-BwCepLvY216C5bTpoih11-838u_MPvXnaM_iMVSJNYQyTHVCCgiLZp4DvgXZiI6jz-Mp1MrzdatP_xUIn7gVqu-Hqwpue8/s912/Figure_Pyroscope_Spark_Slow_Redacted.png" style="cursor: move;" width="800" /></a></div> <div class="separator" data-original-attrs="{&quot;style&quot;:&quot;&quot;}" style="clear: both; text-align: center;"> </div> <div><b>Figure 2:</b><span> </span>This is a snapshot from the Grafana Pyroscope dashboard with data collected during the execution of the slow query (join between EVENTHISTORY and LUMINOSITY). The query runs in 27 minutes, using 32 cores. The Flame Graph shows the top executed methods and the Flame Graph. Notably, a large fraction of the execution time appears to be spent into SparkDateTimeUtils performing date-datatype conversion operations. This is a crucial finding for the rest of the troubleshooting and proposed fix.</div> <div> </div> <div> </div> <h3 data-original-attrs="{&quot;style&quot;:&quot;&quot;}" style="text-align: left;"><font color="#ff0000" data-keep-original-tag="false" data-original-attrs="{&quot;style&quot;:&quot;&quot;}">The Insight</font><b>  </b></h3> <div> </div> <div>Using profiling data from Pyroscope, we pinpointed the root cause of the query's sluggishness. Spark was expending excessive CPU cycles on data type conversion operations during the evaluation of the join predicate. Upon revisiting the WebUI and delving deeper into the execution plan under the SQL/DataFrame tab, we discovered, almost concealed in plain view, the specific step responsible for the heightened CPU consumption:</div> <div> </div> <div> <div><font data-keep-original-tag="false" data-original-attrs="{&quot;style&quot;:&quot;&quot;}" face="courier">(9) BroadcastNestedLoopJoin [codegen id : 2]</font></div> <div><font data-keep-original-tag="false" data-original-attrs="{&quot;style&quot;:&quot;&quot;}" face="courier">Join condition: ((ts#1 &gt;= cast(starttime_dec#57 as timestamp)) AND (ts#1 &lt;= cast(endtime_dec#58 as timestamp)))</font></div> </div> <div> </div> <div>The extra operations of "cast to timestamp" appear to be key in explaining the issue.</div> <div>Why do we have date format conversions? </div> <div>By inspecting the schema of the involved tables, it turns out that in the LUMINOSITY table the fields used for joining with the timestamp are of type Decimal.</div> <div> </div> <div>To recap, profiling data, together with the execution plan, showed that the query was slow because it forced data type conversion over and over for each row where the join condition was evaluated.</div> <div> </div> <div><b>The fix:  </b></div> <div>The solution we applied for this was simple: we converted to use the same data type for all the columns involved in the join, in particular converting to timestamp the columns starttime and endtime of the LUMINOSITY table. </div> <div> </div> <div><b>Results: improved performance 70x:</b>  </div> <div>The results are that the query after the change runs in 23 sec, compared to the previous runtime of 27 minutes. Figure 3 shows the Flame graph after the fix was applied.</div> <div> </div> <div> </div> <div class="separator" data-original-attrs="{&quot;style&quot;:&quot;&quot;}" style="clear: both; text-align: center;"><a data-original-attrs="{&quot;data-original-href&quot;:&quot;https://blogger.googleusercontent.com/img/b/R29vZ2xl/AVvXsEjj0cQv4lACBdjjplyHJch-EnXCHW-a2rv3yZvKstV2BfZKgt6LYL8xWB0v0nNOK_ORPioXEzzIMuY_1v6RIbDLWKv_FTsdMooFnn9O6kPtxHt0B0Ya5-sFZ9eXYUmHI_-Gyoajt0pVapwJ66b7JoqnZoiwXJaf4wGPbht54MO8eYxiE6SGF6i_58PxwWE/s1628/Figure_Pyroscope_Spark_Fast.png&quot;,&quot;style&quot;:&quot;&quot;}" href="https://www.blogger.com/blog/post/edit/7003976656201910397/6424794412119323238#" style="margin-left: 1em; margin-right: 1em;"><img alt="" border="0" data-entity-type="" data-entity-uuid="" data-original-height="1175" data-original-width="1628" height="577" src="https://blogger.googleusercontent.com/img/b/R29vZ2xl/AVvXsEjj0cQv4lACBdjjplyHJch-EnXCHW-a2rv3yZvKstV2BfZKgt6LYL8xWB0v0nNOK_ORPioXEzzIMuY_1v6RIbDLWKv_FTsdMooFnn9O6kPtxHt0B0Ya5-sFZ9eXYUmHI_-Gyoajt0pVapwJ66b7JoqnZoiwXJaf4wGPbht54MO8eYxiE6SGF6i_58PxwWE/s1628/Figure_Pyroscope_Spark_Fast.png" style="cursor: move;" width="800" /></a></div> <div> </div> <div><b>Figure 3:</b><span> </span>This is a snapshot of the Grafana Pyroscope dashboard with data collected during the execution of the query after tuning. The query takes only 23 seconds compared to 27 minutes before tuning (see Figure 2)</div> <div> </div> <h3><font color="#ff0000" data-keep-original-tag="false" data-original-attrs="{&quot;style&quot;:&quot;&quot;}">Related work and links</font></h3> <div>Details of how to use Pyroscope with Spark can be found in the note:  </div> <div> <div><a data-original-attrs="{&quot;data-original-href&quot;:&quot;https://github.com/LucaCanali/Miscellaneous/blob/master/Spark_Notes/Tools_Spark_Pyroscope_FlameGraph.md&quot;}" href="https://github.com/LucaCanali/Miscellaneous/blob/master/Spark_Notes/Tools_Spark_Pyroscope_FlameGraph.md">How-to profile Apache Spark jobs using Grafana Pyroscope</a></div> <div> </div> </div> <div>Related work of interest for Apache Spark performance troubleshooting:</div> <div> <ul><li><a data-original-attrs="{&quot;data-original-href&quot;:&quot;https://github.com/cerndb/spark-dashboard&quot;}" href="https://github.com/cerndb/spark-dashboard">Spark Dashboard</a> - tooling and configuration for deploying an Apache Spark Performance Dashboard using containers technology.</li> <li><a data-original-attrs="{&quot;data-original-href&quot;:&quot;https://github.com/LucaCanali/sparkMeasure&quot;}" href="https://github.com/LucaCanali/sparkMeasure">Spark Measure</a> - a tool for performance troubleshooting of Apache Spark workloads. It simplifies the collection and analysis of Spark task and stage metrics data.</li> <li><a data-original-attrs="{&quot;data-original-href&quot;:&quot;https://github.com/cerndb/SparkPlugins&quot;}" href="https://github.com/cerndb/SparkPlugins">Spark Plugins</a><span> </span>- Code and examples of how to write and deploy Apache Spark Plugins.</li> <li><a data-original-attrs="{&quot;data-original-href&quot;:&quot;https://github.com/LucaCanali/Miscellaneous/tree/master/Spark_Notes&quot;}" href="https://github.com/LucaCanali/Miscellaneous/tree/master/Spark_Notes">Spark Notes</a> and<span> </span><a data-original-attrs="{&quot;data-original-href&quot;:&quot;https://github.com/LucaCanali/Miscellaneous/tree/master/Performance_Testing&quot;}" href="https://github.com/LucaCanali/Miscellaneous/tree/master/Performance_Testing">Performance Testing notes</a></li> </ul><div> </div> </div> <h3 data-original-attrs="{&quot;style&quot;:&quot;&quot;}" style="text-align: left;"><font color="#ff0000" data-keep-original-tag="false" data-original-attrs="{&quot;style&quot;:&quot;&quot;}">Wrapping up</font></h3> <div> <div>Wrapping Up: Stack profiling and Flame Graph visualization aren’t just jargon—they’re game-changers. Our deep dive illuminated how they transformed an Apache Spark query performance by 70x. Using Grafana Pyroscope with Spark, we demonstrated a holistic approach to gather, analyze, and leverage stack profile data.</div> <div> </div> <div>A hearty thank you to my colleagues at CERN for their guidance. A special nod to the CERN data analytics, monitoring, and web notebook services, and to the ATLAS database team.</div> </div> <div> </div> </div> </div> <span><a title="View user profile." href="/users/luca-canali" lang="" about="/users/luca-canali" typeof="schema:Person" property="schema:name" datatype="">canali</a></span> <span><time datetime="2023-09-27T15:47:25+02:00" title="Wednesday, September 27, 2023 - 15:47">Wed, 09/27/2023 - 15:47</time> </span> <div class="field field--name-field-tags field--type-entity-reference field--label-above"> <div class="field--label"><b>Tags</b></div> <div class="field--items"> <div class="field--item"><a href="/tags/apache-spark" hreflang="en">Apache Spark</a></div> <div class="field--item"><a href="/tags/performance" hreflang="en">Performance</a></div> <div class="field--item"><a href="/tags/flame-graph" hreflang="en">flame graph</a></div> </div> </div> <section> <h2>Add new comment</h2> <drupal-render-placeholder callback="comment.lazy_builders:renderForm" arguments="0=node&amp;1=193&amp;2=comment_node_blog_post&amp;3=comment_node_blog_post" token="0l0I2UDgnTMgwLy5q8YSzuqh_4hdmGjEwK8AoIPEOMk"></drupal-render-placeholder> </section> Wed, 27 Sep 2023 13:47:25 +0000 canali 193 at https://db-blog.web.cern.ch Performance Comparison of 5 JDKs on Apache Spark https://db-blog.web.cern.ch/node/192 <span>Performance Comparison of 5 JDKs on Apache Spark</span> <div class="field field--name-body field--type-text-with-summary field--label-above"> <div class="field--label"><b>Blog article:</b></div> <div class="field--item"><p style="color: rgb(0, 0, 0); font-family: &quot;Times New Roman&quot;; font-size: medium; font-style: normal; font-variant-ligatures: normal; font-variant-caps: normal; font-weight: 400; letter-spacing: normal; orphans: 2; text-align: left; text-indent: 0px; text-transform: none; widows: 2; word-spacing: 0px; -webkit-text-stroke-width: 0px; white-space: normal; text-decoration-thickness: initial; text-decoration-style: initial; text-decoration-color: initial;">Dive into a comprehensive load-testing exploration using Apache Spark with CPU-intensive workloads. This blog provides a comparative analysis of five distinct JDKs' performance under heavy-duty tasks generated through Spark. Discover a meticulous breakdown of our testing methodology, tools, and insightful results. Keep in mind, our observations primarily indicate the test toolkit and system's performance rather than offering a broad evaluation of the JDKs.</p> <p style="color: rgb(0, 0, 0); font-family: &quot;Times New Roman&quot;; font-size: medium; font-style: normal; font-variant-ligatures: normal; font-variant-caps: normal; font-weight: 400; letter-spacing: normal; orphans: 2; text-align: left; text-indent: 0px; text-transform: none; widows: 2; word-spacing: 0px; -webkit-text-stroke-width: 0px; white-space: normal; text-decoration-thickness: initial; text-decoration-style: initial; text-decoration-color: initial;">In this post, we'll also emphasize:</p> <ul style="color: rgb(0, 0, 0); font-family: &quot;Times New Roman&quot;; font-size: medium; font-style: normal; font-variant-ligatures: normal; font-variant-caps: normal; font-weight: 400; letter-spacing: normal; orphans: 2; text-align: left; text-indent: 0px; text-transform: none; widows: 2; word-spacing: 0px; -webkit-text-stroke-width: 0px; white-space: normal; text-decoration-thickness: initial; text-decoration-style: initial; text-decoration-color: initial;"><li>The rationale behind focusing on CPU and memory-intensive workloads, especially when handling large Parquet datasets.</li> <li>The load testing tool's design: stressing CPU and memory bandwidth with large Parquet files.</li> <li>Key findings from our tests, offering insights into variations across different JDKs.</li> <li>Tools and methods employed for the most accurate measurements, ensuring our results are as reflective of real-world scenarios as possible.</li> </ul><p style="color: rgb(0, 0, 0); font-family: &quot;Times New Roman&quot;; font-size: medium; font-style: normal; font-variant-ligatures: normal; font-variant-caps: normal; font-weight: 400; letter-spacing: normal; orphans: 2; text-align: left; text-indent: 0px; text-transform: none; widows: 2; word-spacing: 0px; -webkit-text-stroke-width: 0px; white-space: normal; text-decoration-thickness: initial; text-decoration-style: initial; text-decoration-color: initial;">Join us on this journey to decipher the intricate landscape of JDKs in the realm of Apache Spark performance!</p> <h2 dir="auto" tabindex="-1"><a aria-hidden="true" class="anchor" href="https://github.com/LucaCanali/Miscellaneous/tree/master/Performance_Testing/Spark_CPU_memory_load_testkit/Test_JDKs#on-the-testing-tool-and-instrumentation" id="user-content-on-the-testing-tool-and-instrumentation"> <svg aria-hidden="true" class="octicon octicon-link" height="16" version="1.1" viewbox="0 0 16 16" width="16"></svg><p></p></a></h2> <h2 dir="auto" tabindex="-1">On the load testing tool and instrumentation</h2> <p dir="auto"><strong>What is being measured:</strong></p> <ul dir="auto"><li>this is a microbenchmark of CPU and memory bandwidth, the tool is not intended to measure the performance of Spark SQL.</li> <li>this follows the general ideas of active benchmarking: a load generator is used to produce CPU and memory-intensive load, while the load is measured with instrumentation.</li> </ul><p dir="auto"><strong>Why testing with a CPU and memory-intensive workload:</strong><br /> In real life, the CPU and memory intensive workloads are often the most critical ones. In particular, when working with large datasets in Parquet format, the CPU and memory-intensive workloads are often the most critical ones. Moreover, workloads that include I/O time from object storage can introduce a lot of variability in the results that does not reflect the performance of Apache Spark but rather of the object storage system. Working on a single large machine also reduces the variability of the results and makes it easier to compare the performance of different test configurations.</p> <p dir="auto"><strong>The test kit:</strong><br /> The testing toolkit used for this exercise is described at <a href="https://github.com/LucaCanali/Miscellaneous/blob/master/Performance_Testing/Spark_CPU_memory_load_testkit/README.md">test_Spark_CPU_memory</a>.</p> <ul dir="auto"><li>The tool generates CPU and memory-intensive load, with a configurable number of concurrent workers.</li> <li>It works by reading a large Parquet file. The test setup is such that the file is cached in the system memory therefore the tool mostly stresses CPU and memory bandwidth.</li> </ul><p dir="auto"><strong>Instrumentation:</strong><br /> The workload is mostly CPU-bound, therefore the main metrics of interest are CPU time and elapsed time. Using sparkMeasure, we can also collect metrics on the Spark executors, notably the executors' cumulative elapsed time, CPU time, and time in garbage collection.</p> <p dir="auto"><strong>Download test data:</strong><br /> The test data used to generate the workload is a large Parquet table, store_sales, taken from the open source TPCDS benchmark. The size of the test data is 200 GB, and it is stored in multiple Parquet files. You can also use a subset of the files in case you want to scale down the benchmark.<br /> The files are cached in the filesystem cache, so that the test kit mostly stresses CPU and memory bandwidth (note, this requires 512GB of RAM on the test system, if you have less RAM, reduce the dataset size).</p> <p dir="auto">Download using download using: <span style="color: rgb(0, 0, 0); font-family: courier; font-size: medium; font-style: normal; font-variant-ligatures: normal; font-variant-caps: normal; font-weight: 400; letter-spacing: normal; orphans: 2; text-align: left; text-indent: 0px; text-transform: none; widows: 2; word-spacing: 0px; -webkit-text-stroke-width: 0px; white-space: normal; text-decoration-thickness: initial; text-decoration-style: initial; text-decoration-color: initial; display: inline !important; float: none;">wget -r -np -nH --cut-dirs=2 -R "index.html*" -e robots=off <a href="https://sparkdltrigger.web.cern.ch/sparkdltrigger/TPCDS/store_sales.parquet">https://sparkdltrigger.web.cern.ch/sparkdltrigger/TPCDS/store_sales.par…</a></span></p> <p dir="auto"><strong>Test results:</strong><br /> Tests were run using the script <a href="https://github.com/LucaCanali/Miscellaneous/blob/master/Performance_Testing/Spark_CPU_memory_load_testkit/Test_JDKs/Data/spark_test_JDKs.sh">spark_test_JDKs.sh</a> that runs <a href="https://github.com/LucaCanali/Miscellaneous/blob/master/Performance_Testing/Spark_CPU_memory_load_testkit/test_Spark_CPU_memory.py">test_Spark_CPU_memory.py</a> with different JDKs and prints out the results. The output of three different tests were collected and stored in txt files that can be found in the <a href="https://github.com/LucaCanali/Miscellaneous/blob/master/Performance_Testing/Spark_CPU_memory_load_testkit/Test_JDKs/Data">Data </a>folder.</p> <p dir="auto"><strong>Test system:</strong><br /> A server with dual CPUS (AMD Zen 2 architecture), 16 physical cores each, 512 GB RAM, ~300 GB of storage space.</p> <p dir="auto"><strong>Spark configuration:</strong><br /> We use Apache Spark run in local mode (that is on a single machine, not scaling out on a cluster) for these tests, with 64GB of heap memory and 20 cores allocated to Spark. The large heap memory allocation is to reduce Garbage Collection overhead, which still fits in the available RAM.<br /> The number of cores for Spark (that is the maximum number of concurrent tasks being executed by Spark) is set to 20, which brings the CPU load during the test execution to use about 60% of the physical cores, the workload keeps the CPUs busy with processing Parquet files, the rest of the CPU power is available for running other accessory load, notably Garbage collection activities, the OS and other processes.</p> <p dir="auto"><strong>Example performance test results:</strong><br /> This shows how you can use the toolkit to run the performance tests and collect performance measurements:</p> <div class="snippet-clipboard-content notranslate position-relative overflow-auto"> <pre class="notranslate"> <code>$ export JAVA_HOME=.... # Set the JDK that will be used by Spark $ ./test_Spark_CPU_memory.py --num_workers 20 # Run the 3 tests using 20 concurrent workers (Spark cores) </code>Allocating a Spark session in local mode with 20 concurrent tasks Heap memory size = 64g, data_path = ./store_sales.parquet sparkmeasure_path = spark-measure_2.12-0.23.jar Scheduling job number 1 Job finished, job_run_time (elapsed time) = 43.93 sec ...executors Run Time = 843.76 sec ...executors CPU Time = 800.18 sec ...executors jvmGC Time = 27.43 sec Scheduling job number 2 Job finished, job_run_time (elapsed time) = 39.13 sec ...executors Run Time = 770.83 sec ...executors CPU Time = 755.55 sec ...executors jvmGC Time = 14.93 sec Scheduling job number 3 Job finished, job_run_time (elapsed time) = 38.82 sec ...executors Run Time = 765.22 sec ...executors CPU Time = 751.68 sec ...executors jvmGC Time = 13.32 sec</pre><div class="zeroclipboard-container"> </div> </div> <p dir="auto"><strong>Notes:</strong><br /> The elapsed time and the Run time decrease with each test run, in particular from the first to the second run we see a noticeable improvement, this is because various internal Spark structures are being "warmed up" and cached. In all cases, data is read from the Filesystem cache, except for the first warm-up runs that are discarded. Therefore, the test kit mostly stresses CPU and memory bandwidth. For the test results and comparisons, we will use the values measured at the 3rd run of each test and average over the available test results for each category.</p> <h2 dir="auto" tabindex="-1"><a aria-hidden="true" class="anchor" href="https://github.com/LucaCanali/Miscellaneous/tree/master/Performance_Testing/Spark_CPU_memory_load_testkit/Test_JDKs#jdk-comparison-tests" id="user-content-jdk-comparison-tests"> <svg aria-hidden="true" class="octicon octicon-link" height="16" version="1.1" viewbox="0 0 16 16" width="16"></svg><p></p></a></h2> <h2 dir="auto" tabindex="-1">JDK comparison tests</h2> <p dir="auto">The following tests compare the performance of 5 different JDKs, running on Linux (CentOS 7.9), on a server with dual Zen 2 CPUs, 16 physical cores each, 512 GB RAM, 300 GB of storage space for the test data. The Apache Spark version is 3.5.0 the test kit is <a href="https://github.com/LucaCanali/Miscellaneous/blob/master/Performance_Testing/Spark_CPU_memory_load_testkit/test_Spark_CPU_memory.py">test_Spark_CPU_memory.py</a>. The JDK tested are:</p> <ul dir="auto"><li>Adoptium jdk8u392-b08</li> <li>Adoptium jdk-11.0.21+9</li> <li>Adoptium jdk-17.0.9+9</li> <li>Oracle jdk-17.0.9</li> <li>Oracle graalvm-jdk-17.0.9+11.1</li> </ul><p dir="auto">The openJDKs were downloaded from <a href="https://adoptium.net/temurin/releases/" rel="nofollow">Adoptium Temurin JDK</a>, the Oracle JDKs were downloaded from <a href="https://www.oracle.com/java/technologies/downloads/" rel="nofollow">Oracle JDK</a>.<br /> The Adoptium Temurin OpenJDK are free to use (see website).<br /> Notably, the Oracle download page also reports that the JDK binaries are available at no cost under the Oracle No-Fee Terms and Conditions, and the GraalVM Free Terms and Conditions, respectively, see Oracle's webpage for details.</p> <h2 dir="auto" tabindex="-1"><a aria-hidden="true" class="anchor" href="https://github.com/LucaCanali/Miscellaneous/tree/master/Performance_Testing/Spark_CPU_memory_load_testkit/Test_JDKs#test-results-and-measurements" id="user-content-test-results-and-measurements"> <svg aria-hidden="true" class="octicon octicon-link" height="16" version="1.1" viewbox="0 0 16 16" width="16"></svg><p></p></a></h2> <h2 dir="auto" tabindex="-1">Test results and measurements</h2> <p dir="auto">Test results summarized in this table are from the test output files, see <a href="https://github.com/LucaCanali/Miscellaneous/blob/master/Performance_Testing/Spark_CPU_memory_load_testkit/Test_JDKs/Data">Data</a>. The values reported here are taken from the test reports, measured at the 3rd run of each test, as the run time improves when running the tests a couple of times in a row (as internal structures and caches are warming up, for example).The results are further averaged over the available test results (6 test runs) and reported for each category.</p> <table><thead><tr><th>JDK and Metric name</th> <th><strong>OpenJDK Java 8</strong></th> <th><strong>OpenJDK Java 11</strong></th> <th><strong>OpenJDK Java 17</strong></th> <th><strong>Oracle Java 17</strong></th> <th><strong>GraalVM Java 17</strong></th> </tr></thead><tbody><tr><td>JDK</td> <td>Adoptium jdk8u392-b08</td> <td>Adoptium jdk-11.0.21+9</td> <td>Adoptium jdk-11.0.21+9</td> <td>Oracle jdk-17.0.9</td> <td>Oracle graalvm-jdk-17.0.9+11.1</td> </tr><tr><td><strong>Elapsed time (sec)</strong></td> <td><strong>45.4</strong></td> <td><strong>39.3</strong></td> <td><strong>42.0</strong></td> <td><strong>41.9</strong></td> <td><strong>34.1</strong></td> </tr><tr><td>Executors' cumulative<br /> ... run time (sec)</td> <td>896.1</td> <td>775.9</td> <td>829.7</td> <td>828.6</td> <td>672.3</td> </tr><tr><td>... CPU time (sec)</td> <td>851.9</td> <td>763.4</td> <td>800.6</td> <td>796.4</td> <td>649.5</td> </tr><tr><td>... Garbage Collection time (sec)</td> <td>42.6</td> <td>12.3</td> <td>29.4</td> <td>32.5</td> <td>23.0</td> </tr></tbody></table><p> </p> <h2 dir="auto" tabindex="-1">Performance data analysis</h2> <p dir="auto">From the metrics and elapsed time measurements reported above, the key findings are:</p> <ul dir="auto"><li>Java 8 has the slowest elapsed time, Java 11 and 17 are about 10% faster than Java 8, GraalVM is about 25% faster than Java 8.</li> <li>The workload is CPU bound.</li> </ul><p dir="auto">The instrumentation metrics provide additional clues on understanding the workload and its performance:</p> <ul dir="auto"><li><code>Run time</code>, reports the cumulative elapsed time for the executors</li> <li><code>CPU time</code> reports the cumulative time spent on CPU.</li> <li><code>Garbage Collection Time</code> is the time spent by the executors on JVM Garbage collection, and it is a subset of the "Run time" metric.</li> <li>From the measured values (see table above) we can conclude that the executors spend most of the time running tasks "on CPU", with some time spent on Garbage collection</li> <li>We can see some fluctuations on Garbage Collection time, with Java 8 having the longest GC time. Note that the algorithm G1GC was used in all the tests (its use is set</li> <li>as a configuration by the load generation tool <a href="https://github.com/LucaCanali/Miscellaneous/blob/master/Performance_Testing/Spark_CPU_memory_load_testkit/test_Spark_CPU_memory.py">test_Spark_CPU_memory.py</a>).</li> <li>We can see the GraalVM 17 stands out as having the shortest Executors' runtime. We can speculate that is due to the GraalVM just-in-time compiler and the Native Image feature, which provide several optimizations compared to the standard HotSpot JVM (note, before running to install GraalVM for your Spark jobs, please note that there are other factors at play here, including that Native Image feature in an optional early adopter technology, see Oracle documentation for details).</li> <li>Java 8 shows the worst performance in terms of run time and CPU time, and it also has the longest Garbage Collection time. This is not surprising as Java 8 is the oldest of the JDKs tested here, and it is known to have worse performance than newer JDKs.</li> <li>Java 11 and Java 17 have similar performance, with Java 11 being a bit faster than Java 17 (of the order of 5% for this workload), the origin of this difference is not investigated here.<br />  </li> </ul><h2 dir="auto" tabindex="-1">Active benchmarking and sanity checks</h2> <p dir="auto">The key idea of active benchmarking is that while the load testing tool is running, we also take several measurements and metrics using a variety of monitoring and measuring tools, for OS metrics and application-specific metrics. These measurements are used to complement the analysis results, provide sanity checks, and in general to help understand the performance of the system under test (why is the performance that we see what it is? why not higher/lower? Are there any bottlenecks or other issues/errors limiting the performance?).</p> <p dir="auto"><strong>Spark tools:</strong> the application-specific instrumentation used for these tests were the Spark WebUI and the instrumentation with <a href="https://github.com/LucaCanali/sparkMeasure">sparkMeasure</a> that allowed us to understand the workload as CPU-bound and to measure the CPU time and Garbage collection time.</p> <p dir="auto"><strong>Java FlameGraph:</strong> Link to a <a href="http://canali.web.cern.ch/svg/FlameGraph_test_Spark_CPU_memory.html" rel="nofollow">FlameGraph of the execution profile</a> taken during a test run of <a href="https://github.com/LucaCanali/Miscellaneous/blob/master/Performance_Testing/Spark_CPU_memory_load_testkit/test_Spark_CPU_memory.py">test_Spark_CPU_memory.py</a>. The FlameGraph shows that the workload is CPU-bound, and that the time is spent in the Spark SQL code, in particular in the Parquet reader. FlameGraphs are a visualization tool for profiling the performance of applications, see also <a href="https://github.com/LucaCanali/Miscellaneous/blob/master/Performance_Testing/Tools_FlameGraphs.md">Tools_FlameGraphs.md</a>.</p> <p dir="auto"><strong>OS Tools:</strong> (see also <a href="https://github.com/LucaCanali/Miscellaneous/blob/master/Performance_Testing/Tools_Linux_OS_CPU_Disk_Network.md">OS monitoring tools</a>): Another important aspect was to ensure that the data was cached in the filesystem cache, to avoid the overhead of reading from disk, for this tools like <code>iostat</code> and <code>iotop</code> were used to monitor the disk activity and ensure that the I/O on the system was minimal, therefore implying that data was read from the filesystem cache.<br /> A more direct measurement was taken using <code>cachestat</code>, a tool that can be found in the perf-tools collection and bcc-tool, which allows measuring how many reads hit the filesystem cache, we could see that the hit rate was 100%, after the first couple of runs that populated the cache (and that were not taken in consideration for the test results).<br /> CPU measurements were taken using <code>top</code>, <code>htop</code>, and <code>vmstat</code> to monitor the CPU usage and ensure that the CPUs were not saturated.</p> <p dir="auto"><strong>Other sanity checks:</strong> were about checking that the intended JDK was used in a given test, for that we used <code>top</code> and <code>jps</code>, for example.<br /> Another important check is about the stability of the performance tests' measurements. We notice fluctuations in the execution time for different runs with the same parameters, for example. For this reason the load-testing tool is run on a local machine rather than a cluster, where these differences are amplified, moreover the tests are run multiple times, and the results reported are averages. We estimated the errors in the metrics measurements due to these fluctuations to be less than 3%, see also the raw test results reported available at <a href="https://github.com/LucaCanali/Miscellaneous/blob/master/Performance_Testing/Spark_CPU_memory_load_testkit/Test_JDKs/Data">Data</a>.</p> <h2 dir="auto" tabindex="-1"><a aria-hidden="true" class="anchor" href="https://github.com/LucaCanali/Miscellaneous/tree/master/Performance_Testing/Spark_CPU_memory_load_testkit/Test_JDKs#related-work" id="user-content-related-work"> <svg aria-hidden="true" class="octicon octicon-link" height="16" version="1.1" viewbox="0 0 16 16" width="16"></svg><p></p></a></h2> <h2 dir="auto" tabindex="-1">Related work</h2> <p dir="auto">The following references provide additional information on the topics covered in this note.</p> <ul dir="auto"><li><a href="https://github.com/LucaCanali/Miscellaneous/blob/master/Performance_Testing/Spark_CPU_memory_load_testkit/test_Spark_CPU_memory.py">test_Spark_CPU_memory.py</a> used to test CPU performance on two difference architectures<br /><ul dir="auto"><li>see <a href="https://github.com/LucaCanali/Miscellaneous/blob/master/Performance_Testing/Spark_CPU_memory_load_testkit/Spark_CPU_memory_load_testkit/Test_CPUs">CPU and Memory testing with Spark</a> and <a href="https://canali.web.cern.ch/docs/Spark_CPU_and_memory_load_testing_HDP6_RAC55_May2023.pdf" rel="nofollow">pdf</a></li> </ul></li> <li>CPU load-testing kit<br /><ul dir="auto"><li><a href="https://github.com/LucaCanali/Miscellaneous/blob/master/Performance_Testing/Test_CPU_parallel_Python">Python version</a></li> <li><a href="https://github.com/LucaCanali/Miscellaneous/blob/master/Performance_Testing/Test_CPU_parallel_Rust">Rust version</a></li> <li><a href="https://db-blog.web.cern.ch/node/189" rel="nofollow">How to use the CPU load-testing kit (blog)**</a></li> </ul></li> <li>Metrics collection for Apache Spark performance troubleshooting: <a href="https://github.com/LucaCanali/sparkMeasure">sparkMeasure </a></li> <li><a href="https://www.brendangregg.com/activebenchmarking.html">Active benchmarking</a></li> <li>A short list of <a href="https://github.com/LucaCanali/Miscellaneous/blob/master/Performance_Testing/Tools_Linux_OS_CPU_Disk_Network.md">OS monitoring tools</a></li> <li>A note on : <a href="https://github.com/LucaCanali/Miscellaneous/blob/master/Spark_Notes/Spark_Set_Java_Home_Howto.md">How specify a custom Java Home/Java version for Spark executors on YARN</a></li> </ul><h2 dir="auto" tabindex="-1"><a aria-hidden="true" class="anchor" href="https://github.com/LucaCanali/Miscellaneous/tree/master/Performance_Testing/Spark_CPU_memory_load_testkit/Test_JDKs#conclusions" id="user-content-conclusions"> <svg aria-hidden="true" class="octicon octicon-link" height="16" version="1.1" viewbox="0 0 16 16" width="16"></svg><p></p></a></h2> <h2 dir="auto" tabindex="-1">Conclusions</h2> <p dir="auto">This blog post presents an exploration of load methodologies using Apache Spark and a custom CPU and memory-intensive testing toolkit. The focus is on comparing different JDKs and producing insights into their respective performance when running Apache Spark jobs under specific conditions (CPU and memory-intensive load when reading Parquet files). Upon evaluating Apache Spark's performance across different JDKs in CPU and memory-intensive tasks involving Parquet files, several key findings emerged:</p> <ol><li><strong>JDK's Impact</strong>: The chosen JDK affects performance, with significant differences observed among Java 8, 11, 17, and GraalVM.</li> <li><strong>Evolution of JDKs</strong>: Newer JDK versions like Java 11 and 17 showcased better outcomes compared to Java 8. GraalVM, with its specific optimizations, also stood out.</li> <li><strong>Developer Insights</strong>: Beyond personal preference, JDK selection can drive performance optimization. Regular software updates are essential.</li> <li><strong>Limitations</strong>: Our results are based on specific test conditions. Real-world scenarios might differ, emphasizing the need for continuous benchmarking.</li> <li><strong>Guidance for System Specialists</strong>: This study offers actionable insights for architects and administrators to enhance system configurations for Spark tasks.</li> </ol><p>In essence, the choice of JDK, coupled with the nature of the workload, plays a significant role in Apache Spark's efficiency. Continuous assessment is crucial to maintain optimal performance.</p> <h2 dir="auto" tabindex="-1"> </h2> <h2 dir="auto" tabindex="-1">Acknowledgements</h2> <p dir="auto">I would like to express my sincere gratitude to my colleagues at CERN for their invaluable assistance and insightful suggestions, in particular I'd like to acknowledge the CERN data analytics and web notebook services, and the ATLAS database and data engineering teams.</p> <p dir="auto"> </p> </div> </div> <span><a title="View user profile." href="/users/luca-canali" lang="" about="/users/luca-canali" typeof="schema:Person" property="schema:name" datatype="">canali</a></span> <span><time datetime="2023-08-11T16:39:59+02:00" title="Friday, August 11, 2023 - 16:39">Fri, 08/11/2023 - 16:39</time> </span> <div class="field field--name-field-tags field--type-entity-reference field--label-above"> <div class="field--label"><b>Tags</b></div> <div class="field--items"> <div class="field--item"><a href="/tags/apache-spark" hreflang="en">Apache Spark</a></div> <div class="field--item"><a href="/tags/java" hreflang="en">Java</a></div> <div class="field--item"><a href="/tags/performance" hreflang="en">Performance</a></div> </div> </div> <section> <h2>Add new comment</h2> <drupal-render-placeholder callback="comment.lazy_builders:renderForm" arguments="0=node&amp;1=192&amp;2=comment_node_blog_post&amp;3=comment_node_blog_post" token="pBq1U-WtFRhXY3zidFWlhia-14PyGbD0wm4CrClZZSE"></drupal-render-placeholder> </section> Fri, 11 Aug 2023 14:39:59 +0000 canali 192 at https://db-blog.web.cern.ch Building a Semantic Search Engine and RAG Applications with Vector Databases and Large Language Models https://db-blog.web.cern.ch/node/191 <span>Building a Semantic Search Engine and RAG Applications with Vector Databases and Large Language Models</span> <div class="field field--name-body field--type-text-with-summary field--label-above"> <div class="field--label"><b>Blog article:</b></div> <div class="field--item"><p>This blog post is about building a getting-started example for semantic search using vector databases and large language models (LLMs), an example of retrieval augmented generation (RAG) architecture. You can find the accompanying <a href="https://github.com/cerndb/NotebooksExamples/blob/main/AITools/LangChain_OpenSearch_semantic_search_with_Vector_DB.ipynb">notebook at this link</a>. See also the <a href="https://swan-gallery.web.cern.ch/aitools/">SWAN gallery</a>.</p> <p>CERN users can run the notebooks using the <a href="https://swan.web.cern.ch/swan/">SWAN</a> platform and GPU resources.<br /><a href="https://swan-k8s.cern.ch/user-redirect/download?projurl=https://github.com/cerndb/NotebooksExamples.git" rel="nofollow"><img alt="SWAN" data-canonical-src="https://swan.web.cern.ch/sites/swan.web.cern.ch/files/pictures/open_in_swan.svg" data-entity-type="" data-entity-uuid="" src="https://camo.githubusercontent.com/a6c6e6b9567789e9f2a43a9663da84c90322943e109785d7464a364c5fdc8a98/68747470733a2f2f7377616e2e7765622e6365726e2e63682f73697465732f7377616e2e7765622e6365726e2e63682f66696c65732f70696374757265732f6f70656e5f696e5f7377616e2e737667" style="max-width: 100%;" /></a></p> <p>Other options for running the notebooks in the cloud with a GPU include Google's <a href="https://colab.research.google.com/">Colab</a>. <br /><a href="https://colab.research.google.com/github/cerndb/NotebooksExamples/blob/main/AITools/LangChain_OpenSearch_semantic_search_with_Vector_DB.ipynb" target="_blank"><img alt="Open In Colab" data-entity-type="" data-entity-uuid="" src="https://colab.research.google.com/assets/colab-badge.svg" /></a></p> <p> </p> <h3 style="text-align:left"><span style="font-family:Times New Roman, serif">Goals and Scope</span></h3> <p><span style="font-family:Times New Roman, serif">Our primary goal is to demonstrate the implementation of a search engine that focuses on understanding the meaning of documents rather than relying solely on keywords.</span></p> <p><span style="font-family:Times New Roman, serif">The proposed implementation uses resources currently available to CERN users: Jupyter notebooks with GPUs, Python packages from the open source ecosystem, a vector database.</span></p> <p><span style="font-family:Times New Roman, serif">Limitations:it's important to note that this example does not cover building a fully-fledged search service or chat engine. We leave those topics for future work, here were limit the discussion to a getting-started example and a technology demonstrator. </span></p> <p><span style="font-family:Times New Roman, serif"></span></p> <h3 style="text-align:left"> </h3> <h3 style="text-align:left"><span style="font-family:Times New Roman, serif">Understanding Key Concepts</span></h3> <p><span style="font-family:Times New Roman, serif">Semantic search: Semantic search involves searching for meaning rather than just literal matches of query words. By understanding the context and intent behind the query, semantic search engines can provide more accurate and relevant results.  </span></p> <p><span style="font-family:&quot;Times New Roman&quot;, serif">Vector Database: A vector database is a specialized type of database designed to handle vector embeddings. These embeddings represent data in a way that captures essential semantic information. They are widely used in applications such as large language models, generative AI, and semantic search.  </span></p> <p><span style="font-family:&quot;Times New Roman&quot;, serif">Large Language Models (LLMs): LLMs are powerful language models built using artificial neural networks with a vast number of parameters (ranging from tens of millions to billions). These models are trained on extensive amounts of unlabeled text data using self-supervised or semi-supervised learning techniques.  </span></p> <p> </p> <h3><span style="font-family:Times New Roman, serif">Implementation details</span></h3> <p><span style="font-size:12pt"><span style="font-family:&quot;Times New Roman&quot;,serif"><span style="line-height:107%">Building a semantic search prototype has become more accessible due to recent advancements in natural language processing and applied ML/AI. Using off-the-shelf components and integrating them effectively can accelerate the development process. Here are some notable key ingredients that facilitate this implementation:</span></span></span></p> <ul style="text-align:left"><li><span style="font-family:&quot;Times New Roman&quot;,serif"><span style="line-height:107%"><b>Large Language Models (LLMs) and e<b>mbedding Libraries</b>: </b></span></span> <ul><li><span style="font-family:&quot;Times New Roman&quot;,serif"><span style="line-height:107%">The availability of powerful LLMs such as <b>OpenAI</b> GPT-3.5 and GPT-4, Google's Palm 2, and of embedding libraries, significantly simplifies the implementation of semantic search and natural language processing in general. These models provide comprehensive language understanding and generation capabilities, enabling us to extract meaning from text inputs.</span></span></li> </ul></li> <li><span style="font-family:&quot;Times New Roman&quot;,serif"><span style="line-height:107%"><b>Platforms:</b> </span></span> <ul><li><span style="font-family:&quot;Times New Roman&quot;,serif"><span style="line-height:107%">Platforms and cloud services such as <a href="https://huggingface.co/" style="font-size:12pt"><b>Hugging Face</b></a> offer valuable resources for operating with ML models as these libraries provide pre-trained models, tokenization utilities, and interfaces to interact with LLMs, reducing the implementation complexity.</span></span></li> </ul></li> <li><span style="font-family:&quot;Times New Roman&quot;,serif"><span style="line-height:107%"><b>Open Source Libraries like LangChain:</b></span></span> <ul><li><span style="font-family:&quot;Times New Roman&quot;,serif"><span style="line-height:107%">Open source libraries like <a href="https://python.langchain.com/docs/get_started/introduction.html" style="font-size:16px"><b>LangChain</b></a> provide a convenient way to integrate and orchestrate the different components required for building applications in the semantic search domain. These libraries often offer pre-defined pipelines, data processing tools, and easy-to-use APIs, allowing developers to focus on the core logic of their applications.</span></span></li> </ul></li> <li><span style="font-family:&quot;Times New Roman&quot;,serif"><span style="line-height:107%"><b>Vector Databases </b> and Vector Libraries:  </span></span> <ul><li><span style="font-family:&quot;Times New Roman&quot;,serif"><span style="line-height:107%">Vector libraries play a crucial role in working with semantic embeddings. They provide functionalities for vector manipulation, similarity calculations, and operations necessary for processing and analyzing embedding data. Additionally, vector databases are recommended for advanced deployments, as they offer storage and querying capabilities for embeddings, along with metadata storage options. Several solutions are available in this area, ranging from mature products offered as cloud services to open source alternatives.</span></span></li> </ul></li> </ul><div> </div> <h3 style="text-align:left"><span style="font-family:Times New Roman, serif">Back-end: prepare the embeddings and indexes in a vector database</span></h3> <div><span style="font-family:&quot;Times New Roman&quot;, serif">To ensure factual accuracy and preserve the original document references, we will prepare the embeddings and indexes in a vector database for our semantic search query engine. Additionally, we aim to enable indexing of private documents, which necessitates storing the embeddings rather than relying on the LLM model directly.</span></div> <div><span style="font-family:&quot;Times New Roman&quot;, serif"></span></div> <div><span style="font-family:Times New Roman, serif">Transforming document chunks into embedding vectors is a crucial step in the process. There are specialized libraries available that utilize neural networks for this task. These libraries can be accessed as cloud services or downloaded to run on local GPU resources. In the accompanying notebook, you can find in the </span><a href="https://github.com/cerndb/NotebooksExamples/blob/main/AITools/LangChain_OpenSearch_semantic_search_with_Vector_DB.ipynb" style="font-family:&quot;Times New Roman&quot;, serif; font-size:16px">accompanying notebook</a> <span style="font-family:&quot;Times New Roman&quot;, serif">an example demonstrating this process. </span></div> <div><span style="font-family:Times New Roman, serif"></span></div> <div><span style="font-family:Times New Roman, serif">A second import part is about storing the embeddings. For this a vector library or a vector database can be quite useful. A library like </span><a href="https://github.com/facebookresearch/faiss" style="font-family:&quot;Times New Roman&quot;, serif; font-size:16px">FAISS </a>is a good idea is you have a small amount of documents and/or are just prototyping. A vector DB <span style="font-family:Times New Roman, serif">can provide more features than a simple library, in particular when handling large amounts of documents. </span><span style="font-size:16px"><span style="font-family:&quot;Times New Roman&quot;, serif">In the </span></span><a href="https://github.com/cerndb/NotebooksExamples/blob/main/AITools/LangChain_OpenSearch_semantic_search_with_Vector_DB.ipynb" style="font-family:&quot;Times New Roman&quot;, serif; font-size:16px">accompanying notebook</a><span style="font-size:16px"><span style="font-family:&quot;Times New Roman&quot;, serif"> we use the </span></span><a href="https://github.com/facebookresearch/faiss" style="font-family:&quot;Times New Roman&quot;, serif; font-size:16px">FAISS </a><span style="font-size:16px"><span style="font-family:&quot;Times New Roman&quot;, serif">library and, as alternative option, </span></span><a href="https://opensearch.org/docs/latest/search-plugins/knn/index/" style="font-family:&quot;Times New Roman&quot;, serif; font-size:16px">OpenSearch k-NN indexing</a>.<span style="font-size:16px"><span style="font-family:&quot;Times New Roman&quot;, serif"> Note that several</span></span> other vector database products can be readily "substituted" to offer comparable and, in some cases, extended functionality.</div> <div><span style="font-family:Times New Roman, serif"></span></div> <div><span style="font-family:Times New Roman, serif">Note: CERN users have the option to contact the OpenSearch service to request an instance of </span><a href="https://opensearch.org/platform/search/vector-database.html" style="font-family:&quot;Times New Roman&quot;, serif">OpenSearch equipped with the plugin for k-NN search</a><span style="font-family:&quot;Times New Roman&quot;, serif">. This can be a valuable resource for your semantic search implementation.</span></div> <div> </div> <div><span style="font-family:Times New Roman, serif"></span></div> <div> <div class="separator" style="text-align:center"><span style="clear:both"><a href="https://blogger.googleusercontent.com/img/b/R29vZ2xl/AVvXsEjvbur9Vo23KWiYwge9bz_wZQ6h3Nm6B4Gbkg49AxSKdE27R0JCoxz9b7OR6VI0OrcfcMCvGDCiHOoevGM2rupRJ5mJegbJP1yvwcP4X49MMP39rnj-xAye-k_Oz6eofY3OVIbR2tdqLjgai-XjO9W3m2VZezjQB2rzDOwuijpX9oOhtiIxlRsBZiRVnyA/s1023/Image_backend.png" style="margin-left:13px; margin-right:13px"><img alt="" data-entity-type="" data-entity-uuid="" data-original-height="743" data-original-width="1023" src="https://blogger.googleusercontent.com/img/b/R29vZ2xl/AVvXsEjvbur9Vo23KWiYwge9bz_wZQ6h3Nm6B4Gbkg49AxSKdE27R0JCoxz9b7OR6VI0OrcfcMCvGDCiHOoevGM2rupRJ5mJegbJP1yvwcP4X49MMP39rnj-xAye-k_Oz6eofY3OVIbR2tdqLjgai-XjO9W3m2VZezjQB2rzDOwuijpX9oOhtiIxlRsBZiRVnyA/w640-h464/Image_backend.png" style="width:640px; height:464px" /></a></span></div> <div><b>Figure 1: </b>A schematic diagram of how to prepare a set of documents for semantic search. The documents are split in chunks, for each chunk embeddings are computed with a specilized library and then stored in a vector database.</div> <div> </div> <div>When using FAISS as the Vector library, this is how embedding and indexing can be done:</div> <div> </div> <div class="separator" style="text-align:center"><span style="clear:both"><a href="https://blogger.googleusercontent.com/img/b/R29vZ2xl/AVvXsEg5APIuR8_8brAsZYc1xA3t0QxGwuIX56WDFAVsURne2FrvooEl8AShvIIB0sTrK5VyOC-9twrGS1sT_EIO4-0mJi96AJCkmeX1T6b8vCsW--lPXZbFl8MZLRv3VLXOWWckM477fffHUDr9XWZIg2ptP9WmstF1lMNyf4f-xxTvjr310oa7Phjeu9e_P1I/w640-h164/snippet_embedding_FAISS.png" style="margin-left:13px; margin-right:13px"><img alt="" data-entity-type="" data-entity-uuid="" data-original-height="145" data-original-width="564" src="https://blogger.googleusercontent.com/img/b/R29vZ2xl/AVvXsEg5APIuR8_8brAsZYc1xA3t0QxGwuIX56WDFAVsURne2FrvooEl8AShvIIB0sTrK5VyOC-9twrGS1sT_EIO4-0mJi96AJCkmeX1T6b8vCsW--lPXZbFl8MZLRv3VLXOWWckM477fffHUDr9XWZIg2ptP9WmstF1lMNyf4f-xxTvjr310oa7Phjeu9e_P1I/w640-h164/snippet_embedding_FAISS.png" style="width:640px; height:164px" /></a></span></div> <p> </p> <div>This is the equivalent code when using OpenSearch as Vector DB:</div> <div> </div> <div> <div class="separator" style="text-align:left"><span style="clear:both"><a href="https://blogger.googleusercontent.com/img/b/R29vZ2xl/AVvXsEg281zpf84GKFmqSxG4XdE6Utgqt0bW30TRco7UWfJ4lgIMVUSiNpEHk4-C-3AJPVRAkR7nZa4VNtT0BDl8ag_dC2ky5AimdnNTcoN4dD5No1GiXcZDaRZePTNKat_O-JXoujoXU6JfInIWvhxUYXgDNS6sp2Q9JXiF3udy82DzwmjbkuJRonIjLBc04L4/s413/snippet_embedding_opensearch.png" style="margin-left:13px; margin-right:13px"><img alt="" data-entity-type="" data-entity-uuid="" data-original-height="134" data-original-width="413" src="https://blogger.googleusercontent.com/img/b/R29vZ2xl/AVvXsEg281zpf84GKFmqSxG4XdE6Utgqt0bW30TRco7UWfJ4lgIMVUSiNpEHk4-C-3AJPVRAkR7nZa4VNtT0BDl8ag_dC2ky5AimdnNTcoN4dD5No1GiXcZDaRZePTNKat_O-JXoujoXU6JfInIWvhxUYXgDNS6sp2Q9JXiF3udy82DzwmjbkuJRonIjLBc04L4/w640-h206/snippet_embedding_opensearch.png" style="width:640px; height:206px" /></a></span></div> <div class="separator" style="text-align:center"><span style="clear:both"></span></div> </div> <p> </p> <h3 style="text-align:left">Semantic querying using similarity search and vector DB indexes</h3> </div> <div> </div> <div>This uses a key functionality of vector libraries and vector databases: similarity search. The general idea is to create a vector embedding for the query and find in the database of embedded vectors the closest elements to the query. For large document collections this can be slow, so vector libraries and databases implement specialized indexes and algorithms for this, for example approximate k-nearest neighbors search.</div> <div>  <div class="separator" style="text-align:center"><span style="clear:both"><a href="https://blogger.googleusercontent.com/img/b/R29vZ2xl/AVvXsEiANFNx-VDcGYGKDA_caWcBC-PmubTqGEeYLXtyQSSG4Ddgl5GXhRIe_YdXR4X-igfWIE3G0aNG_iSYUzAyQdyVxeNJ4oV7oFzUCiiuHyH07zMsPoJ8QPjIVPzC5F_DdhiTaREVRBYg8Xh60bBD1gT16moywRWur6M1tyP9CcZaTUBDD0eWf2NnCRotLOU/s854/Diagram_semantic_query.png" style="margin-left:13px; margin-right:13px"><img alt="" data-entity-type="" data-entity-uuid="" data-original-height="663" data-original-width="854" src="https://blogger.googleusercontent.com/img/b/R29vZ2xl/AVvXsEiANFNx-VDcGYGKDA_caWcBC-PmubTqGEeYLXtyQSSG4Ddgl5GXhRIe_YdXR4X-igfWIE3G0aNG_iSYUzAyQdyVxeNJ4oV7oFzUCiiuHyH07zMsPoJ8QPjIVPzC5F_DdhiTaREVRBYg8Xh60bBD1gT16moywRWur6M1tyP9CcZaTUBDD0eWf2NnCRotLOU/s854/Diagram_semantic_query.png" style="width:640px; height:496px" /></a></span></div> <div><b>Figure 2: </b> A diagram of the similarity query process. The query is converted into embeddings and similarity search via the specialized indexes is performed using a vector database or vector library. Algorithms such as k-nearest neighbors are used to find the matching document chunks for the given query.</div> <div> </div> <div>Semantic search provides a list of relevant documents for a user query, list the page and text chunk reference, as in this example:</div> <div> </div> <div class="separator" style="text-align:center"><span style="clear:both"><a href="https://blogger.googleusercontent.com/img/b/R29vZ2xl/AVvXsEgTn7ZcwUziR-5c2IEi74Slbx92fLZLXF-xs9JV74x8pBCske1CVEuxBZ-UblZ19plpfnBB9Dmt__1fLoYt5ysuAZopJz__rQ0dq-5c_dDXFUidbV99oUtbykoF7zDMIkxxUwcQ4FaCoOvPxgi3C6X7MVJbWX2N1R7fjpPtcRpFsiK4W4S-ptJVvAAEStA/s700/basic_semantic_search.png" style="margin-left:13px; margin-right:13px"><img alt="" data-entity-type="" data-entity-uuid="" data-original-height="480" data-original-width="700" src="https://blogger.googleusercontent.com/img/b/R29vZ2xl/AVvXsEgTn7ZcwUziR-5c2IEi74Slbx92fLZLXF-xs9JV74x8pBCske1CVEuxBZ-UblZ19plpfnBB9Dmt__1fLoYt5ysuAZopJz__rQ0dq-5c_dDXFUidbV99oUtbykoF7zDMIkxxUwcQ4FaCoOvPxgi3C6X7MVJbWX2N1R7fjpPtcRpFsiK4W4S-ptJVvAAEStA/s700/basic_semantic_search.png" style="width:640px; height:438px" /></a></span></div> <p> </p> <div> </div> <div> <h3>Grand Finale: a Large Language Model for natural language query capabilities</h3> </div> <div>Semantic search returns a list of relevant document snippets, as the last (optional) step we want to convert that into a coherent text answer. For this we can use LLM models. The technique is simple, we just need to feed the query and the relevant pieces of text to the LLM and then take the answer from the model. For this we need to use a rather sophisticated LLM model. The best ones currently work as cloud services (some are free and some charge per use), other models available for free download currently require rather powerful GPUs to run locally.</div> <div> </div> </div> <div><strong>This is the final result:</strong> a system capable of querying the indexed text(s) using natural language. In the following example we apply it to replying to queries about the future of LHC computing, based on the document <a href="https://arxiv.org/abs/1712.06982">A Roadmap for HEP Software and Computing R&amp;D for the 2020s</a></div> <div> <div> </div> <div class="separator" style="clear: both; text-align: center;"><a href="https://blogger.googleusercontent.com/img/b/R29vZ2xl/AVvXsEh9u-8TS_W0hSYeu511qXZqt0QvskrLzXpXVqrXDmYQuUOsK0a8jYmx_KvcX25XqpIT7ZlCY5G_4jeqrARvFynhvRTe6OWw7m3DiS25LFloIZc4KqMqVszoGOmIYIztH2x61KhD25K7b6NUeCbOv03dhmwh9ydzhm97rt4U6ni9NWuifM3ZrtHaZtheNHY/s948/semantic_search_LLM.png" style="margin-left: 1em; margin-right: 1em;"><img alt="" border="0" data-entity-type="" data-entity-uuid="" data-original-height="225" data-original-width="948" height="225" src="https://blogger.googleusercontent.com/img/b/R29vZ2xl/AVvXsEh9u-8TS_W0hSYeu511qXZqt0QvskrLzXpXVqrXDmYQuUOsK0a8jYmx_KvcX25XqpIT7ZlCY5G_4jeqrARvFynhvRTe6OWw7m3DiS25LFloIZc4KqMqVszoGOmIYIztH2x61KhD25K7b6NUeCbOv03dhmwh9ydzhm97rt4U6ni9NWuifM3ZrtHaZtheNHY/s948/semantic_search_LLM.png" width="948" /></a></div> <p> </p> <div> </div> </div> <h3 style="text-align:left">Conclusions</h3> <p>In this blog post, we have demonstrated how to build a beginner's semantic search system using vector databases and large language models (LLMs). Our example has utilized Jupyter notebooks with GPUs, Python packages, and a vector database, proving that a semantic search engine that queries documents for meaning, instead of just keywords, can be feasibly built using existing resources.</p> <p>In our implementation, we demonstrated how embeddings and indexing can be performed using FAISS as the vector library, or in alternative with OpenSearch as the vector database. We then moved onto the semantic query process using similarity search and vector DB indexes. To finalize the results, we utilized an LLM to convert the relevant document snippets into a coherent text answer.</p> <p>Though the example provided is not intended to function as a fully-developed search service, it serves as an excellent starting point and technological demonstrator for those interested in semantic search engines. Additionally, we acknowledge the potential of these methods to handle private documents and produce factually accurate results with original document references.</p> <p>We believe the combination of semantic search, vector databases, and large language models holds large potential for transforming how we approach information retrieval and natural language processing tasks.</p> <p><strong>The accompanying notebook, providing step-by-step code and more insights, is accessible on <a href="https://github.com/cerndb/NotebooksExamples/blob/main/AITools/LangChain_OpenSearch_semantic_search_with_Vector_DB.ipynb" target="_new">GitHub</a> and via the <a href="https://swan-gallery.web.cern.ch/">CERN SWAN Gallery</a>. </strong>For researchers and developers interested in delving into this exciting area of applied ML/AI, it offers a working example that can be run using CERN resources on SWAN, and also can run on Colab.</p> <div> </div> <h3>Acknowledgements</h3> <div>I would like to express my sincere gratitude to my colleagues at CERN for their invaluable assistance and insightful suggestions, in particular I'd like to acknowledge the CERN data analytics and web notebook services, the OpenSearch service, and the ATLAS database and data engineering teams. Their expertise and support have played a crucial role in making this collection of notebooks possible. Thank you for your contributions and dedication.</div> <div> </div> </div> </div> <span><a title="View user profile." href="/users/luca-canali" lang="" about="/users/luca-canali" typeof="schema:Person" property="schema:name" datatype="">canali</a></span> <span><time datetime="2023-06-22T11:20:34+02:00" title="Thursday, June 22, 2023 - 11:20">Thu, 06/22/2023 - 11:20</time> </span> <section> <h2>Add new comment</h2> <drupal-render-placeholder callback="comment.lazy_builders:renderForm" arguments="0=node&amp;1=191&amp;2=comment_node_blog_post&amp;3=comment_node_blog_post" token="V-gdtIMtT-K9Yg3SpqdeI34Wx8INCWy42qAyBH7poOs"></drupal-render-placeholder> </section> Thu, 22 Jun 2023 09:20:34 +0000 canali 191 at https://db-blog.web.cern.ch Exploratory Notebooks for Deep Learning, AI, and Data Tools: A Beginner's Guide https://db-blog.web.cern.ch/node/190 <span>Exploratory Notebooks for Deep Learning, AI, and Data Tools: A Beginner&#039;s Guide</span> <div class="field field--name-body field--type-text-with-summary field--label-above"> <div class="field--label"><b>Blog article:</b></div> <div class="field--item"><p>Are you looking at some resources to get you up to speed with popular Deep Learning and Data processing frameworks? This blog entry provides a curated collection of notebooks that will help you kickstart your journey.</p> <p>You can find the <a href="https://github.com/cerndb/NotebooksExamples">notebooks at this link</a>. See also the <a href="https://swan-gallery.web.cern.ch/deeplearning-GPU/">SWAN gallery</a>.</p> <p>CERN users can run the notebooks on the <a href="https://swan.web.cern.ch/swan/">SWAN</a> platform, using GPU resources. <a href="https://swan-k8s.cern.ch/user-redirect/download?projurl=https://github.com/cerndb/NotebooksExamples.git" rel="nofollow"><img alt="SWAN" data-canonical-src="https://swan.web.cern.ch/sites/swan.web.cern.ch/files/pictures/open_in_swan.svg" data-entity-type="" data-entity-uuid="" src="https://camo.githubusercontent.com/c4ad352b1fcc81d395f338dcc9e50e62d3874151954c68ba6974943a4d30af48/68747470733a2f2f7377616e2e7765622e6365726e2e63682f73697465732f7377616e2e7765622e6365726e2e63682f66696c65732f70696374757265732f6f70656e5f696e5f7377616e2e737667" style="max-width: 100%;" /></a></p> <p>Other options for running the notebooks in the cloud with a GPU include Google's <a href="https://colab.research.google.com/">Colab</a>.  <a href="https://colab.research.google.com/github/cerndb/NotebooksExamples" rel="nofollow"><img alt="Open in Colab" data-canonical-src="https://colab.research.google.com/assets/colab-badge.svg" data-entity-type="" data-entity-uuid="" src="https://camo.githubusercontent.com/84f0493939e0c4de4e6dbe113251b4bfb5353e57134ffd9fcab6b8714514d4d1/68747470733a2f2f636f6c61622e72657365617263682e676f6f676c652e636f6d2f6173736574732f636f6c61622d62616467652e737667" style="max-width: 100%;" /></a></p> <p> </p> <h2 dir="auto" tabindex="-1">Getting started with Deep Learning</h2> <p> </p> <p dir="auto">These notebook showcase a digit recognition classifier using the MNIST dataset, which serves as a "Hello World!" for Deep Learning. Choose from the following options to get started:</p> <ul dir="auto"><li><a href="https://github.com/cerndb/NotebooksExamples/blob/main/DeepLearning-GPU/TensorFlow_Keras_MNIST.ipynb">MNIST with TensorFlow Keras</a></li> <li><a href="https://github.com/cerndb/NotebooksExamples/blob/main/DeepLearning-GPU/PyTorch_MNIST.ipynb">MNIST with PyTorch</a></li> <li><a href="https://github.com/cerndb/NotebooksExamples/blob/main/DeepLearning-GPU/PyTorch_Lightning_MNIST.ipynb">MNIST with Pytorch Lightning</a></li> </ul><div class="separator" style="text-align:center"><span style="clear:both"><a href="https://blogger.googleusercontent.com/img/b/R29vZ2xl/AVvXsEiBVjC_HahQPANeh1i7AiIyRA7PIwdYcuNDSUbzJvnIfLwHi_6g31Q6Qk2OJ4eXVSVYGy8J-NGsowtTLnk2yACteS2crmbpQt-wezqODG4gbpnKniyl4_OMEwMljH6h7lzGn8MASPmZ-N35fq_fSxnF_HyyM8FRcFPWjZXF6j4mpt5KwhejfpdaRPEK/s1062/TensorFlow_Keras_MNIST.png" style="margin-left:13px; margin-right:13px"><img alt="" data-entity-type="" data-entity-uuid="" data-original-height="776" data-original-width="1062" src="https://blogger.googleusercontent.com/img/b/R29vZ2xl/AVvXsEiBVjC_HahQPANeh1i7AiIyRA7PIwdYcuNDSUbzJvnIfLwHi_6g31Q6Qk2OJ4eXVSVYGy8J-NGsowtTLnk2yACteS2crmbpQt-wezqODG4gbpnKniyl4_OMEwMljH6h7lzGn8MASPmZ-N35fq_fSxnF_HyyM8FRcFPWjZXF6j4mpt5KwhejfpdaRPEK/s1062/TensorFlow_Keras_MNIST.png" style="width:800px; height:586px" /></a></span></div> <p> </p> <h2 dir="auto" tabindex="-1">Deep Learning and basic Data pipelines</h2> <p dir="auto">Learn how to integrate Deep Learning frameworks with basic data pipelines using Pandas to feed data into the DL training step. These notebooks implement a Particle classifier using various DL frameworks. The data is stored in Parquet format, offering efficient data reading. </p> <div class="separator" style="text-align:center"><span style="clear:both"><br /><a href="https://blogger.googleusercontent.com/img/b/R29vZ2xl/AVvXsEiEyEw2bWT8fU-otjmComSVCxrgnKaQbWIfpcJ6VnMgIETRL2ShCSSlkYMmxnOadN7D_6zsqpx4KI0MOoxTkKQfRiPoqKZXLDHsihl0MbcqtcbpcPrsFe7MZTS-k7ql4QQIwl2RBstnKfjgjk4D4PBbTZBcB6kCTlBwt-PCpzzEignwa9XVqZ_qU9UZ/s1603/Physics_use_case.png" style="margin-left:13px; margin-right:13px"><img alt="" data-entity-type="" data-entity-uuid="" data-original-height="435" data-original-width="1603" src="https://blogger.googleusercontent.com/img/b/R29vZ2xl/AVvXsEiEyEw2bWT8fU-otjmComSVCxrgnKaQbWIfpcJ6VnMgIETRL2ShCSSlkYMmxnOadN7D_6zsqpx4KI0MOoxTkKQfRiPoqKZXLDHsihl0MbcqtcbpcPrsFe7MZTS-k7ql4QQIwl2RBstnKfjgjk4D4PBbTZBcB6kCTlBwt-PCpzzEignwa9XVqZ_qU9UZ/s1603/Physics_use_case.png" style="width:800px; height:218px" /></a></span></div> <p> </p> <ul dir="auto"><li><a href="https://github.com/cerndb/NotebooksExamples/blob/main/DeepLearning-GPU/TensorFlow_Keras_HLF_with_Pandas_Parquet.ipynb">TensorFlow classifier with data from Pandas</a></li> <li><a href="https://github.com/cerndb/NotebooksExamples/blob/main/DeepLearning-GPU/PyTorch_HLF_with_Pandas_Parquet.ipynb">Pytorch classifier with data from Pandas</a></li> <li><a href="https://github.com/cerndb/NotebooksExamples/blob/main/DeepLearning-GPU/PyTorch_Lightning_HLF_with_Pandas_Parquet.ipynb">Pytorch Lightning classifier with data from Pandas</a></li> <li><a href="https://github.com/cerndb/NotebooksExamples/blob/main/DeepLearning-GPU/XGBoost_with_Pandas_Parquet.ipynb">XGBoost classifier with data from Pandas</a></li> </ul><div> </div> <h2 dir="auto" tabindex="-1">More advanced Data pipelines</h2> <p dir="auto">Take your data processing skills to the next level with these notebooks, which demonstrate advanced data pipelines suitable for large datasets. Discover how to leverage the Petastorm library to read data from Parquet files with TensorFlow and PyTorch, as well as utilizing the TFRecord format with TensorFlow.</p> <ul dir="auto"><li><a href="https://github.com/cerndb/NotebooksExamples/blob/main/DeepLearning-GPU/TensorFlow_Keras_HLF_with_Petastorm_Parquet.ipynb">TensorFlow and Petastorm</a></li> <li><a href="https://github.com/cerndb/NotebooksExamples/blob/main/DeepLearning-GPU/PyTorch_HLF_with_Petastorm_Parquet.ipynb">PyTorch and Petastorm</a></li> <li><a href="https://github.com/cerndb/NotebooksExamples/blob/main/DeepLearning-GPU/TensorFlow_Keras_HLF_with_TFRecord.ipynb">TensorFlow with TFRecord</a></li> </ul><div> </div> <h2 dir="auto" tabindex="-1">Additional complexity with models and data</h2> <p dir="auto" style="color: rgb(0, 0, 0); font-family: &quot;Times New Roman&quot;; font-size: medium; font-style: normal; font-variant-ligatures: normal; font-variant-caps: normal; font-weight: 400; letter-spacing: normal; orphans: 2; text-align: left; text-indent: 0px; text-transform: none; widows: 2; word-spacing: 0px; -webkit-text-stroke-width: 0px; white-space: normal; text-decoration-thickness: initial; text-decoration-style: initial; text-decoration-color: initial;">Building upon the previous examples, these notebooks introduce more complex models and larger datasets for the Particle classifier. Explore the capabilities of TensorFlow, GRU, Transformer, and TFRecord with:</p> <ul style="color: rgb(0, 0, 0); font-family: &quot;Times New Roman&quot;; font-size: medium; font-style: normal; font-variant-ligatures: normal; font-variant-caps: normal; font-weight: 400; letter-spacing: normal; orphans: 2; text-align: left; text-indent: 0px; text-transform: none; widows: 2; word-spacing: 0px; -webkit-text-stroke-width: 0px; white-space: normal; text-decoration-thickness: initial; text-decoration-style: initial; text-decoration-color: initial;"><li><a data-original-attrs="{&quot;data-original-href&quot;:&quot;https://github.com/cerndb/NotebooksExamples/blob/main/DeepLearning-GPU/TensorFlow_Inclusive_Classifier_TFRecord.ipynb&quot;}" href="https://github.com/cerndb/NotebooksExamples/blob/main/DeepLearning-GPU/TensorFlow_Inclusive_Classifier_GRU_TFRecord.ipynb">TensorFlow for the Inclusive Classifier, with GRU and TFRecord</a> <ul><li>Description: This notebook focuses on training with data stored in TFRecord format.</li> <li>TensorFlow is configured to run on a GPU, and an LSTM-based model architecture is employed.</li> </ul></li> <li><a data-original-attrs="{&quot;data-original-href&quot;:&quot;https://github.com/cerndb/SparkDLTrigger/blob/master/Training_Inclusive_Classifier/TensorFlow_Inclusive_Classifier_Transformer_TFRecord.ipynb&quot;}" href="https://github.com/cerndb/SparkDLTrigger/blob/master/Training_Inclusive_Classifier/TensorFlow_Inclusive_Classifier_Transformer_TFRecord.ipynb">TensorFlow for the Inclusive Classifier, with Transformer and TFRecord</a> <ul><li>Description: This notebook focuses on training with data stored in TFRecord format.</li> <li>TensorFlow is configured to run on a GPU, and a Transformer-based model architecture is employed.</li> </ul></li> </ul><ul dir="auto"><li><span style="clear:both"><a href="https://blogger.googleusercontent.com/img/b/R29vZ2xl/AVvXsEhYAVZ3ZQJkONN90PT7oKs4pdUWsVoAO5RkkEw7NHE6j-Ezk-2JCPvD4rz91zBxmWBxjn0HWh9UU2vXcmVrBVBSc1teRbkRLKZic9Vz0_vgf-vCNlugDkhgcsMtKaYSB9bTFQmiY5XVLXEphK22U7R-yUBCNRlPpy1GX8UCCPzLuaaLWTHPgulfqdwp/s604/TensorFlow_Inclusive_Classifier_TFRecord.png" style="margin-left:13px; margin-right:13px"><img alt="" data-entity-type="" data-entity-uuid="" data-original-height="466" data-original-width="604" src="https://blogger.googleusercontent.com/img/b/R29vZ2xl/AVvXsEhYAVZ3ZQJkONN90PT7oKs4pdUWsVoAO5RkkEw7NHE6j-Ezk-2JCPvD4rz91zBxmWBxjn0HWh9UU2vXcmVrBVBSc1teRbkRLKZic9Vz0_vgf-vCNlugDkhgcsMtKaYSB9bTFQmiY5XVLXEphK22U7R-yUBCNRlPpy1GX8UCCPzLuaaLWTHPgulfqdwp/w640-h494/TensorFlow_Inclusive_Classifier_TFRecord.png" style="width:640px; height:494px" /></a></span></li> </ul><p> </p> <h1 dir="auto" tabindex="-1">AI Tools Examples</h1> <p dir="auto">This section contains Jupyter notebook examples of AI tools, including LLMs, Transformers, vector databases. The notebooks are intended to be run using GPU resources.</p> <h2 dir="auto" tabindex="-1"><span id="cke_bm_344S" style="display: none;"> </span>Transformers library</h2> <p dir="auto">Explore the powerful Transformers library from Hugging Face, widely used for LLM, Natural Language Processing (NLP), image, and speech tasks.</p> <ul dir="auto"><li dir="ltr"><a href="https://github.com/cerndb/NotebooksExamples/blob/main/DeepLearning-GPU/Transformers_text_example.ipynb">Transformers for text classification</a></li> <li dir="ltr"><a href="https://github.com/cerndb/NotebooksExamples/blob/main/DeepLearning-GPU/Transformers_image_example.ipynb">Transformers for image classifier</a></li> <li dir="ltr"><a href="https://github.com/cerndb/NotebooksExamples/blob/main/DeepLearning-GPU/Transformers_stable_diffusion_example.ipynb">Stable diffusion with transformers</a></li> <li dir="ltr"><a href="https://github.com/cerndb/NotebooksExamples/blob/main/DeepLearning-GPU/Transformers_speech_recognition.ipynb">Transformers for speech recognition</a></li> </ul><div class="separator" style="text-align:center"><span style="clear:both"></span></div> <div class="separator" style="text-align:center"><span style="clear:both"><br /><a href="https://blogger.googleusercontent.com/img/b/R29vZ2xl/AVvXsEg-ihP6AQKg8PQM-5GQ-CBZ4_SIDA5Y3BWiDTF89_zOLdVzRkIrlS69WidIObaUKdBA7ZLJ0NgxEv7GggnudwcfzXGX9f_ISVoEOj1IaTQ20fh1t8KlFRcx1FRlIH3JIq6G8i0jcFTE4KndSUJv-FCxJJ2TsmoeUWWZ59m7RI9QjgKYLcoz3RnXJ6ve/s1085/Transformers_stable_diffusion_example." style="margin-left:13px; margin-right:13px"><span style="clear:both"></span></a><a href="https://blogger.googleusercontent.com/img/b/R29vZ2xl/AVvXsEg-ihP6AQKg8PQM-5GQ-CBZ4_SIDA5Y3BWiDTF89_zOLdVzRkIrlS69WidIObaUKdBA7ZLJ0NgxEv7GggnudwcfzXGX9f_ISVoEOj1IaTQ20fh1t8KlFRcx1FRlIH3JIq6G8i0jcFTE4KndSUJv-FCxJJ2TsmoeUWWZ59m7RI9QjgKYLcoz3RnXJ6ve/s1085/Transformers_stable_diffusion_example." style="margin-left:13px; margin-right:13px"><img alt="" data-entity-type="" data-entity-uuid="" data-original-height="730" data-original-width="1085" src="https://blogger.googleusercontent.com/img/b/R29vZ2xl/AVvXsEg-ihP6AQKg8PQM-5GQ-CBZ4_SIDA5Y3BWiDTF89_zOLdVzRkIrlS69WidIObaUKdBA7ZLJ0NgxEv7GggnudwcfzXGX9f_ISVoEOj1IaTQ20fh1t8KlFRcx1FRlIH3JIq6G8i0jcFTE4KndSUJv-FCxJJ2TsmoeUWWZ59m7RI9QjgKYLcoz3RnXJ6ve/s1085/Transformers_stable_diffusion_example.png" style="width:800px; height:538px" /></a></span></div> <p> </p> <div> <h2 dir="auto" tabindex="-1">Large language models</h2> </div> <div> <p dir="auto">These notebooks provide examples of how to use LLMs in notebook environments for tests and prototyping</p> <ul dir="auto"><li><a href="https://github.com/cerndb/NotebooksExamples/blob/main/AITools/Transformers_Large_Language_Models.ipynb">Transformers Large Language Models</a></li> <li><a href="https://github.com/cerndb/NotebooksExamples/blob/main/AITools/LangChain_LLMs.ipynb">LangChain LLMs</a></li> </ul><div> </div> </div> <div> <h2 dir="auto" tabindex="-1">Semantic search with Vector Databases and LLM</h2> <p dir="auto">Semantic search allows to query a set of documents. This examples shows how to create vector embeddings, store them in a vector database, and perform semantic queries enhanced with LLM.</p> <ul dir="auto"><li><a href="https://github.com/cerndb/NotebooksExamples/blob/main/AITools/LangChain_OpenSearch_semantic_search_with_Vector_DB.ipynb">Semantic search with Vector Databases and LLM</a></li> </ul></div> <p dir="auto"> </p> <h1 dir="auto" tabindex="-1">Data Tools Examples</h1> <p dir="auto">This section offers example notebooks featuring popular frameworks and libraries for handling data. Please note that it does not cover scale-out data solutions such as Spark and Dask.</p> <p dir="auto">For Apache Spark see <a href="https://sparktraining.web.cern.ch/" rel="nofollow">SparkTraining</a></p> <p dir="auto">If you require access to relational databases for testing, CERN users can reach out to Oracle and DBOD services. You can also set up test databases using container technology. Here's how:</p> <p dir="auto">Running a test Oracle instance on a container:</p> <ul dir="auto"><li>Run Oracle Free on a container from gvenzl dockerhub repo <a href="https://github.com/gvenzl/oci-oracle-free">https://github.com/gvenzl/oci-oracle-free</a><br /><ul dir="auto"><li>see also <a href="https://github.com/gvenzl/oci-oracle-free">https://github.com/gvenzl/oci-oracle-free</a></li> <li><code>docker run -d --name mydb1 -e ORACLE_PASSWORD=oracle -p 1521:1521 gvenzl/oracle-free:latest</code></li> <li>Wait until the DB is started (this may take a few minutes). Check progress with: <code>docker logs -f mydb1</code></li> <li>install the Python library for connecting to Oracle: <code>pip install oracledb</code></li> </ul></li> </ul><p dir="auto">Setting up a PostgreSQL instance for testing using a Docker image:</p> <ul dir="auto"><li><code>docker run --name some-postgres -p 5432:5432 -e POSTGRES_PASSWORD=mysecretpassword -d postgres</code></li> <li>wait till the DB is started, check logs at: <code>docker logs -f some-postgres</code></li> <li>install the Python library for connecting to PostgreSQL: <code>pip install psycopg2-binary</code></li> </ul><h3 dir="auto" tabindex="-1">Pandas and numpy examples</h3> <ul dir="auto"><li><a href="https://github.com/cerndb/NotebooksExamples/blob/main/DataTools/Pandas_examples_with_plots.ipynb">Pandas examples with plots</a></li> <li><a href="https://github.com/cerndb/NotebooksExamples/blob/main/DataTools/Numpy_examples_with_plots.ipynb">Numpy examples with plots</a></li> </ul><div> <div> <h3 dir="auto" tabindex="-1">Database examples</h3> </div> <ul dir="auto"><li><a href="https://github.com/cerndb/NotebooksExamples/blob/main/DataTools/Query_Oracle.ipynb">Queries to Oracle databases from Jupyter</a></li> <li><a href="https://github.com/cerndb/NotebooksExamples/blob/main/DataTools/Query_PostgreSQL.ipynb">How to access PostgreSQL databases</a></li> </ul></div> <div> <div class="separator" style="text-align:center"><span style="clear:both"><br /><a href="https://blogger.googleusercontent.com/img/b/R29vZ2xl/AVvXsEg3PQbc-S6h41IKZZLdytoxcvB9g_YPZlIU5rHEtx8k1db6r_xIWkcXcwhbH-AOGe1nefUEn7aBJE_daWMC0di6d0lqXx7waDvSkPEBVJ48xD0zk6JgUVd_jE8jZrHvacIKN2upsJizyqANWwIHbKfsPKu8Mkr1aHtLhdiiWagv8TtbmcFc9BlmhdTg/s986/Numpy_examples_with_plots.png" style="margin-left:13px; margin-right:13px"><img alt="" data-entity-type="" data-entity-uuid="" data-original-height="948" data-original-width="986" src="https://blogger.googleusercontent.com/img/b/R29vZ2xl/AVvXsEg3PQbc-S6h41IKZZLdytoxcvB9g_YPZlIU5rHEtx8k1db6r_xIWkcXcwhbH-AOGe1nefUEn7aBJE_daWMC0di6d0lqXx7waDvSkPEBVJ48xD0zk6JgUVd_jE8jZrHvacIKN2upsJizyqANWwIHbKfsPKu8Mkr1aHtLhdiiWagv8TtbmcFc9BlmhdTg/s986/Numpy_examples_with_plots.png" style="width:800px; height:962px" /></a></span></div> </div> <p> </p> <div> <h1>Conclusions and acknowledgments</h1> <p>This blog entry provides a valuable collection of exploratory notebooks for individuals who are new to deep learning and data processing. With a focus on popular frameworks and libraries, these notebooks cover a range of topics including digit recognition, transformers for various tasks, integrating deep learning with data pipelines, advanced data processing techniques, and examples of data tools. Whether you are a CERN user or prefer cloud-based platforms like Google's Colab, these notebooks will help you quickly grasp the fundamentals and get started on your deep learning and data processing journey.</p> <p>I would like to express my sincere gratitude to my colleagues at CERN for their invaluable assistance and insightful suggestions, in particular I'd like to acknowledge the CERN data analytics and web notebook services and ATLAS database and data engineering teams. Their expertise and support have played a crucial role in making this collection of notebooks possible. Thank you for your contributions and dedication.</p> <p> </p> </div> <p> </p> </div> </div> <span><a title="View user profile." href="/users/luca-canali" lang="" about="/users/luca-canali" typeof="schema:Person" property="schema:name" datatype="">canali</a></span> <span><time datetime="2023-06-01T16:53:43+02:00" title="Thursday, June 1, 2023 - 16:53">Thu, 06/01/2023 - 16:53</time> </span> <div class="field field--name-field-tags field--type-entity-reference field--label-above"> <div class="field--label"><b>Tags</b></div> <div class="field--items"> <div class="field--item"><a href="/tags/deep-learning" hreflang="en">Deep learning</a></div> <div class="field--item"><a href="/tags/jupyter-notebook" hreflang="en">Jupyter notebook</a></div> </div> </div> <section> <h2>Add new comment</h2> <drupal-render-placeholder callback="comment.lazy_builders:renderForm" arguments="0=node&amp;1=190&amp;2=comment_node_blog_post&amp;3=comment_node_blog_post" token="LPUjvPxvIObmsanJFqQuN0Sa0ySutsxsTvvwjuKs2aQ"></drupal-render-placeholder> </section> Thu, 01 Jun 2023 14:53:43 +0000 canali 190 at https://db-blog.web.cern.ch CPU Load Testing Exercises: Tools and Analysis for Oracle Database Servers https://db-blog.web.cern.ch/node/189 <span>CPU Load Testing Exercises: Tools and Analysis for Oracle Database Servers</span> <div class="field field--name-body field--type-text-with-summary field--label-above"> <div class="field--label"><b>Blog article:</b></div> <div class="field--item"><p style="text-align:left"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}">This document describes some basic CPU load testing </font></font><span style="color: rgb(0, 0, 0); font-family: &quot;Times New Roman&quot;; font-size: medium; font-style: normal; font-variant-ligatures: normal; font-variant-caps: normal; font-weight: 400; letter-spacing: normal; orphans: 2; text-align: left; text-indent: 0px; text-transform: none; white-space: normal; widows: 2; word-spacing: 0px; -webkit-text-stroke-width: 0px; text-decoration-thickness: initial; text-decoration-style: initial; text-decoration-color: initial; display: inline !important; float: none;">exercises </span><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}">on three different types of database servers used by the Oracle Service at CERN. It reports on the tests performed, tools used for data gathering, data analysis, findings, and lessons learned.</font></font></span></span></span></span></span></span></span></span></span></span></p> <h2 style="text-align: left;"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}">Motivations</font></font></span></span></span></span></span></span></span></span></h2> <p style="text-align:left"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}">CPU usage is important for data processing: We observe that workloads on Oracle database services at CERN are often CPU-bound. Database workloads for transactional processing perform many random read operations. In the past, this mostly stressed the I/O subsystem, these days we deploy databases with large buffer caches (400 GB or more of data block caches) and most operations are CPU bound, reading data from buffer cache.</font></font></span></span></span></span></span></span></span></span></span></span></p> <p style="text-align:left"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}">Server consolidation, quality of service and licensing: We deploy on commodity HW considering various constraints: striking a balance between consolidating workloads and isolating critical workloads from different users’ communities. Moreover, Oracle licensing costs, which are proportional to the deployed CPUs, play a key input in the efforts streamlining the CPU deployments across the DB service.</font></font> </span></span></span></span></span></span></span></span></span></span></p> <p style="text-align:left"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"></span></span></span></span></span></span></span></span></span></span></p> <h2 style="text-align: left;"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}">Description and limitations of the tests</font></font></span></span></span></span></span></span></span></span></h2> <p style="text-align:left"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}">The tests reported here are extremely limited in scope, as they focus only on CPU performance and with two specific and “narrow” workloads. However, I believe they provide some indications on the behavior of the server CPU performance and the overall CPU capacity of the installed servers. The comparison between three different server models is the original motivation of this work as we wanted to understand how newer model can be deployed to replace old ones. This work is not a benchmark of the tested systems.</font></font> </span></span></span></span></span></span></span></span></span></span></p> <p style="text-align:left"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"></span></span></span></span></span></span></span></span></span></span></p> <h2 style="text-align: left;"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}">Tools used for load testing</font></font></span></span></span></span></span></span></span></span></h2> <p style="text-align:left"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial">The first workload generator and testing tool is a simple script burning CPU cycles in a loop and executed using multiple workers running in parallel, two implementations have been used, one in Python and one in Rust compiled to binary. Both provide similar results.</span></span></span></span></span></span></span></span></span></span></p> <p style="text-align:left"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial">The second workload generator is SLOB a tool that runs on top of Oracle databases for testing and specifically stresses “Logical IO”, that is reading blocks from the Oracle buffer cache (memory).</span></span></span></span></span></span></span></span></span></span></p> <p style="text-align:left"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}">Links to the code, measured data, and data analyses using notebooks:</font></font></span></span></span></span></span></span></span></span></span></span></p> <table class="Table" style="font-family:&quot;Times New Roman&quot;; text-decoration-thickness:initial; text-decoration-style:initial; text-decoration-color:initial"><tbody><tr><td data-original-attrs="{&quot;style&quot;:&quot;&quot;}" style="padding:1px"> <p data-original-attrs="{&quot;style&quot;:&quot;&quot;}"><span style="line-height:normal"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;lang&quot;:&quot;EN-GB&quot;}"><a data-original-attrs="{&quot;data-original-href&quot;:&quot;https://github.com/LucaCanali/Miscellaneous/blob/master/Performance_Testing/Test_CPU_parallel_Python&quot;}" href="https://github.com/LucaCanali/Miscellaneous/tree/master/Performance_Testing/Test_CPU_parallel_Python"><b><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US; mso-fareast-font-family: \&quot;Times New Roman\&quot;;&quot;,&quot;lang&quot;:&quot;EN-US&quot;}"><font style="font-size:12pt"><font face="&quot;Times New Roman&quot;,serif"><font color="#0000ff">CPU load testing kit - Python version</font></font></font></font></font></b></a></font></font></span></p> </td> <td data-original-attrs="{&quot;style&quot;:&quot;&quot;}" style="padding:1px"> <p data-original-attrs="{&quot;style&quot;:&quot;&quot;}"><span style="line-height:normal"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US; mso-fareast-font-family: \&quot;Times New Roman\&quot;;&quot;}"><font style="font-size:12pt"><font face="&quot;Times New Roman&quot;,serif">Kit for load testing and measuring CPU-intensive workloads, Python version.</font></font></font></font></span></p> </td> </tr><tr><td data-original-attrs="{&quot;style&quot;:&quot;&quot;}" style="padding:1px"> <p data-original-attrs="{&quot;style&quot;:&quot;&quot;}"><span style="line-height:normal"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;lang&quot;:&quot;EN-GB&quot;}"><a data-original-attrs="{&quot;data-original-href&quot;:&quot;https://github.com/LucaCanali/Miscellaneous/blob/master/Performance_Testing/Test_CPU_parallel_Rust&quot;}" href="https://github.com/LucaCanali/Miscellaneous/tree/master/Performance_Testing/Test_CPU_parallel_Rust"><b><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US; mso-fareast-font-family: \&quot;Times New Roman\&quot;;&quot;,&quot;lang&quot;:&quot;EN-US&quot;}"><font style="font-size:12pt"><font face="&quot;Times New Roman&quot;,serif"><font color="#0000ff">CPU load testing kit - Rust version</font></font></font></font></font></b></a></font></font></span></p> </td> <td data-original-attrs="{&quot;style&quot;:&quot;&quot;}" style="padding:1px"> <p data-original-attrs="{&quot;style&quot;:&quot;&quot;}"><span style="line-height:normal"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US; mso-fareast-font-family: \&quot;Times New Roman\&quot;;&quot;}"><font style="font-size:12pt"><font face="&quot;Times New Roman&quot;,serif">Kit for Load testing and measuring CPU-intensive workloads, Rust version.</font></font></font></font></span></p> </td> </tr><tr><td data-original-attrs="{&quot;style&quot;:&quot;&quot;}" style="padding:1px"> <p data-original-attrs="{&quot;style&quot;:&quot;&quot;}"><span style="line-height:normal"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;lang&quot;:&quot;EN-GB&quot;}"><a data-original-attrs="{&quot;data-original-href&quot;:&quot;https://github.com/LucaCanali/Miscellaneous/blob/master/Performance_Testing/Oracle_load_testing_with_SLOB&quot;}" href="https://github.com/LucaCanali/Miscellaneous/tree/master/Performance_Testing/Oracle_load_testing_with_SLOB"><b><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US; mso-fareast-font-family: \&quot;Times New Roman\&quot;;&quot;,&quot;lang&quot;:&quot;EN-US&quot;}"><font style="font-size:12pt"><font face="&quot;Times New Roman&quot;,serif"><font color="#0000ff">Oracle CPU load testing using SLOB</font></font></font></font></font></b></a></font></font></span></p> </td> <td data-original-attrs="{&quot;style&quot;:&quot;&quot;}" style="padding:1px"> <p data-original-attrs="{&quot;style&quot;:&quot;&quot;}"><span style="line-height:normal"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US; mso-fareast-font-family: \&quot;Times New Roman\&quot;;&quot;}"><font style="font-size:12pt"><font face="&quot;Times New Roman&quot;,serif">Load testing Oracle using the SLOB test kit.</font></font></font></font></span></p> </td> </tr></tbody></table><div data-original-attrs="{&quot;style&quot;:&quot;&quot;}" style="text-align:left"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"></span></span></span></span></span></span></span></span></span></span></div> <div data-original-attrs="{&quot;style&quot;:&quot;&quot;}" style="text-align:left"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"></span></span></span></span></span></span></span></span></span></span></div> <h2 style="text-align: left;"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}">Key findings</font></font></span></span></span></span></span></span></span></span></h2> <p data-original-attrs="{&quot;style&quot;:&quot;mso-add-space: auto; mso-list: l1 level1 lfo1;&quot;}" style="text-align:left; margin-left:27px; text-indent:-18pt"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US; mso-ascii-font-family: Calibri; mso-bidi-font-family: Calibri; mso-fareast-font-family: Calibri; mso-hansi-font-family: Calibri;&quot;}"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-list: Ignore;&quot;}">-<font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;&quot;}"><font style="font:7pt &quot;Times New Roman&quot;">          </font></font></font></font></font></font></font><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}">RAC55 is the newest server model of the three tested and shows the highest CPU per-thread performance and highest CPU total throughput at saturation.</font></font></span></span></span></span></span></span></span></span></span></span></p> <p data-original-attrs="{&quot;style&quot;:&quot;mso-add-space: auto; mso-list: l1 level1 lfo1;&quot;}" style="text-align:left; margin-left:27px; text-indent:-18pt"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US; mso-ascii-font-family: Calibri; mso-bidi-font-family: Calibri; mso-fareast-font-family: Calibri; mso-hansi-font-family: Calibri;&quot;}"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-list: Ignore;&quot;}">-<font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;&quot;}"><font style="font:7pt &quot;Times New Roman&quot;">          </font></font></font></font></font></font></font><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}">RAC55 has about 2.0x single-thread performance increase compared to RAC52.</font></font></span></span></span></span></span></span></span></span></span></span></p> <p data-original-attrs="{&quot;style&quot;:&quot;mso-add-space: auto; mso-list: l1 level1 lfo1;&quot;}" style="text-align:left; margin-left:27px; text-indent:-18pt"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US; mso-ascii-font-family: Calibri; mso-bidi-font-family: Calibri; mso-fareast-font-family: Calibri; mso-hansi-font-family: Calibri;&quot;}"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-list: Ignore;&quot;}">-<font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;&quot;}"><font style="font:7pt &quot;Times New Roman&quot;">          </font></font></font></font></font></font></font><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}">RAC55 has about 1.5x single-thread performance increase over RAC54, but this is valid only for low load, as RAC54 has only 8 physical cores vs 16 cores in RAC55. Moreover, RAC54 provides considerably less total CPU throughput compared to RAC52 and RAC55.</font></font></span></span></span></span></span></span></span></span></span></span></p> <p data-original-attrs="{&quot;style&quot;:&quot;mso-add-space: auto; mso-list: l1 level1 lfo1;&quot;}" style="text-align:left; margin-left:27px; text-indent:-18pt"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US; mso-ascii-font-family: Calibri; mso-bidi-font-family: Calibri; mso-fareast-font-family: Calibri; mso-hansi-font-family: Calibri;&quot;}"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-list: Ignore;&quot;}">-<font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;&quot;}"><font style="font:7pt &quot;Times New Roman&quot;">          </font></font></font></font></font></font></font><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}">RAC55 has about 2.0x more total CPU throughput at saturation compared to RAC52 despite having only 16 physical cores compared to 20 physical cores in RAC52.</font></font></span></span></span></span></span></span></span></span></span></span></p> <p style="text-align:left"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}"> </font></font></span></span></span></span></span></span></span></span></span></span></p> <h2 style="text-align: left;"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}">Description of the platforms</font></font></span></span></span></span></span></span></span></span></h2> <p style="text-align:left"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}">CPU load tests have been performed on three dedicated test servers representative of the production database servers in March 2023: RAC52, RAC54, and RAC55.</font></font></span></span></span></span></span></span></span></span></span></span></p> <p style="text-align:left"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}">The servers were installed with RHEL 7.9 and Oracle tests used Oracle 19c (v. 19.17). We omit the configuration of networking and I/O, as not relevant for these tests. We don't report the exact CPU models in this doc.</font></font></span></span></span></span></span></span></span></span></span></span></p> <p style="text-align:left"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}"> </font></font></span></span></span></span></span></span></span></span></span></span></p> <p style="text-align:left"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"></span></span></span></span></span></span></span></span></span></span></p> <p style="text-align:left"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><span style="line-height:normal"><b><span style="font-size:13.5pt"><span style="font-family:&quot;Times New Roman&quot;, serif"></span></span></b></span></span></span></span></span></span></span></span></span></span></span></p> <p style="margin-bottom:11px"><span style="font-size:11pt"><span style="line-height:normal"><span style="font-family:&quot;Calibri&quot;,sans-serif"><b><span style="font-size:13.5pt"><span style="font-family:&quot;Times New Roman&quot;,serif"><span style="color:black">RAC52 configuration:</span></span></span></b><span style="font-size:13.5pt"><span style="font-family:&quot;Times New Roman&quot;,serif"><span style="color:black"></span></span></span></span></span></span></p> <ul><li style="margin-left:8px"> <p><span style="font-size:12pt"><span style="font-family:&quot;Times New Roman&quot;,serif"><span style="font-size:13.5pt"><span style="font-family:Symbol"></span></span><span style="font-size:13.5pt"><span style="color:black">20 physical cores (2 sockets, 10 physical cores each), 40 logical cores visible on the OS due to hyperthreading</span></span></span></span></p> </li> <li style="margin-left:8px"> <p><span style="font-size:12pt"><span style="font-family:&quot;Times New Roman&quot;,serif"><span style="font-size:13.5pt"><span style="font-family:Symbol"></span></span><span style="font-size:13.5pt"><span style="color:black">CPU nominal frequency: 2.20 GHz</span></span></span></span></p> </li> <li style="margin-left:8px"> <p><span style="font-size:12pt"><span style="font-family:&quot;Times New Roman&quot;,serif"><span style="font-size:13.5pt"><span style="font-family:Symbol"></span></span><span style="font-size:13.5pt"><span style="color:black">CPU from 2016, L1 caches: 32K + 32K, L2 cache 256K, L3 cache 25600K</span></span></span></span></p> </li> <li style="margin-left:8px"> <p><span style="font-size:12pt"><span style="font-family:&quot;Times New Roman&quot;,serif"><span style="font-size:13.5pt"><span style="font-family:Symbol"></span></span><span style="font-size:13.5pt"><span style="color:black">RAM: DDR4, 512 GB</span></span></span></span></p> </li> </ul><p style="margin-bottom:11px"><span style="font-size:11pt"><span style="line-height:normal"><span style="font-family:&quot;Calibri&quot;,sans-serif"> </span></span></span></p> <p style="margin-bottom:11px"><span style="font-size:11pt"><span style="line-height:normal"><span style="font-family:&quot;Calibri&quot;,sans-serif"><b><span style="font-size:13.5pt"><span style="font-family:&quot;Times New Roman&quot;,serif"><span style="color:black">RAC54 configuration:</span></span></span></b><span style="font-size:13.5pt"><span style="font-family:&quot;Times New Roman&quot;,serif"><span style="color:black"></span></span></span></span></span></span></p> <ul><li style="margin-left:11px"> <p><span style="font-size:12pt"><span style="font-family:&quot;Times New Roman&quot;,serif"><span style="font-size:13.5pt"><span style="font-family:Symbol"></span></span><span style="font-size:13.5pt"><span style="color:black">8 physical cores (2 sockets, 4 physical cores each), 16 logical cores visible on the OS due to hyperthreading</span></span></span></span></p> </li> <li style="margin-left:11px"> <p><span style="font-size:12pt"><span style="font-family:&quot;Times New Roman&quot;,serif"><span style="font-size:13.5pt"><span style="font-family:Symbol"></span></span><span style="font-size:13.5pt"><span style="color:black">CPU nominal frequency: 3.80 GHz</span></span></span></span></p> </li> <li style="margin-left:11px"> <p><span style="font-size:12pt"><span style="font-family:&quot;Times New Roman&quot;,serif"><span style="font-size:13.5pt"><span style="font-family:Symbol"></span></span><span style="font-size:13.5pt"><span style="color:black">CPU from 2019, L1 caches: 32K + 32K, L2 cache 1024K, L3 cache 16896K</span></span></span></span></p> </li> <li style="margin-left:11px"> <p><span style="font-size:12pt"><span style="font-family:&quot;Times New Roman&quot;,serif"><span style="font-size:13.5pt"><span style="font-family:Symbol"></span></span><span style="font-size:13.5pt"><span style="color:black">RAM: DDR4, 768 GB</span></span></span></span></p> </li> </ul><p style="margin-bottom:11px"><span style="font-size:11pt"><span style="line-height:normal"><span style="font-family:&quot;Calibri&quot;,sans-serif"> </span></span></span></p> <p style="margin-bottom:11px"><span style="font-size:11pt"><span style="line-height:normal"><span style="font-family:&quot;Calibri&quot;,sans-serif"><b><span style="font-size:13.5pt"><span style="font-family:&quot;Times New Roman&quot;,serif"><span style="color:black">RAC55 configuration:</span></span></span></b><span style="font-size:13.5pt"><span style="font-family:&quot;Times New Roman&quot;,serif"><span style="color:black"></span></span></span></span></span></span></p> <ul><li style="margin-left:11px"> <p><span style="font-size:12pt"><span style="font-family:&quot;Times New Roman&quot;,serif"><span style="font-size:13.5pt"><span style="font-family:Symbol"></span></span><span style="font-size:13.5pt"><span style="color:black">16 physical cores (2 sockets, 8 physical cores each), 32 logical cores visible on the OS due to hyperthreading</span></span></span></span></p> </li> <li style="margin-left:11px"> <p><span style="font-size:12pt"><span style="font-family:&quot;Times New Roman&quot;,serif"><span style="font-size:13.5pt"><span style="font-family:Symbol"></span></span><span style="font-size:13.5pt"><span style="color:black">CPU nominal frequency: 3.7 GHz</span></span></span></span></p> </li> <li style="margin-left:11px"> <p><span style="font-size:12pt"><span style="font-family:&quot;Times New Roman&quot;,serif"><span style="font-size:13.5pt"><span style="font-family:Symbol"></span></span><span style="font-size:13.5pt"><span style="color:black">CPU from 2019, L1 caches: 32K + 32K, L2 cache 512K, L3 cache 32768K</span></span></span></span></p> </li> <li style="margin-left:11px"> <p><span style="font-size:12pt"><span style="font-family:&quot;Times New Roman&quot;,serif"><span style="font-size:13.5pt"><span style="font-family:Symbol"></span></span><span style="font-size:13.5pt"><span style="color:black">RAM: DDR4, 1 TB</span></span></span></span></p> </li> </ul><p style="text-align:left; margin-left:51px; text-indent:-18pt"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><span style="font-size:13.5pt"></span></span></span></span></span></span></span></span></span></span></span></p> <p data-original-attrs="{&quot;style&quot;:&quot;mso-add-space: auto; mso-list: l1 level1 lfo1;&quot;}" style="text-align:left; margin-left:27px; text-indent:-18pt"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"></span></span></span></span></span></span></span></span></span></span></p> <p style="text-align:left"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}"> </font></font> </span></span></span></span></span></span></span></span></span></span></p> <p style="text-align:left"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"></span></span></span></span></span></span></span></span></span></span></p> <h1 style="text-align:left"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}">Test 1 – Concurrent workers burning CPU cycles in a loop and in parallel</font></font></span></span></span></span></span></span></span></span></h1> <p style="text-align:left"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}">The workload generator and testing tool is a simple Python script burning CPU cycles in a loop.</font></font></span></span></span></span></span></span></span></span></span></span></p> <p style="text-align:left"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}">The script is executed running on a configurable number of concurrent workers. The script measures the time spent executing a simple CPU-burning loop.</font></font></span></span></span></span></span></span></span></span></span></span></p> <p style="text-align:left"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}">This provides a simple way to generate CPU load on the system.</font></font></span></span></span></span></span></span></span></span></span></span></p> <p style="text-align:left"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}">Example of how the data was collected with the testing tool written in Rust and compiled to binary:</font></font></span></span></span></span></span></span></span></span></span></span></p> <p style="text-align:left"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}"><font face="&quot;Courier New&quot;">./test_cpu_parallel --num_workers 8 --full --output myout.csv</font></font></font></span></span></span></span></span></span></span></span></span></span></p> <p style="text-align:left"> </p> <p style="text-align:left"><a href="https://github.com/LucaCanali/Miscellaneous/tree/master/Performance_Testing/Test_CPU_parallel_Rust"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}">See the code and instructions on how to run it at this link</font></font></span></span></span></span></span></span></span></span></span></span></a></p> <p style="text-align:left"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}">The advantage of this approach is that the testing tool is easy to write and can be easily automated.</font></font></span></span></span></span></span></span></span></span></span></span></p> <p style="text-align:left"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}">The weak point of testing this way is that the test workload is somewhat “artificial” and disconnected with the server actual purpose as a DB server. For example, the CPU-burning loop used for this test is mostly instruction-intensive on the CPU and does not spend much time on memory access.</font></font></span></span></span></span></span></span></span></span></span></span></p> <p style="text-align:left"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}"> </font></font></span></span></span></span></span></span></span></span></span></span></p> <h2 style="text-align:left"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}">Measurements and results:</font></font></span></span></span></span></span></span></span></span></h2> <p style="text-align:left"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}">See also <a href="https://github.com/LucaCanali/Miscellaneous/tree/master/Performance_Testing/Test_CPU_parallel_Rust">Data and Notebooks at this link</a> </font></font></span></span></span></span></span></span></span></span></span></span></p> <p style="text-align:left"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}">The following figures represent the same data in different ways to highlight different performance and scalability characteristics.</font></font></span></span></span></span></span></span></span></span></span></span></p> <p style="text-align:left"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"></span></span></span></span></span></span></span></span></span></span></p> <p style="text-align:left"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"></span></span></span></span></span></span></span></span></span></span></p> <div class="separator" data-original-attrs="{&quot;style&quot;:&quot;&quot;}" style="text-align:center"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><span style="clear:both"><a data-entity-type="" data-entity-uuid="" href="https://blogger.googleusercontent.com/img/b/R29vZ2xl/AVvXsEjFTJahNA0eLG0Y-WBXEzmRkerFEdlisnOKiiPzyhYArQ1JjcBGmdPEh3rYI-KebR6omSWVy3uQaHu-QoPgxBQc8FmS8aHqw95j9bku47rgDXe3hXtZc0N7hStuGoUwUiZOjDBSdjtyNg2YrlxRhVea3VuWlpiB9XZVtJSPitNrjYlzp1eOY5t_Bmn_/s1008/Figure1.png"><img alt="" data-entity-type="" data-entity-uuid="" data-original-height="468" data-original-width="1008" src="https://blogger.googleusercontent.com/img/b/R29vZ2xl/AVvXsEjFTJahNA0eLG0Y-WBXEzmRkerFEdlisnOKiiPzyhYArQ1JjcBGmdPEh3rYI-KebR6omSWVy3uQaHu-QoPgxBQc8FmS8aHqw95j9bku47rgDXe3hXtZc0N7hStuGoUwUiZOjDBSdjtyNg2YrlxRhVea3VuWlpiB9XZVtJSPitNrjYlzp1eOY5t_Bmn_/s1008/Figure1.png" style="cursor:move; width:1024px; height:475px" /></a></span></span></span></span></span></span></span></span></span></span></span></div> <p style="text-align:left"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"></span></span></span></span></span></span></span></span></span></span></p> <p style="text-align:left"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><b>Figure 1 – Raw data</b></span></span></span></span></span></span></span></span></span></span></p> <p data-original-attrs="{&quot;style&quot;:&quot;mso-add-space: auto; mso-list: l1 level1 lfo1;&quot;}" style="text-align:left; margin-left:27px; text-indent:-18pt"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US; mso-ascii-font-family: Calibri; mso-bidi-font-family: Calibri; mso-fareast-font-family: Calibri; mso-hansi-font-family: Calibri;&quot;}"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-list: Ignore;&quot;}">-<font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;&quot;}"><font style="font:7pt &quot;Times New Roman&quot;">          </font></font></font></font></font></font></font><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}">The figure reports the testing job execution time, measured for varying server load on the three tested servers.</font></font></span></span></span></span></span></span></span></span></span></span></p> <p data-original-attrs="{&quot;style&quot;:&quot;mso-add-space: auto; mso-list: l1 level1 lfo1;&quot;}" style="text-align:left; margin-left:27px; text-indent:-18pt"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US; mso-ascii-font-family: Calibri; mso-bidi-font-family: Calibri; mso-fareast-font-family: Calibri; mso-hansi-font-family: Calibri;&quot;}"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-list: Ignore;&quot;}">-<font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;&quot;}"><font style="font:7pt &quot;Times New Roman&quot;">          </font></font></font></font></font></font></font><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}">A common pattern is that at low load (see data with just a few parallel workers) the job run time is almost constant.</font></font></span></span></span></span></span></span></span></span></span></span></p> <p data-original-attrs="{&quot;style&quot;:&quot;mso-add-space: auto; mso-list: l1 level1 lfo1;&quot;}" style="text-align:left; margin-left:27px; text-indent:-18pt"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US; mso-ascii-font-family: Calibri; mso-bidi-font-family: Calibri; mso-fareast-font-family: Calibri; mso-hansi-font-family: Calibri;&quot;}"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-list: Ignore;&quot;}">-<font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;&quot;}"><font style="font:7pt &quot;Times New Roman&quot;">          </font></font></font></font></font></font></font><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}">An important difference is that the job run time is different on the different platforms, in order of increasing performance: RAC52, RAC54, RAC55 (the newest server and the fastest).</font></font></span></span></span></span></span></span></span></span></span></span></p> <p data-original-attrs="{&quot;style&quot;:&quot;mso-add-space: auto; mso-list: l1 level1 lfo1;&quot;}" style="text-align:left; margin-left:27px; text-indent:-18pt"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US; mso-ascii-font-family: Calibri; mso-bidi-font-family: Calibri; mso-fareast-font-family: Calibri; mso-hansi-font-family: Calibri;&quot;}"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-list: Ignore;&quot;}">-<font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;&quot;}"><font style="font:7pt &quot;Times New Roman&quot;">          </font></font></font></font></font></font></font><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}">Another pattern is that the job running starts to increase linearly at higher load.</font></font></span></span></span></span></span></span></span></span></span></span></p> <p data-original-attrs="{&quot;style&quot;:&quot;mso-add-space: auto; mso-list: l1 level1 lfo1;&quot;}" style="text-align:left; margin-left:27px; text-indent:-18pt"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US; mso-ascii-font-family: Calibri; mso-bidi-font-family: Calibri; mso-fareast-font-family: Calibri; mso-hansi-font-family: Calibri;&quot;}"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-list: Ignore;&quot;}">-<font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;&quot;}"><font style="font:7pt &quot;Times New Roman&quot;">          </font></font></font></font></font></font></font><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}">The job execution time curve starts to bend upwards as the load increases. Typically, we see this happening when the num of parallel workers is greater than the number of physical cores on the server (20 cores on RAC52, 8 cores on RAC54 and 16 cores on RAC55)</font></font></span></span></span></span></span></span></span></span></span></span></p> <p data-original-attrs="{&quot;style&quot;:&quot;mso-add-space: auto; mso-list: l1 level1 lfo1;&quot;}" style="text-align:left; margin-left:27px; text-indent:-18pt"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}">  </font></font></p> <p style="text-align:left"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-no-proof: yes;&quot;,&quot;lang&quot;:&quot;EN-GB&quot;}"><span alt="Chart, scatter chart&lt;/p&gt;&#10;&lt;p&gt;Description automatically generated" data-original-attrs="{&quot;o:spid&quot;:&quot;_x0000_i1031&quot;,&quot;style&quot;:&quot;mso-wrap-style: square;&quot;,&quot;type&quot;:&quot;#_x0000_t75&quot;}" data-original-tag="V:SHAPE" id="Picture_x0020_12" style="height:336.75pt; width:726pt"></span></font></font></span></span></span></span></span></span></span></span></span></span></p> <p style="text-align:left"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}"> </font></font></span></span></span></span></span></span></span></span></span></span></p> <p style="text-align:left"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"></span></span></span></span></span></span></span></span></span></span></p> <div class="separator" data-original-attrs="{&quot;style&quot;:&quot;&quot;}" style="text-align:center"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><span style="clear:both"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}"><a data-entity-type="" data-entity-uuid="" href="https://blogger.googleusercontent.com/img/b/R29vZ2xl/AVvXsEiHWanVlcUTEleKp2LZeHOlf-V8pd38Ee7j_XK6qDykXcIENlAv2ApwXeN-k4MZ--YZwK5GWPJDq2BFOlIErFGnTQ6udeUUGp5wGfdeOFo6UPJFi-JsTHLiOCtBCBvVHKkcNc_UfTBwfYfKuQbyrN0dnBKVBMad4wXSdYkwtxd0YMpkmemRzQuq-x24/s1003/Figure2.png"><img alt="" data-entity-type="" data-entity-uuid="" data-original-height="468" data-original-width="1003" src="https://blogger.googleusercontent.com/img/b/R29vZ2xl/AVvXsEiHWanVlcUTEleKp2LZeHOlf-V8pd38Ee7j_XK6qDykXcIENlAv2ApwXeN-k4MZ--YZwK5GWPJDq2BFOlIErFGnTQ6udeUUGp5wGfdeOFo6UPJFi-JsTHLiOCtBCBvVHKkcNc_UfTBwfYfKuQbyrN0dnBKVBMad4wXSdYkwtxd0YMpkmemRzQuq-x24/s1003/Figure2.png" style="cursor:move; width:1024px; height:479px" /></a></font></font></span></span></span></span></span></span></span></span></span></span></span></div> <p style="text-align:left"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"></span></span></span></span></span></span></span></span></span></span></p> <p style="text-align:left"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><b>Figure 2 - Speed</b></span></span></span></span></span></span></span></span></span></span></p> <p data-original-attrs="{&quot;style&quot;:&quot;mso-add-space: auto; mso-list: l1 level1 lfo1;&quot;}" style="text-align:left; margin-left:27px; text-indent:-18pt"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US; mso-ascii-font-family: Calibri; mso-bidi-font-family: Calibri; mso-fareast-font-family: Calibri; mso-hansi-font-family: Calibri;&quot;}"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-list: Ignore;&quot;}">-<font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;&quot;}"><font style="font:7pt &quot;Times New Roman&quot;">          </font></font></font></font></font></font></font><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}">This plot reports the number of jobs per minute per worker</font></font></span></span></span></span></span></span></span></span></span></span></p> <p data-original-attrs="{&quot;style&quot;:&quot;mso-add-space: auto; mso-list: l1 level1 lfo1;&quot;}" style="text-align:left; margin-left:27px; text-indent:-18pt"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US; mso-ascii-font-family: Calibri; mso-bidi-font-family: Calibri; mso-fareast-font-family: Calibri; mso-hansi-font-family: Calibri;&quot;}"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-list: Ignore;&quot;}">-<font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;&quot;}"><font style="font:7pt &quot;Times New Roman&quot;">          </font></font></font></font></font></font></font><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}">Data points can be interpreted as a measure of the “speed of the CPU” for a new job coming into the system given a defined system load</font></font></span></span></span></span></span></span></span></span></span></span></p> <p data-original-attrs="{&quot;style&quot;:&quot;mso-add-space: auto; mso-list: l1 level1 lfo1;&quot;}" style="text-align:left; margin-left:27px; text-indent:-18pt"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US; mso-ascii-font-family: Calibri; mso-bidi-font-family: Calibri; mso-fareast-font-family: Calibri; mso-hansi-font-family: Calibri;&quot;}"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-list: Ignore;&quot;}">-<font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;&quot;}"><font style="font:7pt &quot;Times New Roman&quot;">          </font></font></font></font></font></font></font><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}">We see that the “effective CPU speed” decreases as the load increases, with sudden changes at the points where the number of parallel workers is equal to the number of physical cores</font></font></span></span></span></span></span></span></span></span></span></span></p> <p data-original-attrs="{&quot;style&quot;:&quot;mso-add-space: auto; mso-list: l1 level1 lfo1;&quot;}" style="text-align:left; margin-left:27px; text-indent:-18pt"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US; mso-ascii-font-family: Calibri; mso-bidi-font-family: Calibri; mso-fareast-font-family: Calibri; mso-hansi-font-family: Calibri;&quot;}"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-list: Ignore;&quot;}">-<font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;&quot;}"><font style="font:7pt &quot;Times New Roman&quot;">          </font></font></font></font></font></font></font><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}">The CPU speed per thread is also different depending on the CPU architecture, in order of increasing performance: RAC52, RAC54, RAC55 (the newest server and the fastest).</font></font></span></span></span></span></span></span></span></span></span></span></p> <p data-original-attrs="{&quot;style&quot;:&quot;mso-add-space: auto; mso-list: l1 level1 lfo1;&quot;}" style="text-align:left; margin-left:27px; text-indent:-18pt"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}">  </font></font></span></span></span></span></span></span></span></span></span></span><br data-original-attrs="{&quot;style&quot;:&quot;mso-special-character: line-break;&quot;}" /><br />  </p> <p data-original-attrs="{&quot;style&quot;:&quot;mso-add-space: auto; mso-list: l1 level1 lfo1;&quot;}" style="text-align:left; margin-left:27px; text-indent:-18pt"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"></span></span></span></span></span></span></span></span></span></span></p> <p data-original-attrs="{&quot;style&quot;:&quot;mso-add-space: auto; mso-list: l1 level1 lfo1;&quot;}" style="text-align:left; margin-left:27px; text-indent:-18pt"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"></span></span></span></span></span></span></span></span></span></span></p> <div class="separator" data-original-attrs="{&quot;style&quot;:&quot;&quot;}" style="text-align:center"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><span style="clear:both"><a data-entity-type="" data-entity-uuid="" href="https://blogger.googleusercontent.com/img/b/R29vZ2xl/AVvXsEjQ7_49Zm-EWMt_EVJ_gY3RpzPmJvHvJ6eBM4w_9js54Wu2Xv06vZHDLPagXI_OEPgc0SWvaiQ3O7M1li7a3pK1veWaqfsZawXKC9FRDyjvpckogFCOCSferZfmXpxJrKHFRbDkUXuCTJLgwn7zPCLw86K687OeCaClZI7Uqz5S4PIihhprHmGp485x/s999/Figure3.png"><img alt="" data-entity-type="" data-entity-uuid="" data-original-height="468" data-original-width="999" src="https://blogger.googleusercontent.com/img/b/R29vZ2xl/AVvXsEjQ7_49Zm-EWMt_EVJ_gY3RpzPmJvHvJ6eBM4w_9js54Wu2Xv06vZHDLPagXI_OEPgc0SWvaiQ3O7M1li7a3pK1veWaqfsZawXKC9FRDyjvpckogFCOCSferZfmXpxJrKHFRbDkUXuCTJLgwn7zPCLw86K687OeCaClZI7Uqz5S4PIihhprHmGp485x/s999/Figure3.png" style="cursor:move; width:1024px; height:480px" /></a></span></span></span></span></span></span></span></span></span></span></span></div> <p data-original-attrs="{&quot;style&quot;:&quot;mso-add-space: auto; mso-list: l1 level1 lfo1;&quot;}" style="text-align:left; margin-left:27px; text-indent:-18pt"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"></span><span style="text-decoration-color:initial"><b>Figure 3 - Capacity</b></span></span></span></span></span></span></span></span></span></span></p> <p data-original-attrs="{&quot;style&quot;:&quot;mso-add-space: auto; mso-list: l1 level1 lfo1;&quot;}" style="text-align:left; margin-left:27px; text-indent:-18pt"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US; mso-ascii-font-family: Calibri; mso-bidi-font-family: Calibri; mso-fareast-font-family: Calibri; mso-hansi-font-family: Calibri;&quot;}"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-list: Ignore;&quot;}">-<font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;&quot;}"><font style="font:7pt &quot;Times New Roman&quot;">          </font></font></font></font></font></font></font><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}">This plot shows the number of jobs executed per minute summed over all the running worker threads.</font></font></span></span></span></span></span></span></span></span></span></span></p> <p data-original-attrs="{&quot;style&quot;:&quot;mso-add-space: auto; mso-list: l1 level1 lfo1;&quot;}" style="text-align:left; margin-left:27px; text-indent:-18pt"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US; mso-ascii-font-family: Calibri; mso-bidi-font-family: Calibri; mso-fareast-font-family: Calibri; mso-hansi-font-family: Calibri;&quot;}"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-list: Ignore;&quot;}">-<font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;&quot;}"><font style="font:7pt &quot;Times New Roman&quot;">          </font></font></font></font></font></font></font><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}">As the load increases the server capacity increases, reaching a maximum value at number of workers = number of logical cores (40 for RAC52, 8 for RAC54, 32 for RAC55)</font></font></span></span></span></span></span></span></span></span></span></span></p> <p data-original-attrs="{&quot;style&quot;:&quot;mso-add-space: auto; mso-list: l1 level1 lfo1;&quot;}" style="text-align:left; margin-left:27px; text-indent:-18pt"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US; mso-ascii-font-family: Calibri; mso-bidi-font-family: Calibri; mso-fareast-font-family: Calibri; mso-hansi-font-family: Calibri;&quot;}"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-list: Ignore;&quot;}">-<font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;&quot;}"><font style="font:7pt &quot;Times New Roman&quot;">          </font></font></font></font></font></font></font><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}">This allows to compare the “Total CPU capacity” of the three servers. In order of increasing capacity: lowest capacity with RAC54 (the server with fewer cores), then RAC52, finally RAC55 has the highest CPU throughput (it’s the newest server)</font></font></span></span></span></span></span></span></span></span></span></span></p> <p data-original-attrs="{&quot;style&quot;:&quot;mso-add-space: auto; mso-list: l1 level1 lfo1;&quot;}" style="text-align:left; margin-left:27px; text-indent:-18pt"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"></span></span></span></span></span></span></span></span></span></span></p> <p data-original-attrs="{&quot;style&quot;:&quot;mso-add-space: auto; mso-list: l1 level1 lfo1;&quot;}" style="text-align:left; margin-left:27px; text-indent:-18pt"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"></span></span></span></span></span></span></span></span></span></span></p> <div class="separator" data-original-attrs="{&quot;style&quot;:&quot;&quot;}" style="text-align:center"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><span style="clear:both"></span></span></span></span></span></span></span></span></span></span></span></div> <p data-original-attrs="{&quot;style&quot;:&quot;mso-add-space: auto; mso-list: l1 level1 lfo1;&quot;}" style="text-align:left; margin-left:27px; text-indent:-18pt"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"></span></span></span></span></span></span></span></span></span></span> </p> <div class="separator" data-original-attrs="{&quot;style&quot;:&quot;&quot;}" style="text-align:center"> <p><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><span style="clear:both"><br /><a data-entity-type="" data-entity-uuid="" href="https://blogger.googleusercontent.com/img/b/R29vZ2xl/AVvXsEhPECMvv1x6Chz-s5cx--T9TWJxMBPcjA1BJVYqLb4rcSkmVutUbO8tO_UoaYnAuqlOqeT_NzDXndNw2dakrL_9vjyVqqzoVH3mlY3CVG4ogbQxGsFjo5ZOUAg3xFw99InY6svHRQ7zr78rSOeThVLc-8ITpEZcfE9NgBDTQfB4lrOY8qp1aqBeLUlE/s1012/Figure4.png"><img alt="" data-entity-type="" data-entity-uuid="" data-original-height="585" data-original-width="1012" src="https://blogger.googleusercontent.com/img/b/R29vZ2xl/AVvXsEhPECMvv1x6Chz-s5cx--T9TWJxMBPcjA1BJVYqLb4rcSkmVutUbO8tO_UoaYnAuqlOqeT_NzDXndNw2dakrL_9vjyVqqzoVH3mlY3CVG4ogbQxGsFjo5ZOUAg3xFw99InY6svHRQ7zr78rSOeThVLc-8ITpEZcfE9NgBDTQfB4lrOY8qp1aqBeLUlE/s1012/Figure4.png" style="cursor:move; width:1024px; height:474px" /></a></span></span></span></span></span></span></span></span></span></span></span></p> </div> <p data-original-attrs="{&quot;style&quot;:&quot;mso-add-space: auto; mso-list: l1 level1 lfo1;&quot;}" style="text-align:left; margin-left:27px; text-indent:-18pt"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><b>Figure 4 - Scalability</b></span></span></span></span></span></span></span></span></span></span></p> <p data-original-attrs="{&quot;style&quot;:&quot;mso-add-space: auto; mso-list: l1 level1 lfo1;&quot;}" style="text-align:left; margin-left:27px; text-indent:-18pt"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-list: Ignore;&quot;}">-<font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;&quot;}"><font style="font:7pt &quot;Times New Roman&quot;">          </font></font></font></font></font>This shows the speedup, a measure of scalability. For the scope of this plot, speedup is calculated as N * (job execution time at load n) / (job execution time at load 1)</font></font></span></span></span></span></span></span></span></span></span></span></p> <p data-original-attrs="{&quot;style&quot;:&quot;mso-add-space: auto; mso-list: l1 level1 lfo1;&quot;}" style="text-align:left; margin-left:27px; text-indent:-18pt"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US; mso-ascii-font-family: Calibri; mso-bidi-font-family: Calibri; mso-fareast-font-family: Calibri; mso-hansi-font-family: Calibri;&quot;}"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-list: Ignore;&quot;}">-<font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;&quot;}"><font style="font:7pt &quot;Times New Roman&quot;">          </font></font></font></font></font></font></font><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}">We see almost linear scalability for low loads (up to the number of physical cores), then a slower increase up to the number of logical cores, and, eventually, the speedup reaches saturation</font></font></span></span></span></span></span></span></span></span></span></span></p> <p data-original-attrs="{&quot;style&quot;:&quot;mso-add-space: auto; mso-list: l1 level1 lfo1;&quot;}" style="text-align:left; margin-left:27px; text-indent:-18pt"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US; mso-ascii-font-family: Calibri; mso-bidi-font-family: Calibri; mso-fareast-font-family: Calibri; mso-hansi-font-family: Calibri;&quot;}"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-list: Ignore;&quot;}">-<font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;&quot;}"><font style="font:7pt &quot;Times New Roman&quot;">          </font></font></font></font></font></font></font><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}">RAC54 and RAC55 appear to scale almost linearly up to the number of physical cores (respectively 8 and 16)</font></font></span></span></span></span></span></span></span></span></span></span></p> <p data-original-attrs="{&quot;style&quot;:&quot;mso-add-space: auto; mso-list: l1 level1 lfo1;&quot;}" style="text-align:left; margin-left:27px; text-indent:-18pt"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"></span></span></span></span></span></span></span></span></span></span></p> <h3 data-original-attrs="{&quot;style&quot;:&quot;&quot;}" style="text-align:left; margin-left:27px; text-indent:-18pt"> </h3> <h3 data-original-attrs="{&quot;style&quot;:&quot;&quot;}" style="text-align:left; margin-left:27px; text-indent:-18pt"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial">Notes:</span></span></span></span></span></span></span></span></h3> <p data-original-attrs="{&quot;style&quot;:&quot;mso-add-space: auto; mso-list: l1 level1 lfo1;&quot;}" style="text-align:left; margin-left:27px; text-indent:-18pt"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US; mso-ascii-font-family: Calibri; mso-bidi-font-family: Calibri; mso-fareast-font-family: Calibri; mso-hansi-font-family: Calibri;&quot;}"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-list: Ignore;&quot;}">-<font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;&quot;}"><font style="font:7pt &quot;Times New Roman&quot;">          </font></font></font></font></font></font></font><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}">Of the tested servers RAC55 appears the fastest on per-thread CPU performance at low and high loads and the one with higher CPU capacity.</font></font></span></span></span></span></span></span></span></span></span></span></p> <p data-original-attrs="{&quot;style&quot;:&quot;mso-add-space: auto; mso-list: l1 level1 lfo1;&quot;}" style="text-align:left; margin-left:27px; text-indent:-18pt"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US; mso-ascii-font-family: Calibri; mso-bidi-font-family: Calibri; mso-fareast-font-family: Calibri; mso-hansi-font-family: Calibri;&quot;}"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-list: Ignore;&quot;}">-<font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;&quot;}"><font style="font:7pt &quot;Times New Roman&quot;">          </font></font></font></font></font></font></font><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}">The difference in performance between RAC52 (oldest) and RAC55 (newest) is roughly x1.5 in per-CPU thread performance and x2 in overall CPU capacity at high load.</font></font></span></span></span></span></span></span></span></span></span></span></p> <p data-original-attrs="{&quot;style&quot;:&quot;mso-add-space: auto; mso-list: l1 level1 lfo1;&quot;}" style="text-align:left; margin-left:27px; text-indent:-18pt"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US; mso-ascii-font-family: Calibri; mso-bidi-font-family: Calibri; mso-fareast-font-family: Calibri; mso-hansi-font-family: Calibri;&quot;}"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-list: Ignore;&quot;}">-<font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;&quot;}"><font style="font:7pt &quot;Times New Roman&quot;">          </font></font></font></font></font></font></font><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}">RAC54 performs similarly to RAC55 but only at low loads (&lt;= 8 concurrent workers)</font></font></span></span></span></span></span></span></span></span></span></span></p> <p style="text-align:left"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}"> </font></font></span></span></span></span></span></span></span></span></span></span></p> <h1 style="text-align:left"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}">Test 2 – Parallel workers running “SLOB tests”, measuring Oracle logical IO throughput</font></font></span></span></span></span></span></span></span></span></h1> <p style="text-align:left"> </p> <p style="text-align:left"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}">The second workload generator is SLOB, a tool by Kevin Closson, that runs on top of Oracle databases for load testing and specifically stresses Physical and Logical IO. In the configuration used for these tests we only stressed Logical IO, that is accessing blocks from the Oracle buffer cache (memory).</font></font></span></span></span></span></span></span></span></span></span></span></p> <p style="text-align:left"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}">The tool creates test tables on the database and performs block IO reading from the test tables with a tunable number of concurrent workers.</font></font></span></span></span></span></span></span></span></span></span></span></p> <p style="text-align:left"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}"><a href="https://kevinclosson.net/slob/">See also the official SLOB page</a></font></font>.</span></span></span></span></span></span></span></span></span></span></p> <p style="text-align:left"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}">The Oracle database used for testing was configured with a large SGA (Oracle’s shared memory area), <font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-spacerun: yes;&quot;}"> </font></font>able to cache the test tables in the Oracle buffer cache. The test workload was therefore stressing Oracle’s “Logical IO”, that is the part of the Oracle internal code that takes care of reading data blocks from the buffer cache (Oracle’s consistent read operations). Minimal additional operations were performed on the blocks accessed this way by SLOB. Logical IO is an operation with duration of the order of 1 microsecond during these tests. Notably, it includes time for Oracle-internals operations and serialization needed for reading data from the Oracle buffer cache, and finally the Oracle-internals code for reading the block header. The SLOB Logical IO test workload does not appear to saturate the CPU-memory channel on the tested servers (for example on RAC55, using OS tools we could see the CPU-memory bandwidth being utilized during SLOB logical I/O tests went up to 40 GB/s, while using memory-intensive load generators the system could scale at least up to 220 GB/s, see also </font></font><font data-keep-original-tag="false"><font data-original-attrs="{&quot;lang&quot;:&quot;EN-GB&quot;}"><a data-original-attrs="{&quot;data-original-href&quot;:&quot;https://github.com/LucaCanali/Miscellaneous/blob/master/Performance_Testing/Tools_Linux_Memory_Perf_Measure.md&quot;}" href="https://github.com/LucaCanali/Miscellaneous/blob/master/Performance_Testing/Tools_Linux_Memory_Perf_Measure.md"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;,&quot;lang&quot;:&quot;EN-US&quot;}">Tools_Linux_Memory_Perf_Measure</font></font></a></font></font></span></span></span></span></span></span></span></span></span></span></p> <p style="text-align:left"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}">Standard Oracle instrumentation, notably SQL*Plus (Oracle’ CLI tool) and AWR reports (Oracle’s performance report), was used to collect the number of Logical I/Os recorded during the test and for other sanity checks, notably checking that the test workload was fully reading from memory rather than performing Physical I/O.</font></font></span></span></span></span></span></span></span></span></span></span></p> <p style="text-align:left"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}">SLOB test tables have been created with size 16 GB per user, each concurrent worker running on a dedicated user. Tests for load higher than 16 concurrent users have used 8 GB test tables per user. The Oracle buffer cache for testing was allocated using Linux large pages, as it is recommended by Oracle, and was 400 GB in size, therefore large enough to cache all the test data. </font></font><span style="color: rgb(0, 0, 0); font-family: &quot;Times New Roman&quot;; font-size: medium; font-style: normal; font-variant-ligatures: normal; font-variant-caps: normal; font-weight: 400; letter-spacing: normal; orphans: 2; text-align: left; text-indent: 0px; text-transform: none; white-space: normal; widows: 2; word-spacing: 0px; -webkit-text-stroke-width: 0px; text-decoration-thickness: initial; text-decoration-style: initial; text-decoration-color: initial; display: inline !important; float: none;">We took care of caching Oracle tables' data with a couple of executions of the test workload before starting to measure Logical I/O performance.</span><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}"> The fact that Oracle was not performing Physical I/O during the tests was also double-checked using monitoring tools and by inspection of the AWR reports.</font></font></span></span></span></span></span></span></span></span></span></span></p> <p style="text-align:left"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}">The advantage of this approach is that this is stressing the system on a key operation for the Oracle DB workloads that are CPU-bound: Oracle RDBMS Logical I/O.</font></font></span></span></span></span></span></span></span></span></span></span></p> <p style="text-align:left"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}">These tests take longer to run the simple “Tests with a CPU-burning script” described above, moreover they require some Oracle DBA expertise to configure and validate the test results.</font></font></span></span></span></span></span></span></span></span></span></span></p> <p style="text-align:left"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}"> </font></font></span></span></span></span></span></span></span></span></span></span></p> <h2 style="text-align:left"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}">Measurements and results:</font></font></span></span></span></span></span></span></span></span></h2> <p style="text-align:left"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial">The following figures represent the same data in different ways to highlight different performance and scalability characteristics.</span></span></span></span></span></span></span></span></span></span></p> <p style="text-align:left"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}">Data with the graphs <a href="https://github.com/LucaCanali/Miscellaneous/tree/master/Performance_Testing/Oracle_load_testing_with_SLOB">on </a></font></font><a href="https://github.com/LucaCanali/Miscellaneous/tree/master/Performance_Testing/Oracle_load_testing_with_SLOB"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}">Notebooks </font></font></span></span></span></span></span></span></span></span></span></span></a><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}"><a href="https://github.com/LucaCanali/Miscellaneous/tree/master/Performance_Testing/Oracle_load_testing_with_SLOB">at this link</a> </font></font></span></span></span></span></span></span></span></span></span></span></p> <p style="text-align:left"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}"> </font></font></span></span></span></span></span></span></span></span></span></span></p> <p style="text-align:left"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"></span></span></span></span></span></span></span></span></span></span></p> <div class="separator" data-original-attrs="{&quot;style&quot;:&quot;&quot;}" style="text-align:center"> <p><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><span style="clear:both"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}"><a data-entity-type="" data-entity-uuid="" href="https://blogger.googleusercontent.com/img/b/R29vZ2xl/AVvXsEi6ptVTSCKWyuPmo2nzS9giqER-OoV_SFNENE8cYamXAh2JUDUPxUGYMqtI3ecY2CIleJc6GJZ7k042sdnBWLfQ-6vqSnV2OkmhLqKSqNnJ2ZH02BDYZEv9s9IzHSwhEaegAZf0exLC__n7cBMUUnz-lomdvnlYZJjeIT2CBLr7OH6ZUr51HlRzuTk2/s1010/Figure5.png"><img alt="" data-entity-type="" data-entity-uuid="" data-original-height="468" data-original-width="1010" src="https://blogger.googleusercontent.com/img/b/R29vZ2xl/AVvXsEi6ptVTSCKWyuPmo2nzS9giqER-OoV_SFNENE8cYamXAh2JUDUPxUGYMqtI3ecY2CIleJc6GJZ7k042sdnBWLfQ-6vqSnV2OkmhLqKSqNnJ2ZH02BDYZEv9s9IzHSwhEaegAZf0exLC__n7cBMUUnz-lomdvnlYZJjeIT2CBLr7OH6ZUr51HlRzuTk2/s1010/Figure5.png" style="cursor:move; width:1024px; height:475px" /></a></font></font></span></span></span></span></span></span></span></span></span></span></span></p> </div> <p><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}"><font style="color:#000000; font-family:&quot;Times New Roman&quot;; font-size:medium; font-style:normal; font-variant-ligatures:normal; font-weight:400; text-align:left; white-space:normal; text-decoration-thickness:initial; text-decoration-style:initial; text-decoration-color:initial"><b>Figure 5 – Raw Data and Capacity</b></font></font></font></p> <p style="text-align:left"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"></span></span></span></span></span></span></span></span></span></span></p> <p data-original-attrs="{&quot;style&quot;:&quot;mso-add-space: auto; mso-list: l1 level1 lfo1;&quot;}" style="text-align:left; margin-left:27px; text-indent:-18pt"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US; mso-ascii-font-family: Calibri; mso-bidi-font-family: Calibri; mso-fareast-font-family: Calibri; mso-hansi-font-family: Calibri;&quot;}"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-list: Ignore;&quot;}">-<font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;&quot;}"><font style="font:7pt &quot;Times New Roman&quot;">          </font></font></font></font></font></font></font><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}">The figure shows how the cumulative Oracle logical IO throughput increases with the number of parallel workers for the three servers tested.</font></font></span></span></span></span></span></span></span></span></span></span></p> <p data-original-attrs="{&quot;style&quot;:&quot;mso-add-space: auto; mso-list: l1 level1 lfo1;&quot;}" style="text-align:left; margin-left:27px; text-indent:-18pt"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US; mso-ascii-font-family: Calibri; mso-bidi-font-family: Calibri; mso-fareast-font-family: Calibri; mso-hansi-font-family: Calibri;&quot;}"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-list: Ignore;&quot;}">-<font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;&quot;}"><font style="font:7pt &quot;Times New Roman&quot;">          </font></font></font></font></font></font></font><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}">The common trend is that the Logical IO throughput increases with load up to the number of logical CPUs (16 for RAC54, 32 for RAC55, and 40 for RAC52).</font></font></span></span></span></span></span></span></span></span></span></span></p> <p data-original-attrs="{&quot;style&quot;:&quot;mso-add-space: auto; mso-list: l1 level1 lfo1;&quot;}" style="text-align:left; margin-left:27px; text-indent:-18pt"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US; mso-ascii-font-family: Calibri; mso-bidi-font-family: Calibri; mso-fareast-font-family: Calibri; mso-hansi-font-family: Calibri;&quot;}"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-list: Ignore;&quot;}">-<font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;&quot;}"><font style="font:7pt &quot;Times New Roman&quot;">          </font></font></font></font></font></font></font><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}">Measurements are “noisy” so we should take about 10% as the error margin on the collected data points.</font></font></span></span></span></span></span></span></span></span></span></span></p> <p data-original-attrs="{&quot;style&quot;:&quot;mso-add-space: auto; mso-list: l1 level1 lfo1;&quot;}" style="text-align:left; margin-left:27px; text-indent:-18pt"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US; mso-ascii-font-family: Calibri; mso-bidi-font-family: Calibri; mso-fareast-font-family: Calibri; mso-hansi-font-family: Calibri;&quot;}"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-list: Ignore;&quot;}">-<font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;&quot;}"><font style="font:7pt &quot;Times New Roman&quot;">          </font></font></font></font></font></font></font><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}">There are differences in performance and total throughput with RAC55 being the most performance and with the highest throughput.</font></font></span></span></span></span></span></span></span></span></span></span></p> <p data-original-attrs="{&quot;style&quot;:&quot;mso-add-space: auto; mso-list: l1 level1 lfo1;&quot;}" style="text-align:left; margin-left:27px; text-indent:-18pt"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US; mso-ascii-font-family: Calibri; mso-bidi-font-family: Calibri; mso-fareast-font-family: Calibri; mso-hansi-font-family: Calibri;&quot;}"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-list: Ignore;&quot;}">-<font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;&quot;}"><font style="font:7pt &quot;Times New Roman&quot;">          </font></font></font></font></font></font></font><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}">At low load, (&lt;= 8 concurrent processes) RAC54 and RAC55 have similar performance.</font></font></span></span></span></span></span></span></span></span></span></span></p> <p data-original-attrs="{&quot;style&quot;:&quot;mso-add-space: auto; mso-list: l1 level1 lfo1;&quot;}" style="text-align:left; margin-left:27px; text-indent:-18pt"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US; mso-ascii-font-family: Calibri; mso-bidi-font-family: Calibri; mso-fareast-font-family: Calibri; mso-hansi-font-family: Calibri;&quot;}"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-list: Ignore;&quot;}">-<font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;&quot;}"><font style="font:7pt &quot;Times New Roman&quot;">          </font></font></font></font></font></font></font><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}">At high load, RAC55 has about 20% more capacity/throughput of Logical IO than RAC52.</font></font></span></span></span></span></span></span></span></span></span></span></p> <p data-original-attrs="{&quot;style&quot;:&quot;mso-add-space: auto; mso-list: l1 level1 lfo1;&quot;}" style="text-align:left; margin-left:27px; text-indent:-18pt"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}">  </font></font></p> <p style="text-align:left"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-no-proof: yes;&quot;,&quot;lang&quot;:&quot;EN-GB&quot;}"><span alt="Chart, scatter chart&lt;/p&gt;&#10;&lt;p&gt;Description automatically generated" data-original-attrs="{&quot;o:spid&quot;:&quot;_x0000_i1027&quot;,&quot;style&quot;:&quot;mso-wrap-style: square;&quot;,&quot;type&quot;:&quot;#_x0000_t75&quot;}" data-original-tag="V:SHAPE" id="Picture_x0020_5" style="height:336.75pt; width:727.5pt"></span></font></font></span></span></span></span></span></span></span></span></span></span></p> <p style="text-align:left"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}"> </font></font></span></span></span></span></span></span></span></span></span></span></p> <p style="text-align:left"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"></span></span></span></span></span></span></span></span></span></span></p> <div class="separator" data-original-attrs="{&quot;style&quot;:&quot;&quot;}" style="text-align:center"> <p><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><span style="clear:both"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}"><a data-entity-type="" data-entity-uuid="" href="https://blogger.googleusercontent.com/img/b/R29vZ2xl/AVvXsEiCeS_3gwT85ILkMXJ2o2ZNWdoayDiCFMjwqRzrGyQRoPW7K9Cb9_H4arDo987D3MQmSjjhAf9NCf3L5JiKs6EOGeqtPf0UwoQSqGa6vi-qE64SzzFY10dXpzZdRPjjbVX2Eak8TYeGIudNR8rNukpDIOV2NRylTGRQKzlOWJesSe5NEYbysm9ttao4/s1001/Figure6.png"><img alt="" data-entity-type="" data-entity-uuid="" data-original-height="468" data-original-width="1001" src="https://blogger.googleusercontent.com/img/b/R29vZ2xl/AVvXsEiCeS_3gwT85ILkMXJ2o2ZNWdoayDiCFMjwqRzrGyQRoPW7K9Cb9_H4arDo987D3MQmSjjhAf9NCf3L5JiKs6EOGeqtPf0UwoQSqGa6vi-qE64SzzFY10dXpzZdRPjjbVX2Eak8TYeGIudNR8rNukpDIOV2NRylTGRQKzlOWJesSe5NEYbysm9ttao4/s1001/Figure6.png" style="cursor:move; width:1024px; height:479px" /></a></font></font></span></span></span></span></span></span></span></span></span></span></span></p> </div> <p><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}"><font style="color:#000000; font-family:&quot;Times New Roman&quot;; font-size:medium; font-style:normal; font-variant-ligatures:normal; font-weight:400; text-align:left; white-space:normal; text-decoration-thickness:initial; text-decoration-style:initial; text-decoration-color:initial"><b>Figure 6 - Speed</b></font></font></font></p> <p style="text-align:left"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"></span></span></span></span></span></span></span></span></span></span></p> <p data-original-attrs="{&quot;style&quot;:&quot;mso-add-space: auto; mso-list: l1 level1 lfo1;&quot;}" style="text-align:left; margin-left:27px; text-indent:-18pt"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US; mso-ascii-font-family: Calibri; mso-bidi-font-family: Calibri; mso-fareast-font-family: Calibri; mso-hansi-font-family: Calibri;&quot;}"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-list: Ignore;&quot;}">-<font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;&quot;}"><font style="font:7pt &quot;Times New Roman&quot;">          </font></font></font></font></font></font></font><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}">The figure shows Oracle logical IO throughput per worker as function of load.</font></font></span></span></span></span></span></span></span></span></span></span></p> <p data-original-attrs="{&quot;style&quot;:&quot;mso-add-space: auto; mso-list: l1 level1 lfo1;&quot;}" style="text-align:left; margin-left:27px; text-indent:-18pt"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US; mso-ascii-font-family: Calibri; mso-bidi-font-family: Calibri; mso-fareast-font-family: Calibri; mso-hansi-font-family: Calibri;&quot;}"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-list: Ignore;&quot;}">-<font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;&quot;}"><font style="font:7pt &quot;Times New Roman&quot;">          </font></font></font></font></font></font></font><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}">The performance of logical IOs decays with increasing load.</font></font></span></span></span></span></span></span></span></span></span></span></p> <p data-original-attrs="{&quot;style&quot;:&quot;mso-add-space: auto; mso-list: l1 level1 lfo1;&quot;}" style="text-align:left; margin-left:27px; text-indent:-18pt"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US; mso-ascii-font-family: Calibri; mso-bidi-font-family: Calibri; mso-fareast-font-family: Calibri; mso-hansi-font-family: Calibri;&quot;}"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-list: Ignore;&quot;}">-<font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;&quot;}"><font style="font:7pt &quot;Times New Roman&quot;">          </font></font></font></font></font></font></font><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}">Logical IO performance appears close to constant up to the number of physical cores of the server (8 for RAC54, 16 for RAC55 and 20 for RAC52) and then decays for higher load, saturating when the number of logical cores is reached.</font></font></span></span></span></span></span></span></span></span></span></span></p> <p data-original-attrs="{&quot;style&quot;:&quot;mso-add-space: auto; mso-list: l1 level1 lfo1;&quot;}" style="text-align:left; margin-left:27px; text-indent:-18pt"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US; mso-ascii-font-family: Calibri; mso-bidi-font-family: Calibri; mso-fareast-font-family: Calibri; mso-hansi-font-family: Calibri;&quot;}"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-list: Ignore;&quot;}">-<font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;&quot;}"><font style="font:7pt &quot;Times New Roman&quot;">          </font></font></font></font></font></font></font><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}">Measurements are “noisy” so we should take about 10% as the error margin on the collected data points.</font></font></span></span></span></span></span></span></span></span></span></span></p> <p data-original-attrs="{&quot;style&quot;:&quot;mso-add-space: auto; mso-list: l1 level1 lfo1;&quot;}" style="text-align:left; margin-left:27px; text-indent:-18pt"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US; mso-ascii-font-family: Calibri; mso-bidi-font-family: Calibri; mso-fareast-font-family: Calibri; mso-hansi-font-family: Calibri;&quot;}"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-list: Ignore;&quot;}">-<font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;&quot;}"><font style="font:7pt &quot;Times New Roman&quot;">          </font></font></font></font></font></font></font><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}">RAC55 shows the highest performance overall for Logical I/O throughput.</font></font></span></span></span></span></span></span></span></span></span></span></p> <p data-original-attrs="{&quot;style&quot;:&quot;mso-add-space: auto; mso-list: l1 level1 lfo1;&quot;}" style="text-align:left; margin-left:27px; text-indent:-18pt"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US; mso-ascii-font-family: Calibri; mso-bidi-font-family: Calibri; mso-fareast-font-family: Calibri; mso-hansi-font-family: Calibri;&quot;}"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-list: Ignore;&quot;}">-<font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;&quot;}"><font style="font:7pt &quot;Times New Roman&quot;">          </font></font></font></font></font></font></font><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}">At load below 16 concurrent workers, RAC55 appears 1.5x faster than RAC52, the gap closes to about 20% at high load.</font></font></span></span></span></span></span></span></span></span></span></span></p> <p data-original-attrs="{&quot;style&quot;:&quot;mso-add-space: auto; mso-list: l1 level1 lfo1;&quot;}" style="text-align:left; margin-left:27px; text-indent:-18pt"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}">  </font></font></p> <p data-original-attrs="{&quot;style&quot;:&quot;mso-add-space: auto; mso-list: l1 level1 lfo1;&quot;}" style="text-align:left; margin-left:27px; text-indent:-18pt"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"></span></span></span></span></span></span></span></span></span></span></p> <p data-original-attrs="{&quot;style&quot;:&quot;mso-add-space: auto; mso-list: l1 level1 lfo1;&quot;}" style="text-align:left; margin-left:27px; text-indent:-18pt"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"></span></span></span></span></span></span></span></span></span></span></p> <div class="separator" data-original-attrs="{&quot;style&quot;:&quot;&quot;}" style="text-align:center"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><span style="clear:both"><a data-entity-type="" data-entity-uuid="" href="https://blogger.googleusercontent.com/img/b/R29vZ2xl/AVvXsEji01FKqzD9EjrP7qom11YMQTVtwdc7tkwXhZTYfZt3POlUIF3ZWMt6FjOfpnUI6jlSaLaQY8JBcYuwmq1NJGrNFNpeVAa7kz2Mh9BymJ6cEm_uV6KbV6y2OBuclh_skuGdSCHaIJIAwVeDbFqIWebzEDtYA3AACTw3rf2zyAmU3oA476KQdHfoZ-gL/s1010/Figure7.png"><img alt="" data-entity-type="" data-entity-uuid="" data-original-height="468" data-original-width="1010" src="https://blogger.googleusercontent.com/img/b/R29vZ2xl/AVvXsEji01FKqzD9EjrP7qom11YMQTVtwdc7tkwXhZTYfZt3POlUIF3ZWMt6FjOfpnUI6jlSaLaQY8JBcYuwmq1NJGrNFNpeVAa7kz2Mh9BymJ6cEm_uV6KbV6y2OBuclh_skuGdSCHaIJIAwVeDbFqIWebzEDtYA3AACTw3rf2zyAmU3oA476KQdHfoZ-gL/s1010/Figure7.png" style="cursor:move; width:1024px; height:475px" /></a></span></span></span></span></span></span></span></span></span></span></span></div> <p data-original-attrs="{&quot;style&quot;:&quot;mso-add-space: auto; mso-list: l1 level1 lfo1;&quot;}" style="text-align:left; margin-left:27px; text-indent:-18pt"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><b>Figure 7 - Scalability</b></span></span></span></span></span></span></span></span></span></span></p> <p data-original-attrs="{&quot;style&quot;:&quot;mso-add-space: auto; mso-list: l1 level1 lfo1;&quot;}" style="text-align:left; margin-left:27px; text-indent:-18pt"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US; mso-ascii-font-family: Calibri; mso-bidi-font-family: Calibri; mso-fareast-font-family: Calibri; mso-hansi-font-family: Calibri;&quot;}"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-list: Ignore;&quot;}">-<font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;&quot;}"><font style="font:7pt &quot;Times New Roman&quot;">          </font></font></font></font></font></font></font><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}">The figure shows speedup as function of load.</font></font></span></span></span></span></span></span></span></span></span></span></p> <p data-original-attrs="{&quot;style&quot;:&quot;mso-add-space: auto; mso-list: l1 level1 lfo1;&quot;}" style="text-align:left; margin-left:27px; text-indent:-18pt"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US; mso-ascii-font-family: Calibri; mso-bidi-font-family: Calibri; mso-fareast-font-family: Calibri; mso-hansi-font-family: Calibri;&quot;}"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-list: Ignore;&quot;}">-<font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;&quot;}"><font style="font:7pt &quot;Times New Roman&quot;">          </font></font></font></font></font></font></font><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}">The figure shows the speedup, a measure of scalability, for this plot it’s calculated as the ratio of (cumulative Logical I/O at load n) / (cumulative Logical I/O at load 1)</font></font></span></span></span></span></span></span></span></span></span></span></p> <p data-original-attrs="{&quot;style&quot;:&quot;mso-add-space: auto; mso-list: l1 level2 lfo1;&quot;}" style="text-align:left; margin-left:75px; text-indent:-18pt"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US; mso-fareast-font-family: \&quot;Courier New\&quot;;&quot;}"><font face="&quot;Courier New&quot;"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-list: Ignore;&quot;}">o<font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;&quot;}"><font style="font:7pt &quot;Times New Roman&quot;">   </font></font></font></font></font></font></font></font><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}">Linear scalability would be represented by a line with speedup = number of parallel workers.</font></font></span></span></span></span></span></span></span></span></span></span></p> <p data-original-attrs="{&quot;style&quot;:&quot;mso-add-space: auto; mso-list: l1 level1 lfo1;&quot;}" style="text-align:left; margin-left:27px; text-indent:-18pt"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US; mso-ascii-font-family: Calibri; mso-bidi-font-family: Calibri; mso-fareast-font-family: Calibri; mso-hansi-font-family: Calibri;&quot;}"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-list: Ignore;&quot;}">-<font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;&quot;}"><font style="font:7pt &quot;Times New Roman&quot;">          </font></font></font></font></font></font></font><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}">A general trend observed in the data is that the scalability curves start close to the ideal linear scalability and then bend downwards due to contention.</font></font></span></span></span></span></span></span></span></span></span></span></p> <p data-original-attrs="{&quot;style&quot;:&quot;mso-add-space: auto; mso-list: l1 level1 lfo1;&quot;}" style="text-align:left; margin-left:27px; text-indent:-18pt"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US; mso-ascii-font-family: Calibri; mso-bidi-font-family: Calibri; mso-fareast-font-family: Calibri; mso-hansi-font-family: Calibri;&quot;}"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-list: Ignore;&quot;}">-<font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;&quot;}"><font style="font:7pt &quot;Times New Roman&quot;">          </font></font></font></font></font></font></font><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}">RAC55 has the better scalability behavior of the three servers tested. At low load (less than 8 concurrent workers) RAC55 and RAC54 have similar behavior.</font></font></span></span></span></span></span></span></span></span></span></span></p> <p data-original-attrs="{&quot;style&quot;:&quot;mso-add-space: auto; mso-list: l1 level1 lfo1;&quot;}" style="text-align:left; margin-left:27px; text-indent:-18pt"> </p> <p data-original-attrs="{&quot;style&quot;:&quot;mso-add-space: auto; mso-list: l1 level1 lfo1;&quot;}" style="text-align:left; margin-left:27px; text-indent:-18pt"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"></span></span></span></span></span></span></span></span></span></span></p> <p style="text-align:left"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-no-proof: yes;&quot;,&quot;lang&quot;:&quot;EN-GB&quot;}"><span alt="Chart, scatter chart&lt;/p&gt;&#10;&lt;p&gt;Description automatically generated" data-original-attrs="{&quot;o:spid&quot;:&quot;_x0000_i1025&quot;,&quot;style&quot;:&quot;mso-wrap-style: square;&quot;,&quot;type&quot;:&quot;#_x0000_t75&quot;}" data-original-tag="V:SHAPE" id="Picture_x0020_7" style="height:336.75pt; width:727.5pt"></span></font></font></span></span></span></span></span></span></span></span></span></span></p> <h1 style="text-align:left"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}">Conclusions</font></font></span></span></span></span></span></span></span></span></h1> <p style="text-align:left"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"></span></span></span></span></span></span></span></span></span></span></p> <p style="text-align:left"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}">This work collects a few tests and measurements on stress testing and CPU loading on three different platforms of interest for the CERN Oracle database service.</font></font></span></span></span></span></span></span></span></span></span></span></p> <p style="text-align:left"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}">The tests performed are narrow in scope, just addressing the CPU load.</font></font></span></span></span></span></span></span></span></span></span></span></p> <p style="text-align:left"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}">Two different testing tools have been used for these tests: testing with a simple CPU-burning script loop run in parallel, and testing with an Oracle-specific workload generator for Logical I/O. </font></font></span></span></span></span></span></span></span></span></span></span></p> <p style="text-align:left"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial">The tools used, as well as the measured data and their analyses, are <a data-original-attrs="{&quot;data-original-href&quot;:&quot;https://github.com/LucaCanali/Miscellaneous/tree/master/Performance_Testing&quot;}" href="https://github.com/LucaCanali/Miscellaneous/tree/master/Performance_Testing">available at this link</a>.</span></span></span></span></span></span></span></span></span></span></p> <p style="text-align:left"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;mso-ansi-language: EN-US;&quot;}">We find that the newest server model (RAC55) has the highest CPU per-core performance, scalability, and overall CPU throughput.</font></font></span></span></span></span></span></span></span></span></span></span></p> <p style="text-align:left"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial">This work has been done in the context of the CERN databases and analytics services and the ATLAS data engineering efforts, many thanks to my colleagues for their help and suggestions.</span></span></span></span></span></span></span></span></span></span></p> <p style="text-align:left"> </p> </div> </div> <span><a title="View user profile." href="/users/luca-canali" lang="" about="/users/luca-canali" typeof="schema:Person" property="schema:name" datatype="">canali</a></span> <span><time datetime="2023-05-04T14:38:32+02:00" title="Thursday, May 4, 2023 - 14:38">Thu, 05/04/2023 - 14:38</time> </span> <div class="field field--name-field-tags field--type-entity-reference field--label-above"> <div class="field--label"><b>Tags</b></div> <div class="field--items"> <div class="field--item"><a href="/tags/performance-0" hreflang="en">Performance</a></div> <div class="field--item"><a href="/tags/testing" hreflang="en">Testing</a></div> <div class="field--item"><a href="/tags/oracle" hreflang="en">Oracle</a></div> </div> </div> <section> <h2>Add new comment</h2> <drupal-render-placeholder callback="comment.lazy_builders:renderForm" arguments="0=node&amp;1=189&amp;2=comment_node_blog_post&amp;3=comment_node_blog_post" token="MJVyH7jd18_k-jUlqK6ezLB2z0jFi5KTeNz1Un4SVFY"></drupal-render-placeholder> </section> Thu, 04 May 2023 12:38:32 +0000 canali 189 at https://db-blog.web.cern.ch Introduction to Apache Spark APIs for Data Processing https://db-blog.web.cern.ch/node/188 <span>Introduction to Apache Spark APIs for Data Processing</span> <div class="field field--name-body field--type-text-with-summary field--label-above"> <div class="field--label"><b>Blog article:</b></div> <div class="field--item"><div class="WordSection1"> <div style="border-bottom:solid windowtext 1.0pt; padding:0cm 0cm 1.0pt 0cm"> <h1 style="border:none; padding:0cm"><img alt="Text Box: " data-entity-type="" data-entity-uuid="" src="https://sparktraining.web.cern.ch/index_files/image006.png" style="width:1003px; height:92px" /><span style="color:red"><br /> Introduction to Apache Spark APIs for Data Processing</span></h1> <p style="border: medium none; padding: 0cm;"><span style="font-size:16.0pt"><span style="font-weight:normal">Welcome to the website of the course on <a href="https://spark.apache.org/">Apache Spark</a> by CERN IT. The course is self-paced and open, it is a short introduction to the architecture and key abstractions used by Spark. Theory and demos cover the main <a href="https://spark.apache.org/docs/latest/index.html">Spark APIs</a>: DataFrame API, Spark SQL, Streaming, Machine Learning. You will also learn how to deploy Spark on CERN computing resources, notably using the <a href="https://swan.web.cern.ch/swan/">CERN SWAN service</a>. Most tutorials and exercises are in Python and run on Jupyter notebooks.</span></span></p> <p style="border: medium none; padding: 0cm;"><span style="font-size:16.0pt"><span style="font-weight:normal">Apache Spark is a popular engine for data processing at scale. Spark provides an expressive API and a scalable engine that integrates very well with the Hadoop ecosystem as well as with Cloud resources. Spark is currently used by several projects at CERN, notably by IT monitoring, by the security team, by the BE NXCALS project, by teams in ATLAS and CMS. Moreover, Spark is integrated with the CERN Hadoop service, the CERN Cloud service, and the CERN SWAN web notebooks service.</span></span></p> </div> <h1> </h1> <h1><span style="color:red">Accompanying notebooks</span></h1> <ul><li><span style="font-size:16.0pt"></span><span style="font-size:16.0pt">Get the notebooks from:</span><br /><ul style="list-style-type:circle"><li style="margin-left:16px"><span style="font-size:16.0pt"></span><span style="font-size:16.0pt"><a href="https://github.com/cerndb/SparkTraining">https://github.com/cerndb/SparkTraining</a></span></li> <li style="margin-left:16px"><span style="font-size:16.0pt"></span><span style="font-size:16.0pt"><a href="https://gitlab.cern.ch/db/SparkTraining">https://gitlab.cern.ch/db/SparkTraining</a></span><br />  </li> </ul></li> <li><span style="font-size:16.0pt"></span><span style="font-size:16.0pt">How to run the notebooks:</span><br /><ul style="list-style-type:circle"><li style="margin-left:16px"><span style="font-size:16.0pt"></span><span style="font-size:16.0pt">CERN SWAN (recommended option): </span><a href="https://cern.ch/swanserver/cgi-bin/go?projurl=https://github.com/cerndb/SparkTraining.git"><span style="text-decoration:none"><span style="text-underline:none"><img alt="" data-entity-type="" data-entity-uuid="" src="https://sparktraining.web.cern.ch/index_files/image001.png" style="width:112px; height:20px" /></span></span></a><span style="font-size:16.0pt"></span></li> <li style="margin-left:16px"><span style="font-size:16.0pt"></span><span style="font-size:16.0pt">Colab</span> <a href="https://colab.research.google.com/github/cerndb/SparkTraining"><span style="text-decoration:none"><span style="text-underline:none"><img alt="" data-entity-type="" data-entity-uuid="" src="https://sparktraining.web.cern.ch/index_files/image003.png" style="width:117px; height:20px" /></span></span></a><span style="font-size:16.0pt">, Binder </span><a href="https://mybinder.org/v2/gh/cerndb/SparkTraining/master"><span style="text-decoration:none"><span style="text-underline:none"><img alt="" data-entity-type="" data-entity-uuid="" src="https://sparktraining.web.cern.ch/index_files/image002.png" style="width:109px; height:20px" /></span></span></a><span style="font-size:16.0pt"></span></li> <li style="margin-left:16px"><span style="font-size:16.0pt"></span><span style="font-size:16.0pt">Local/private Jupyter notebook</span></li> <li style="margin-left:16px"><span style="font-size:16.0pt"></span><span style="font-size:16.0pt">See also the <a href="https://swan-gallery.web.cern.ch/apache_spark/">SWAN gallery</a> and the <a href="https://sparktraining.web.cern.ch/videos/Spark_CERN_Use_SWAN_notebooks.mp4">video</a>:</span> <p> <a href="https://sparktraining.web.cern.ch/videos/Spark_CERN_Use_SWAN_notebooks.mp4"><span style="text-decoration:none"><span style="text-underline:none"><img alt="" data-entity-type="" data-entity-uuid="" src="https://sparktraining.web.cern.ch/index_files/image008.png" style="width:228px; height:150px" /></span></span></a><span style="font-size:16.0pt"></span></p></li> </ul></li> </ul><p><span style="font-size:16.0pt"></span></p> <h1><span style="color:red"><br /> Course lectures and tutorials</span></h1> <ul><li><span style="font-size:16.0pt"></span><b><span style="font-size:16.0pt">Introduction and objectives:</span></b><span style="font-size:16.0pt"> <a href="https://sparktraining.web.cern.ch/slides/Introduction_to_Spark_Welcome.pdf">slides</a> and <a href="https://sparktraining.web.cern.ch/videos/Spark_CERN_Introduction.mp4">video</a> <p>           </p></span><a href="https://sparktraining.web.cern.ch/videos/Spark_CERN_Introduction.mp4"><span style="font-size:16.0pt"><span style="text-decoration:none"><span style="text-underline:none"><img alt="" data-entity-type="" data-entity-uuid="" src="https://sparktraining.web.cern.ch/index_files/image009.png" style="width:243px; height:137px" /></span></span></span></a><span style="font-size:16.0pt"></span><br />  </li> <li><span style="font-size:16.0pt"></span><b><span style="font-size:16.0pt">Session 1:</span></b><span style="font-size:16.0pt"> Apache Spark fundamentals</span><br />   <ul style="list-style-type:circle"><li style="margin-left:16px"><span style="font-size:16.0pt"></span><span style="font-size:16.0pt">Lecture “<b><span style="color:red">Spark architecture and intro to DataFrames</span></b>”: <a href="https://sparktraining.web.cern.ch/slides/Introduction_to_Spark_Session1.pdf">slides</a> and <a href="https://sparktraining.web.cern.ch/videos/Spark_CERN_Session1_Architecture_and_DataFrames.mp4">video</a></span> <p> <a href="https://sparktraining.web.cern.ch/videos/Spark_CERN_Session1_Architecture_and_DataFrames.mp4"><span style="text-decoration:none"><span style="text-underline:none"><img alt="Graphical user interface, diagram&lt;/p&gt;&#10;&lt;p&gt;Description automatically generated" data-entity-type="" data-entity-uuid="" src="https://sparktraining.web.cern.ch/index_files/image018.png" style="width:246px; height:139px" /></span></span></a><span style="font-size:16.0pt"></span><br />  </p></li> <li style="margin-left:16px"><span style="font-size:16.0pt"></span><span style="font-size:16.0pt">Notebooks: </span><br /><ul style="list-style-type:circle"><li style="margin-left:8px"><span style="font-size:16.0pt"></span><span style="font-size:16.0pt"><a href="https://github.com/cerndb/SparkTraining/blob/master/notebooks/Tutorial-DataFrame.ipynb">Tutorial on DataFrames with exercises</a> – <a href="https://sparktraining.web.cern.ch/videos/Spark_CERN_Tutorial_DataFrame.mp4">video</a> </span><a href="https://sparktraining.web.cern.ch/videos/Spark_CERN_Tutorial_DataFrame.mp4"><span style="text-decoration:none"><span style="text-underline:none"><img alt="Icon&#10;&lt;p&gt;Description automatically generated" data-entity-type="" data-entity-uuid="" src="https://sparktraining.web.cern.ch/index_files/image021.jpg" style="width:26px; height:26px" /></span></span></a><span style="font-size:16.0pt"></span></li> <li style="margin-left:8px"><span style="font-size:16.0pt"></span><span style="font-size:16.0pt"><a href="https://github.com/cerndb/SparkTraining/blob/master/notebooks/Solutions-DataFrame.ipynb">Solutions to the exercises</a></span> <p>  </p></li> </ul></li> </ul></li> <li><span style="font-size:16.0pt"></span><b><span style="font-size:16.0pt">Session 2:</span></b><span style="font-size:16.0pt"> Working with Spark DataFrames and SQL</span><br />   <ul style="list-style-type:circle"><li style="margin-left:16px"><span style="font-size:16.0pt"></span><span style="font-size:16.0pt">Lecture “<b><span style="color:red">Introduction to Spark SQL</span></b>”: <a href="https://sparktraining.web.cern.ch/slides/Introduction_to_Spark_Session2.pdf">slides</a> and <a href="https://sparktraining.web.cern.ch/videos/Spark_CERN_Session2_Spark_SQL.mp4">video</a></span> <p> <a href="https://sparktraining.web.cern.ch/videos/Spark_CERN_Session2_Spark_SQL.mp4"><span style="font-size:16.0pt"><span style="text-decoration:none"><span style="text-underline:none"><img alt="" data-entity-type="" data-entity-uuid="" src="https://sparktraining.web.cern.ch/index_files/image023.png" style="width:251px; height:142px" /></span></span></span></a><span style="font-size:16.0pt"></span><br />  </p></li> <li style="margin-left:16px"><span style="font-size:16.0pt"></span><span style="font-size:16.0pt">Notebooks:</span><br /><ul style="list-style-type:circle"><li style="margin-left:8px"><span style="font-size:16.0pt"></span><span style="font-size:16.0pt"><a href="https://github.com/cerndb/SparkTraining/blob/master/notebooks/Tutorial-SparkSQL.ipynb">Tutorial on Spark SQL</a> – <a href="https://sparktraining.web.cern.ch/videos/Spark_CERN_Tutorial_Spark_SQL.mp4">video</a> </span><a href="https://sparktraining.web.cern.ch/videos/Spark_CERN_Tutorial_Spark_SQL.mp4"><span style="text-decoration:none"><span style="text-underline:none"><img alt="Icon&#10;&lt;p&gt;Description automatically generated" data-entity-type="" data-entity-uuid="" src="https://sparktraining.web.cern.ch/index_files/image027.jpg" style="width:26px; height:26px" /></span></span></a><span style="font-size:16.0pt"></span></li> <li style="margin-left:8px"><span style="font-size:16.0pt"></span><span style="font-size:16.0pt"><a href="https://github.com/cerndb/SparkTraining/blob/master/notebooks/HandsOn-SparkSQL_exercises.ipynb">Exercises on Spark SQL</a></span></li> <li style="margin-left:8px"><span style="font-size:16.0pt"></span><span style="font-size:16.0pt"><a href="https://github.com/cerndb/SparkTraining/blob/master/notebooks/HandsOn-SparkSQL_with_solutions.ipynb">Solutions to the exercises</a></span> <p>  </p></li> </ul></li> </ul></li> <li><span style="font-size:16.0pt"></span><b><span style="font-size:16.0pt">Session 3:</span></b><span style="font-size:16.0pt"> Building on top of the DataFrame API</span><br />   <ul style="list-style-type:circle"><li style="margin-left:16px"><span style="font-size:16.0pt"></span><span style="font-size:16.0pt">Lecture “<b><span style="color:red">Spark as a Data Platform</span></b>”: <a href="https://sparktraining.web.cern.ch/slides/Introduction_to_Spark_Session3_part1.pdf">slides</a> and <a href="https://sparktraining.web.cern.ch/videos/Spark_CERN_Session3_part1_DataFormat.mp4">video</a></span> <p> <a href="https://sparktraining.web.cern.ch/videos/Spark_CERN_Session3_part1_DataFormat.mp4"><span style="font-size:16.0pt"><span style="text-decoration:none"><span style="text-underline:none"><img alt="" data-entity-type="" data-entity-uuid="" src="https://sparktraining.web.cern.ch/index_files/image028.png" style="width:262px; height:147px" /></span></span></span></a><span style="font-size:16.0pt"></span><br />  </p></li> <li style="margin-left:16px"><span style="font-size:16.0pt"></span><span style="font-size:16.0pt">Lecture “<b><span style="color:red">Spark Streaming</span></b>”: <a href="https://sparktraining.web.cern.ch/slides/Introduction_to_Spark_Session3_part2.pdf">slides</a> and <a href="https://sparktraining.web.cern.ch/videos/Spark_CERN_Session3_part2_Streaming.mp4">video</a></span> <p> <a href="https://sparktraining.web.cern.ch/videos/Spark_CERN_Session3_part2_Streaming.mp4"><span style="font-size:16.0pt"><span style="text-decoration:none"><span style="text-underline:none"><img alt="" data-entity-type="" data-entity-uuid="" src="https://sparktraining.web.cern.ch/index_files/image029.png" style="width:267px; height:150px" /></span></span></span></a><span style="font-size:16.0pt"></span><br />  </p></li> <li style="margin-left:16px"><span style="font-size:16.0pt"></span><span style="font-size:16.0pt">Lecture “<b><span style="color:red">Spark and Machine Learning</span></b>”: <a href="https://sparktraining.web.cern.ch/slides/Introduction_to_Spark_Session3_part3.pdf">slides</a> and <a href="https://sparktraining.web.cern.ch/videos/Spark_CERN_Session3_part3_ML.mp4">video</a></span> <p> <a href="https://sparktraining.web.cern.ch/videos/Spark_CERN_Session3_part3_ML.mp4"><span style="font-size:16.0pt"><span style="text-decoration:none"><span style="text-underline:none"><img alt="" data-entity-type="" data-entity-uuid="" src="https://sparktraining.web.cern.ch/index_files/image032.png" style="width:272px; height:162px" /></span></span></span></a><span style="font-size:16.0pt"></span><br />  </p></li> <li style="margin-left:16px"><span style="font-size:16.0pt"></span><span style="font-size:16.0pt">Notebooks:</span><br /><ul style="list-style-type:circle"><li style="margin-left:8px"><span style="font-size:16.0pt"></span><span style="font-size:16.0pt">Tutorial on <a href="https://github.com/cerndb/SparkTraining/blob/master/notebooks/Tutorial-SparkStreaming.ipynb">Spark Streaming</a> – <a href="https://sparktraining.web.cern.ch/videos/Spark_CERN_Tutorial_Streaming.mp4">video</a> </span><a href="https://sparktraining.web.cern.ch/videos/Spark_CERN_Tutorial_Streaming.mp4"><span style="text-decoration:none"><span style="text-underline:none"><img alt="Icon&#10;&lt;p&gt;Description automatically generated" data-entity-type="" data-entity-uuid="" src="https://sparktraining.web.cern.ch/index_files/image033.jpg" style="width:26px; height:26px" /></span></span></a><span style="font-size:16.0pt"></span></li> <li style="margin-left:8px"><span style="font-size:16.0pt"></span><span style="font-size:16.0pt">Tutorial on <a href="https://github.com/cerndb/SparkTraining/blob/master/notebooks/ML_Demo2_Regression.ipynb">Spark Machine Learning – regression task</a> – <a href="https://sparktraining.web.cern.ch/videos/Spark_CERN_Demo_ML_regression.mp4">video</a> </span><a href="https://sparktraining.web.cern.ch/videos/Spark_CERN_Demo_ML_regression.mp4"><span style="text-decoration:none"><span style="text-underline:none"><img alt="Icon&#10;&lt;p&gt;Description automatically generated" data-entity-type="" data-entity-uuid="" src="https://sparktraining.web.cern.ch/index_files/image035.jpg" style="width:26px; height:26px" /></span></span></a><span style="font-size:16.0pt"></span></li> <li style="margin-left:8px"><span style="font-size:16.0pt"></span><span style="font-size:16.0pt">Tutorial on <a href="https://github.com/cerndb/SparkTraining/blob/master/notebooks/ML_Demo1_Classifier.ipynb">Spark Machine Learning – classification task with the Higgs dataset</a> </span></li> <li style="margin-left:8px"><span style="font-size:16.0pt"></span><span style="font-size:16.0pt">Demo of the Spark JDBC data source <a href="https://github.com/cerndb/SparkTraining/blob/master/notebooks/Spark_JDBC_Oracle.ipynb">how to read Oracle tables from Spark</a></span></li> </ul></li> <li style="margin-left:16px"><span style="font-size:16.0pt"></span><span style="font-size:16.0pt">Note on <a href="https://github.com/LucaCanali/Miscellaneous/blob/master/Spark_Notes/Spark_Parquet.md">Spark and Parquet format</a></span> <p>  </p></li> </ul></li> <li><span style="font-size:16.0pt"></span><b><span style="font-size:16.0pt">Session 4:</span></b><span style="font-size:16.0pt"> How to scale out Spark jobs</span><br />   <ul style="list-style-type:circle"><li style="margin-left:16px"><span style="font-size:16.0pt"></span><span style="font-size:16.0pt">Lecture “<b><span style="color:red">Running Spark on CERN resources</span></b>”: <a href="https://sparktraining.web.cern.ch/slides/Introduction_to_Spark_Session4.pdf">slides</a> and <a href="https://sparktraining.web.cern.ch/videos/Spark_CERN_Session4_Spark_at_CERN.mp4">video</a></span> <p> <a href="https://sparktraining.web.cern.ch/videos/Spark_CERN_Session4_Spark_at_CERN.mp4"><span style="font-size:16.0pt"><span style="text-decoration:none"><span style="text-underline:none"><img alt="" data-entity-type="" data-entity-uuid="" src="https://sparktraining.web.cern.ch/index_files/image036.png" style="width:255px; height:143px" /></span></span></span></a><span style="font-size:16.0pt"></span><br />  </p></li> <li style="margin-left:16px"><span style="font-size:16.0pt"></span><span style="font-size:16.0pt">Notebooks:</span><br /><ul style="list-style-type:circle"><li style="margin-left:8px"><span style="font-size:16.0pt"></span><span style="font-size:16.0pt">Demo on using <a href="https://github.com/cerndb/SparkTraining/blob/master/notebooks/Demo_Spark_on_Hadoop.ipynb">SWAN with Spark on Hadoop</a> – <a href="https://sparktraining.web.cern.ch/videos/Spark_CERN_Demo_Spark_on_Hadoop.mp4">video</a> </span><a href="https://sparktraining.web.cern.ch/videos/Spark_CERN_Demo_Spark_on_Hadoop.mp4"><span style="text-decoration:none"><span style="text-underline:none"><img alt="Icon&#10;&lt;p&gt;Description automatically generated" data-entity-type="" data-entity-uuid="" src="https://sparktraining.web.cern.ch/index_files/image039.jpg" style="width:26px; height:26px" /></span></span></a><span style="font-size:16.0pt"></span></li> <li style="margin-left:8px"><span style="font-size:16.0pt"></span><span style="font-size:16.0pt">Demo of <a href="https://github.com/cerndb/SparkTraining/blob/master/notebooks/Demo_Dimuon_mass_spectrum.ipynb">Spark processing Physics data using CERN private Cloud resources</a> – <a href="https://sparktraining.web.cern.ch/videos/Spark_CERN_Demo_Spark_on_Cloud-Physics-use-case.mp4">video</a> </span><a href="https://sparktraining.web.cern.ch/videos/Spark_CERN_Demo_Spark_on_Cloud-Physics-use-case.mp4"><span style="text-decoration:none"><span style="text-underline:none"><img alt="Icon&#10;&lt;p&gt;Description automatically generated" data-entity-type="" data-entity-uuid="" src="https://sparktraining.web.cern.ch/index_files/image042.jpg" style="width:26px; height:26px" /></span></span></a><span style="font-size:16.0pt"></span></li> <li style="margin-left:8px"><span style="font-size:16.0pt"></span><span style="font-size:16.0pt">Example <a href="https://github.com/cerndb/SparkTraining/blob/master/notebooks/NXCals-example.ipynb">notebook for the NXCALS project</a></span></li> </ul></li> </ul></li> </ul><p><span style="font-size:16.0pt"></span></p> <ul><li><span style="font-size:16.0pt"></span><b><span style="font-size:16.0pt">Bonus material:</span></b><br /><ul style="list-style-type:circle"><li style="margin-left:16px"><span style="font-size:16.0pt"></span><b><span style="font-size:16.0pt"><span style="color:red">How to monitor Spark execution</span></span></b><span style="font-size:16.0pt">: <a href="https://sparktraining.web.cern.ch/slides/Introduction_to_Spark_Monitoring_Spark.pdf">slides</a> and <a href="https://sparktraining.web.cern.ch/videos/Spark_CERN_Demo_Spark_Monitoring.mp4">video</a> </span><a href="https://sparktraining.web.cern.ch/videos/Spark_CERN_Demo_Spark_Monitoring.mp4"><span style="text-decoration:none"><span style="text-underline:none"><img alt="Icon&#10;&lt;p&gt;Description automatically generated" data-entity-type="" data-entity-uuid="" src="https://sparktraining.web.cern.ch/index_files/image048.jpg" style="width:26px; height:26px" /></span></span></a><span style="font-size:16.0pt"></span></li> <li style="margin-left:16px"><span style="font-size:16.0pt"></span><b><span style="font-size:16.0pt"><span style="color:red">Spark as a library</span></span></b><span style="font-size:16.0pt">, examples of how to use Spark in Scala and Python programs: <a href="https://github.com/cerndb/SparkTraining/tree/master/code">code</a> and <a href="https://sparktraining.web.cern.ch/videos/Spark_CERN_Demo_Spark_as_a_library.mp4">video</a> </span><a href="https://sparktraining.web.cern.ch/videos/Spark_CERN_Demo_Spark_as_a_library.mp4"><span style="text-decoration:none"><span style="text-underline:none"><img alt="Icon&#10;&lt;p&gt;Description automatically generated" data-entity-type="" data-entity-uuid="" src="https://sparktraining.web.cern.ch/index_files/image048.jpg" style="width:26px; height:26px" /></span></span></a><span style="font-size:16.0pt"></span></li> <li style="margin-left:16px"><span style="font-size:16.0pt"></span><b><span style="font-size:16.0pt"><span style="color:red">Next steps</span></span></b><b><span style="font-size:16.0pt">:</span></b><span style="font-size:16.0pt"> <a href="https://sparktraining.web.cern.ch/slides/Introduction_to_Spark_References.pdf">reading material and links</a>, miscellaneous <a href="https://github.com/LucaCanali/Miscellaneous/tree/master/Spark_Notes">Spark notes</a></span></li> </ul></li> </ul><p><span style="font-size:16.0pt"></span></p> <ul><li><span style="font-size:16.0pt"></span><b><span style="font-size:16.0pt">Read and watch at your pace:</span></b><br /><ul style="list-style-type:circle"><li style="margin-left:16px"><span style="font-size:16.0pt"></span><span style="font-size:16.0pt">Download the course material for offline use:<br />  <a href="https://sparktraining.web.cern.ch/slides/slides.zip">slides.zip</a>, <a href="https://github.com/cerndb/SparkTraining/archive/refs/heads/master.zip">github_repo.zip</a>, <a href="https://sparktraining.web.cern.ch/videos/videos.zip">videos.zip</a></span></li> <li style="margin-left:16px"><span style="font-size:16.0pt"></span><span style="font-size:16.0pt">Watch the <a href="https://www.youtube.com/playlist?list=PLeFOahkdaVOaOB6fz00Lame0Tv70eOIvb">videos on YouTube<span style="font-size:12.0pt"><span style="text-decoration:none"><span style="text-underline:none"> <img alt="Logo, icon&#10;&lt;p&gt;Description automatically generated" data-entity-type="" data-entity-uuid="" src="https://sparktraining.web.cern.ch/index_files/image049.png" style="width:36px; height:36px" /></span></span></span></a></span></li> </ul></li> </ul><p><span style="font-size:16.0pt"></span></p> <div style="border-bottom:solid windowtext 1.0pt; padding:0cm 0cm 1.0pt 0cm">  <p style="border:none; padding:0cm"><span style="font-size:16.0pt"></span></p> </div> <h1>Acknowledgements and feedback</h1> <p><span style="font-size:16.0pt">Author and contact for feedback and questions: Luca Canali - <a href="mailto:Luca.Canali@cern.ch">Luca.Canali@cern.ch</a></span></p> <p><span style="font-size:16.0pt">CERN-IT Spark and data analytics services</span></p> <p><span style="font-size:16.0pt">Former contributors: Riccardo Castellotti, Prasanth Kothuri</span></p> <p><span style="font-size:16.0pt">Many thanks to CERN Technical Training for their collaboration and support</span></p> <p><span style="font-size:16.0pt"></span></p> <p><span style="font-size:16.0pt">License: CC BY-SA 4.0</span></p> <p><span style="font-size:16.0pt">Published in November 2022</span></p> <p><span style="font-size:14.0pt"></span></p> </div> </div> </div> <span><a title="View user profile." href="/users/luca-canali" lang="" about="/users/luca-canali" typeof="schema:Person" property="schema:name" datatype="">canali</a></span> <span><time datetime="2023-02-23T14:12:01+01:00" title="Thursday, February 23, 2023 - 14:12">Thu, 02/23/2023 - 14:12</time> </span> <div class="field field--name-field-tags field--type-entity-reference field--label-above"> <div class="field--label"><b>Tags</b></div> <div class="field--items"> <div class="field--item"><a href="/tags/apache-spark" hreflang="en">Apache Spark</a></div> <div class="field--item"><a href="/tags/jupyter-notebook" hreflang="en">Jupyter notebook</a></div> </div> </div> <section> <h2>Add new comment</h2> <drupal-render-placeholder callback="comment.lazy_builders:renderForm" arguments="0=node&amp;1=188&amp;2=comment_node_blog_post&amp;3=comment_node_blog_post" token="EXEXm-QEFiY8lOjNwkxKXjrAvdv36Wh4JjRbd7Bnr70"></drupal-render-placeholder> </section> Thu, 23 Feb 2023 13:12:01 +0000 canali 188 at https://db-blog.web.cern.ch https://db-blog.web.cern.ch/node/188#comments Making histograms with Apache Spark and other SQL engines https://db-blog.web.cern.ch/node/187 <span>Making histograms with Apache Spark and other SQL engines</span> <div class="field field--name-body field--type-text-with-summary field--label-above"> <div class="field--label"><b>Blog article:</b></div> <div class="field--item"><p style="color: rgb(0, 0, 0); font-family: &quot;Times New Roman&quot;; font-size: medium; font-style: normal; font-variant-ligatures: normal; font-variant-caps: normal; font-weight: 400; letter-spacing: normal; orphans: 2; text-align: left; text-indent: 0px; text-transform: none; white-space: normal; widows: 2; word-spacing: 0px; -webkit-text-stroke-width: 0px; text-decoration-thickness: initial; text-decoration-style: initial; text-decoration-color: initial;"><b>Topic:</b><span> </span><span style="color: rgb(0, 0, 0); font-family: &quot;Times New Roman&quot;; font-size: medium; font-style: normal; font-variant-ligatures: normal; font-variant-caps: normal; font-weight: 400; letter-spacing: normal; orphans: 2; text-align: left; text-indent: 0px; text-transform: none; white-space: normal; widows: 2; word-spacing: 0px; -webkit-text-stroke-width: 0px; text-decoration-thickness: initial; text-decoration-style: initial; text-decoration-color: initial; display: inline !important; float: none;">This post will show you how to generate histograms using Apache Spark. You will find examples using the Spark DataFrame API and with a custom helper package</span>, <a href="https://pypi.org/project/sparkhistogram/">SparkHistogram</a>. <span style="color: rgb(0, 0, 0); font-family: &quot;Times New Roman&quot;; font-size: medium; font-style: normal; font-variant-ligatures: normal; font-variant-caps: normal; font-weight: 400; letter-spacing: normal; orphans: 2; text-align: left; text-indent: 0px; text-transform: none; white-space: normal; widows: 2; word-spacing: 0px; -webkit-text-stroke-width: 0px; text-decoration-thickness: initial; text-decoration-style: initial; text-decoration-color: initial; display: inline !important; float: none;">Additional examples will extend the work to histogram generation for several other databases and SQL engines.</span> </p> <p style="color: rgb(0, 0, 0); font-family: &quot;Times New Roman&quot;; font-size: medium; font-style: normal; font-variant-ligatures: normal; font-variant-caps: normal; font-weight: 400; letter-spacing: normal; orphans: 2; text-align: left; text-indent: 0px; text-transform: none; white-space: normal; widows: 2; word-spacing: 0px; -webkit-text-stroke-width: 0px; text-decoration-thickness: initial; text-decoration-style: initial; text-decoration-color: initial;"><b>Disambiguation:</b><span> </span>we refer here to computing histograms for data analysis, rather than histograms of table columns or statistics used by cost-based optimizers.</p> <p style="color: rgb(0, 0, 0); font-family: &quot;Times New Roman&quot;; font-size: medium; font-style: normal; font-variant-ligatures: normal; font-variant-caps: normal; font-weight: 400; letter-spacing: normal; orphans: 2; text-align: left; text-indent: 0px; text-transform: none; white-space: normal; widows: 2; word-spacing: 0px; -webkit-text-stroke-width: 0px; text-decoration-thickness: initial; text-decoration-style: initial; text-decoration-color: initial;">  </p> <h3 data-original-attrs="{&quot;style&quot;:&quot;&quot;}" style="color: rgb(0, 0, 0); font-family: &quot;Times New Roman&quot;; font-style: normal; font-variant-ligatures: normal; font-variant-caps: normal; letter-spacing: normal; orphans: 2; text-align: left; text-indent: 0px; text-transform: none; white-space: normal; widows: 2; word-spacing: 0px; -webkit-text-stroke-width: 0px; text-decoration-thickness: initial; text-decoration-style: initial; text-decoration-color: initial;"><strong><font color="#ff0000" data-keep-original-tag="false" data-original-attrs="{&quot;style&quot;:&quot;&quot;}">Why histograms with Apache Spark?</font></strong></h3> <p style="color: rgb(0, 0, 0); font-family: &quot;Times New Roman&quot;; font-size: medium; font-style: normal; font-variant-ligatures: normal; font-variant-caps: normal; font-weight: 400; letter-spacing: normal; orphans: 2; text-align: left; text-indent: 0px; text-transform: none; white-space: normal; widows: 2; word-spacing: 0px; -webkit-text-stroke-width: 0px; text-decoration-thickness: initial; text-decoration-style: initial; text-decoration-color: initial;"><a data-original-attrs="{&quot;data-original-href&quot;:&quot;https://en.wikipedia.org/wiki/Histogram&quot;}" href="https://en.wikipedia.org/wiki/Histogram">Histograms </a>are common tools for data analysis and are a key element in most High Energy Physics analyses. See also the post <a data-original-attrs="{&quot;data-original-href&quot;:&quot;https://db-blog.web.cern.ch/node/186&quot;}" href="https://db-blog.web.cern.ch/node/186">Can High Energy Physics Analysis Profit from Apache Spark APIs?</a></p> <p style="color: rgb(0, 0, 0); font-family: &quot;Times New Roman&quot;; font-size: medium; font-style: normal; font-variant-ligatures: normal; font-variant-caps: normal; font-weight: 400; letter-spacing: normal; orphans: 2; text-align: left; text-indent: 0px; text-transform: none; white-space: normal; widows: 2; word-spacing: 0px; -webkit-text-stroke-width: 0px; text-decoration-thickness: initial; text-decoration-style: initial; text-decoration-color: initial;">The advantage of generating histograms using Apache Spark, or other distributed data engines, is that the computation can be run at scale, <span style="color: rgb(0, 0, 0); font-family: &quot;Times New Roman&quot;; font-size: medium; font-style: normal; font-variant-ligatures: normal; font-variant-caps: normal; font-weight: 400; letter-spacing: normal; orphans: 2; text-align: left; text-indent: 0px; text-transform: none; white-space: normal; widows: 2; word-spacing: 0px; -webkit-text-stroke-width: 0px; text-decoration-thickness: initial; text-decoration-style: initial; text-decoration-color: initial; display: inline !important; float: none;">with higher bandwidth to the data.<span> </span></span>This is useful if you have large datasets, for example, datasets that require distributed computing as they cannot be timely computed by one machine.</p> <p style="color: rgb(0, 0, 0); font-family: &quot;Times New Roman&quot;; font-size: medium; font-style: normal; font-variant-ligatures: normal; font-variant-caps: normal; font-weight: 400; letter-spacing: normal; orphans: 2; text-align: left; text-indent: 0px; text-transform: none; white-space: normal; widows: 2; word-spacing: 0px; -webkit-text-stroke-width: 0px; text-decoration-thickness: initial; text-decoration-style: initial; text-decoration-color: initial;">When handling smaller amounts of data, however, you can evaluate the alternative of just processing filters and map functions at scale, then fetching all the results into the driver, and finally using state-of-the-art libraries to generate histograms, such as <a data-original-attrs="{&quot;data-original-href&quot;:&quot;https://pandas.pydata.org/docs/reference/api/pandas.DataFrame.hist.html&quot;}" href="https://pandas.pydata.org/docs/reference/api/pandas.DataFrame.hist.html" rel="nofollow">Pandas histogram</a> or <a data-original-attrs="{&quot;data-original-href&quot;:&quot;https://numpy.org/doc/stable/reference/generated/numpy.histogram.html&quot;}" href="https://numpy.org/doc/stable/reference/generated/numpy.histogram.html" rel="nofollow">numpy histogram</a> or <a data-original-attrs="{&quot;data-original-href&quot;:&quot;https://boost-histogram.readthedocs.io/en/latest/&quot;}" href="https://boost-histogram.readthedocs.io/en/latest/" rel="nofollow">boost-histogram</a>.</p> <p style="color: rgb(0, 0, 0); font-family: &quot;Times New Roman&quot;; font-size: medium; font-style: normal; font-variant-ligatures: normal; font-variant-caps: normal; font-weight: 400; letter-spacing: normal; orphans: 2; text-align: left; text-indent: 0px; text-transform: none; white-space: normal; widows: 2; word-spacing: 0px; -webkit-text-stroke-width: 0px; text-decoration-thickness: initial; text-decoration-style: initial; text-decoration-color: initial;"> <br /><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"></span></span></span></span></span></span></span></span></p> <h3 data-original-attrs="{&quot;style&quot;:&quot;&quot;}" style="text-align: left;"><strong><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"></span></span></span></span></span></span></span></span><font color="#ff0000" data-keep-original-tag="false" data-original-attrs="{&quot;style&quot;:&quot;&quot;}">Vanilla solution: Spark's native histogram function</font></strong></h3> <p dir="auto" style="text-align:left"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><span style="color: rgb(0, 0, 0); font-family: &quot;Times New Roman&quot;; font-size: medium; font-style: normal; font-variant-ligatures: normal; font-variant-caps: normal; font-weight: 400; letter-spacing: normal; orphans: 2; text-align: left; text-indent: 0px; text-transform: none; white-space: normal; widows: 2; word-spacing: 0px; -webkit-text-stroke-width: 0px; text-decoration-thickness: initial; text-decoration-style: initial; text-decoration-color: initial; display: inline !important; float: none;">Apache Spark has a DataFrame aggregate function for generating approximate histograms, </span><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><a href="https://spark.apache.org/docs/latest/api/sql/index.html#histogram_numeric" rel="nofollow">histogram_numeric</a></span></span></span></span></span></span></span></span></span></span><span style="color: rgb(0, 0, 0); font-family: &quot;Times New Roman&quot;; font-size: medium; font-style: normal; font-variant-ligatures: normal; font-variant-caps: normal; font-weight: 400; letter-spacing: normal; orphans: 2; text-align: left; text-indent: 0px; text-transform: none; white-space: normal; widows: 2; word-spacing: 0px; -webkit-text-stroke-width: 0px; text-decoration-thickness: initial; text-decoration-style: initial; text-decoration-color: initial; display: inline !important; float: none;">,<span> </span></span><span style="color: rgb(0, 0, 0); font-family: &quot;Times New Roman&quot;; font-size: medium; font-style: normal; font-variant-ligatures: normal; font-variant-caps: normal; font-weight: 400; letter-spacing: normal; orphans: 2; text-align: left; text-indent: 0px; text-transform: none; white-space: normal; widows: 2; word-spacing: 0px; -webkit-text-stroke-width: 0px; text-decoration-thickness: initial; text-decoration-style: initial; text-decoration-color: initial; display: inline !important; float: none;">since version 3.3.0 (see<span> </span></span><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><a href="https://issues.apache.org/jira/browse/SPARK-16280" rel="nofollow">SPARK-16280</a></span></span></span></span></span></span></span></span></span></span><span style="color: rgb(0, 0, 0); font-family: &quot;Times New Roman&quot;; font-size: medium; font-style: normal; font-variant-ligatures: normal; font-variant-caps: normal; font-weight: 400; letter-spacing: normal; orphans: 2; text-align: left; text-indent: 0px; text-transform: none; white-space: normal; widows: 2; word-spacing: 0px; -webkit-text-stroke-width: 0px; text-decoration-thickness: initial; text-decoration-style: initial; text-decoration-color: initial; display: inline !important; float: none;">). There are a few implementation details and limitations to keep in mind when using histogram_numeric</span></span></span></span></span></span></span></span></span></span></span><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"><span style="color: rgb(0, 0, 0); font-family: &quot;Times New Roman&quot;; font-size: medium; font-style: normal; font-variant-ligatures: normal; font-variant-caps: normal; font-weight: 400; letter-spacing: normal; orphans: 2; text-align: left; text-indent: 0px; text-transform: none; white-space: normal; widows: 2; word-spacing: 0px; -webkit-text-stroke-width: 0px; text-decoration-thickness: initial; text-decoration-style: initial; text-decoration-color: initial; display: inline !important; float: none;">:</span></span></span></span></span></span></span></span></span></span></span></p> <p style="text-align:left"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial"></span></span></span></span></span></span></span></span></span></span></p> <ul dir="auto" style="color:#000000; font-family:&quot;Times New Roman&quot;; font-size:medium; font-style:normal; font-variant-ligatures:normal; font-weight:400; text-align:left; white-space:normal; text-decoration-thickness:initial; text-decoration-style:initial; text-decoration-color:initial"><li>it produces as output an array of (x,y) pairs representing the center of the histogram bins and their corresponding value.</li> <li>bins don't have a uniform size</li> <li>the result is an approximate calculation</li> <li>when using a large number of bins (e.g. more than 1000 bins) the histogram_numeric can become quite slow</li> </ul><p style="text-align:left"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial">See also this <a data-original-attrs="{&quot;data-original-href&quot;:&quot;https://github.com/LucaCanali/Miscellaneous/blob/master/Spark_Notes/Spark_DataFrame_Histograms.md&quot;}" href="https://github.com/LucaCanali/Miscellaneous/blob/master/Spark_Notes/Spark_DataFrame_Histograms.md#sparks-histogram_numeric-function">link for an example of how to use histogram_numeric.</a></span></span></span></span></span></span></span></span></span></span></p> <p style="text-align:left"><span style="font-size:medium"><span style="color:#000000"><span style="font-family:&quot;Times New Roman&quot;"><span style="font-style:normal"><span style="font-variant-ligatures:normal"><span style="font-weight:400"><span style="white-space:normal"><span style="text-decoration-thickness:initial"><span style="text-decoration-style:initial"><span style="text-decoration-color:initial">Given the limitations of histogram_numeric, we have developed a different solution based on the DataFrame API (see next paragraph).</span></span></span></span></span></span></span></span></span></span></p> <p style="color: rgb(0, 0, 0); font-family: &quot;Times New Roman&quot;; font-size: medium; font-style: normal; font-variant-ligatures: normal; font-variant-caps: normal; font-weight: 400; letter-spacing: normal; orphans: 2; text-align: left; text-indent: 0px; text-transform: none; white-space: normal; widows: 2; word-spacing: 0px; -webkit-text-stroke-width: 0px; text-decoration-thickness: initial; text-decoration-style: initial; text-decoration-color: initial;">  </p> <h3 data-original-attrs="{&quot;style&quot;:&quot;&quot;}" style="color: rgb(0, 0, 0); font-family: &quot;Times New Roman&quot;; font-style: normal; font-variant-ligatures: normal; font-variant-caps: normal; letter-spacing: normal; orphans: 2; text-align: left; text-indent: 0px; text-transform: none; white-space: normal; widows: 2; word-spacing: 0px; -webkit-text-stroke-width: 0px; text-decoration-thickness: initial; text-decoration-style: initial; text-decoration-color: initial;"><strong><font color="#ff0000" data-keep-original-tag="false" data-original-attrs="{&quot;style&quot;:&quot;&quot;}">An improved solution: reduce boilerplate code with SparkHistogram,</font></strong></h3> <p style="color: rgb(0, 0, 0); font-family: &quot;Times New Roman&quot;; font-size: medium; font-style: normal; font-variant-ligatures: normal; font-variant-caps: normal; font-weight: 400; letter-spacing: normal; orphans: 2; text-align: left; text-indent: 0px; text-transform: none; white-space: normal; widows: 2; word-spacing: 0px; -webkit-text-stroke-width: 0px; text-decoration-thickness: initial; text-decoration-style: initial; text-decoration-color: initial;">Apache Spark does not have official DataFrame histogram functions, however, it is easy to implement some basic histogram generation using the DataFrame API or Spark SQL. For a few simple cases, a wrapper around  the <a data-original-attrs="{&quot;data-original-href&quot;:&quot;https://spark.apache.org/docs/latest/api/sql/index.html#width_bucket&quot;}" href="https://spark.apache.org/docs/latest/api/sql/index.html#width_bucket" rel="nofollow">width_bucket</a> function can do the job. Width_bucket is a common function in many SQL engines including Apache Spark since version 3.1.0.</p> <p style="color: rgb(0, 0, 0); font-family: &quot;Times New Roman&quot;; font-size: medium; font-style: normal; font-variant-ligatures: normal; font-variant-caps: normal; font-weight: 400; letter-spacing: normal; orphans: 2; text-align: left; text-indent: 0px; text-transform: none; white-space: normal; widows: 2; word-spacing: 0px; -webkit-text-stroke-width: 0px; text-decoration-thickness: initial; text-decoration-style: initial; text-decoration-color: initial;">A simple expression for computing the histogram works by mapping each data value into a bucket and then aggregating the values in each bucket using the count function, as in this example:</p> <table class="highlight tab-size js-file-line-container js-code-nav-container js-tagsearch-file" data-paste-markdown-skip="" data-tab-size="8" data-tagsearch-lang="Python" data-tagsearch-path="Spark_Notes/Spark_Histograms/python/sparkhistogram/histogram.py" height="181" style="font-family: &quot;Times New Roman&quot;; letter-spacing: normal; text-transform: none; word-spacing: 0px; -webkit-text-stroke-width: 0px; text-decoration-thickness: initial; text-decoration-style: initial; text-decoration-color: initial;" width="793"><tbody><tr><td class="blob-code blob-code-inner js-file-line" id="LC32"> <p><span class="pl-s1"><font face="courier">  <br /> hist = (df</font></span></p> <p><font face="courier">.selectExpr("width_bucket(column_name, min_val, max_val, num_bins) as bucket")</font></p> <p><font face="courier">.groupBy("bucket")</font></p> <p><font style="color:#000000; font-size:medium; font-style:normal; font-variant-ligatures:normal; font-weight:400; text-align:left; white-space:normal; text-decoration-thickness:initial; text-decoration-style:initial; text-decoration-color:initial"><font face="courier">.count() )</font></font></p> </td> </tr><tr></tr></tbody></table><p> </p> <div style="color: rgb(0, 0, 0); font-family: &quot;Times New Roman&quot;; font-size: medium; font-style: normal; font-variant-ligatures: normal; font-variant-caps: normal; font-weight: 400; letter-spacing: normal; orphans: 2; text-align: left; text-indent: 0px; text-transform: none; white-space: normal; widows: 2; word-spacing: 0px; -webkit-text-stroke-width: 0px; text-decoration-thickness: initial; text-decoration-style: initial; text-decoration-color: initial;"><span style="color: rgb(0, 0, 0); font-family: &quot;Times New Roman&quot;; font-size: medium; font-style: normal; font-variant-ligatures: normal; font-variant-caps: normal; font-weight: 400; letter-spacing: normal; orphans: 2; text-align: left; text-indent: 0px; text-transform: none; white-space: normal; widows: 2; word-spacing: 0px; -webkit-text-stroke-width: 0px; text-decoration-thickness: initial; text-decoration-style: initial; text-decoration-color: initial; display: inline !important; float: none;">The implementation is straightforward,</span> however, additional code is needed to make it more useful in practice: we need to take care of buckets with no elements, and of  computing the data value to assign to each bucket. The resulting expression can be found, for example, in the<span> </span><a data-original-attrs="{&quot;data-original-href&quot;:&quot;https://github.com/LucaCanali/Miscellaneous/blob/master/Spark_Notes/Spark_Histograms/python/sparkhistogram/histogram.py&quot;}" href="https://github.com/LucaCanali/Miscellaneous/blob/master/Spark_Notes/Spark_Histograms/python/sparkhistogram/histogram.py">code of the computeHistogram</a> function.   <div> <p>The<span> </span><a data-original-attrs="{&quot;data-original-href&quot;:&quot;https://pypi.org/project/sparkhistogram/&quot;}" href="https://pypi.org/project/sparkhistogram/">SparkHistogram<span> </span></a>package is built with the idea of reducing boilerplate code and contains helper functions for generating frequency histograms and also a close variant of it, weighted histograms. Computing histograms with SparkHistogram becomes simply:</p> <table class="highlight tab-size js-file-line-container js-code-nav-container js-tagsearch-file" data-paste-markdown-skip="" data-tab-size="8" data-tagsearch-lang="Python" data-tagsearch-path="Spark_Notes/Spark_Histograms/python/sparkhistogram/histogram.py" style="font-family:&quot;Times New Roman&quot;; text-decoration-thickness:initial; text-decoration-style:initial; text-decoration-color:initial"><tbody><tr><td class="blob-code blob-code-inner js-file-line" id="LC32"> <p><font face="courier">from sparkhistogram import computeHistogram</font></p> <p><font face="courier">hist = computeHistogram(df, f"{data_column}", min_val, max_val, num_bins)</font></p> <p><span style="font-family: courier;"># or, in alternative:</span></p> <p><span style="font-family: courier;">hist = df.transform(computeHistogram, f"{data_column}", min_val, max_val, num_bins)</span></p> </td> </tr><tr></tr></tbody></table><p> </p> <p>More information on the SparkHistogram package for Python and Scala at:</p> </div> </div> <ul style="color: rgb(0, 0, 0); font-family: &quot;Times New Roman&quot;; font-size: medium; font-style: normal; font-variant-ligatures: normal; font-variant-caps: normal; font-weight: 400; letter-spacing: normal; orphans: 2; text-align: left; text-indent: 0px; text-transform: none; white-space: normal; widows: 2; word-spacing: 0px; -webkit-text-stroke-width: 0px; text-decoration-thickness: initial; text-decoration-style: initial; text-decoration-color: initial;"><li><b>Python:<span> </span><a data-original-attrs="{&quot;data-original-href&quot;:&quot;https://pypi.org/project/sparkhistogram/&quot;}" href="https://pypi.org/project/sparkhistogram/">SparkHistogram on Pypi</a></b><br />  </li> <li><b>Scala:<span> </span></b><b><a data-original-attrs="{&quot;data-original-href&quot;:&quot;https://github.com/LucaCanali/Miscellaneous/tree/master/Spark_Notes/Spark_Histograms/scala&quot;}" href="https://github.com/LucaCanali/Miscellaneous/tree/master/Spark_Notes/Spark_Histograms/scala">SparkHistogram in Scala</a></b></li> </ul><div><strong style="color: rgb(0, 0, 0); font-family: &quot;Times New Roman&quot;; font-size: medium; font-style: normal; font-variant-ligatures: normal; font-variant-caps: normal; letter-spacing: normal; orphans: 2; text-align: left; text-indent: 0px; text-transform: none; white-space: normal; widows: 2; word-spacing: 0px; -webkit-text-stroke-width: 0px; text-decoration-thickness: initial; text-decoration-style: initial; text-decoration-color: initial;"><strong>   </strong></strong></div> <h3 data-original-attrs="{&quot;style&quot;:&quot;&quot;}" style="color: rgb(0, 0, 0); font-family: &quot;Times New Roman&quot;; font-style: normal; font-variant-ligatures: normal; font-variant-caps: normal; letter-spacing: normal; orphans: 2; text-align: left; text-indent: 0px; text-transform: none; white-space: normal; widows: 2; word-spacing: 0px; -webkit-text-stroke-width: 0px; text-decoration-thickness: initial; text-decoration-style: initial; text-decoration-color: initial;"><strong><font color="#ff0000" data-keep-original-tag="false" data-original-attrs="{&quot;style&quot;:&quot;&quot;}">Examples</font></strong></h3> <div style="color: rgb(0, 0, 0); font-family: &quot;Times New Roman&quot;; font-size: medium; font-style: normal; font-variant-ligatures: normal; font-variant-caps: normal; font-weight: 400; letter-spacing: normal; orphans: 2; text-align: left; text-indent: 0px; text-transform: none; white-space: normal; widows: 2; word-spacing: 0px; -webkit-text-stroke-width: 0px; text-decoration-thickness: initial; text-decoration-style: initial; text-decoration-color: initial;"><span style="color: rgb(0, 0, 0); font-family: &quot;Times New Roman&quot;; font-size: medium; font-style: normal; font-variant-ligatures: normal; font-variant-caps: normal; font-weight: 400; letter-spacing: normal; orphans: 2; text-align: left; text-indent: 0px; text-transform: none; white-space: normal; widows: 2; word-spacing: 0px; -webkit-text-stroke-width: 0px; text-decoration-thickness: initial; text-decoration-style: initial; text-decoration-color: initial; display: inline !important; float: none;"><span style="color: rgb(0, 0, 0); font-family: &quot;Times New Roman&quot;; font-size: medium; font-style: normal; font-variant-ligatures: normal; font-variant-caps: normal; font-weight: 400; letter-spacing: normal; orphans: 2; text-align: left; text-indent: 0px; text-transform: none; white-space: normal; widows: 2; word-spacing: 0px; -webkit-text-stroke-width: 0px; text-decoration-thickness: initial; text-decoration-style: initial; text-decoration-color: initial; display: inline !important; float: none;">Jupyter notebooks showing how to generate histograms using PySpark and SparkHistogram (see further in this post for Spark SQL examples):</span></span></div> <div style="color: rgb(0, 0, 0); font-family: &quot;Times New Roman&quot;; font-size: medium; font-style: normal; font-variant-ligatures: normal; font-variant-caps: normal; font-weight: 400; letter-spacing: normal; orphans: 2; text-align: left; text-indent: 0px; text-transform: none; white-space: normal; widows: 2; word-spacing: 0px; -webkit-text-stroke-width: 0px; text-decoration-thickness: initial; text-decoration-style: initial; text-decoration-color: initial;"> </div> <ul dir="auto" style="color: rgb(0, 0, 0); font-family: &quot;Times New Roman&quot;; font-size: medium; font-style: normal; font-variant-ligatures: normal; font-variant-caps: normal; font-weight: 400; letter-spacing: normal; orphans: 2; text-align: left; text-indent: 0px; text-transform: none; white-space: normal; widows: 2; word-spacing: 0px; -webkit-text-stroke-width: 0px; text-decoration-thickness: initial; text-decoration-style: initial; text-decoration-color: initial;"><li><a data-original-attrs="{&quot;data-original-href&quot;:&quot;https://github.com/LucaCanali/Miscellaneous/blob/master/Spark_Notes/Spark_Histograms/Spark_Histograms/Spark_DataFrame_Frequency_Histograms.ipynb&quot;}" href="https://github.com/LucaCanali/Miscellaneous/blob/master/Spark_Notes/Spark_Histograms/Spark_Histograms/Spark_DataFrame_Frequency_Histograms.ipynb"><b>Frequency histograms using the DataFrame API</b></a><br />  </li> <li><a data-original-attrs="{&quot;data-original-href&quot;:&quot;https://github.com/LucaCanali/Miscellaneous/blob/master/Spark_Notes/Spark_Histograms/Spark_Histograms/Spark_DataFrame_Weighted_Histograms.ipynb&quot;}" href="https://github.com/LucaCanali/Miscellaneous/blob/master/Spark_Notes/Spark_Histograms/Spark_Histograms/Spark_DataFrame_Weighted_Histograms.ipynb"><b>Weighted histograms using the DataFrame API</b></a></li> </ul><div style="color: rgb(0, 0, 0); font-family: &quot;Times New Roman&quot;; font-size: medium; font-style: normal; font-variant-ligatures: normal; font-variant-caps: normal; font-weight: 400; letter-spacing: normal; orphans: 2; text-align: left; text-indent: 0px; text-transform: none; white-space: normal; widows: 2; word-spacing: 0px; -webkit-text-stroke-width: 0px; text-decoration-thickness: initial; text-decoration-style: initial; text-decoration-color: initial;"> </div> <div style="color: rgb(0, 0, 0); font-family: &quot;Times New Roman&quot;; font-size: medium; font-style: normal; font-variant-ligatures: normal; font-variant-caps: normal; font-weight: 400; letter-spacing: normal; orphans: 2; text-align: left; text-indent: 0px; text-transform: none; white-space: normal; widows: 2; word-spacing: 0px; -webkit-text-stroke-width: 0px; text-decoration-thickness: initial; text-decoration-style: initial; text-decoration-color: initial;">Additional examples in the context of Physics analysis:<br />  </div> <div style="color: rgb(0, 0, 0); font-family: &quot;Times New Roman&quot;; font-size: medium; font-style: normal; font-variant-ligatures: normal; font-variant-caps: normal; font-weight: 400; letter-spacing: normal; orphans: 2; text-align: left; text-indent: 0px; text-transform: none; white-space: normal; widows: 2; word-spacing: 0px; -webkit-text-stroke-width: 0px; text-decoration-thickness: initial; text-decoration-style: initial; text-decoration-color: initial;"> <ul data-original-attrs="{&quot;style&quot;:&quot;&quot;}" style="text-align: left;"><li><a data-original-attrs="{&quot;data-original-href&quot;:&quot;https://github.com/LucaCanali/Miscellaneous/tree/master/Spark_Physics&quot;}" href="https://github.com/LucaCanali/Miscellaneous/tree/master/Spark_Physics"><b>Apache Spark for High Energy Physics</b></a></li> </ul></div> <div style="color: rgb(0, 0, 0); font-family: &quot;Times New Roman&quot;; font-size: medium; font-style: normal; font-variant-ligatures: normal; font-variant-caps: normal; font-weight: 400; letter-spacing: normal; orphans: 2; text-align: left; text-indent: 0px; text-transform: none; white-space: normal; widows: 2; word-spacing: 0px; -webkit-text-stroke-width: 0px; text-decoration-thickness: initial; text-decoration-style: initial; text-decoration-color: initial;">      <div class="separator" data-original-attrs="{&quot;style&quot;:&quot;&quot;}" style="clear: both; text-align: center;"><a data-original-attrs="{&quot;data-original-href&quot;:&quot;https://blogger.googleusercontent.com/img/b/R29vZ2xl/AVvXsEjdZpoGax3gSysOZ-H4FVUXOUJ_eAOZ5Z_1GFRgkVvupFW4vhcmd4fftt-HkKZZm4JXmnmh2jlBC8rALDKMwZ5_9kVWerZnYpQR5kWbXGYUEYw343URvvjnVpgITssnpN67ze9OCW7zUFzLHcb6DIf_L1i_cQoObyxjdWSx6Royy3tXH2R1-hufqHzV/s984/Atlas_histogram_Higgs.png&quot;,&quot;style&quot;:&quot;&quot;}" href="https://www.blogger.com/blog/post/edit/7003976656201910397/3019389568308972542#" style="margin-left: 1em; margin-right: 1em;"></a><a data-entity-type="" data-entity-uuid="" href="https://blogger.googleusercontent.com/img/b/R29vZ2xl/AVvXsEjdZpoGax3gSysOZ-H4FVUXOUJ_eAOZ5Z_1GFRgkVvupFW4vhcmd4fftt-HkKZZm4JXmnmh2jlBC8rALDKMwZ5_9kVWerZnYpQR5kWbXGYUEYw343URvvjnVpgITssnpN67ze9OCW7zUFzLHcb6DIf_L1i_cQoObyxjdWSx6Royy3tXH2R1-hufqHzV/s984/Atlas_histogram_Higgs.png"><img alt="" border="0" data-entity-type="" data-entity-uuid="" data-original-height="686" data-original-width="984" height="446" src="https://blogger.googleusercontent.com/img/b/R29vZ2xl/AVvXsEjdZpoGax3gSysOZ-H4FVUXOUJ_eAOZ5Z_1GFRgkVvupFW4vhcmd4fftt-HkKZZm4JXmnmh2jlBC8rALDKMwZ5_9kVWerZnYpQR5kWbXGYUEYw343URvvjnVpgITssnpN67ze9OCW7zUFzLHcb6DIf_L1i_cQoObyxjdWSx6Royy3tXH2R1-hufqHzV/w640-h446/Atlas_histogram_Higgs.png" style="cursor: move;" width="640" /></a></div> <p><span style="color: rgb(0, 0, 0); font-family: &quot;Times New Roman&quot;; font-size: medium; font-style: normal; font-variant-ligatures: normal; font-variant-caps: normal; font-weight: 400; letter-spacing: normal; orphans: 2; text-align: left; text-indent: 0px; text-transform: none; white-space: normal; widows: 2; word-spacing: 0px; -webkit-text-stroke-width: 0px; text-decoration-thickness: initial; text-decoration-style: initial; text-decoration-color: initial; display: inline !important; float: none;">This histogram has been generated using<span> </span></span><a data-original-attrs="{&quot;data-original-href&quot;:&quot;https://atlas.cern/Resources/Opendata&quot;}" href="https://atlas.cern/Resources/Opendata" style="font-family: &quot;Times New Roman&quot;; font-size: medium; font-style: normal; font-variant-ligatures: normal; font-variant-caps: normal; font-weight: 400; letter-spacing: normal; orphans: 2; text-align: left; text-indent: 0px; text-transform: none; white-space: normal; widows: 2; word-spacing: 0px; -webkit-text-stroke-width: 0px;">ATLAS Open Data</a><span style="color: rgb(0, 0, 0); font-family: &quot;Times New Roman&quot;; font-size: medium; font-style: normal; font-variant-ligatures: normal; font-variant-caps: normal; font-weight: 400; letter-spacing: normal; orphans: 2; text-align: left; text-indent: 0px; text-transform: none; white-space: normal; widows: 2; word-spacing: 0px; -webkit-text-stroke-width: 0px; text-decoration-thickness: initial; text-decoration-style: initial; text-decoration-color: initial; display: inline !important; float: none;"><span> </span>collected at the LHC at CERN and processed using PySpark and the SparkHistogram package.</span></p></div> <div style="color: rgb(0, 0, 0); font-family: &quot;Times New Roman&quot;; font-size: medium; font-style: normal; font-variant-ligatures: normal; font-variant-caps: normal; font-weight: 400; letter-spacing: normal; orphans: 2; text-align: left; text-indent: 0px; text-transform: none; white-space: normal; widows: 2; word-spacing: 0px; -webkit-text-stroke-width: 0px; text-decoration-thickness: initial; text-decoration-style: initial; text-decoration-color: initial;">    </div> <h3 data-original-attrs="{&quot;style&quot;:&quot;&quot;}" style="color: rgb(0, 0, 0); font-family: &quot;Times New Roman&quot;; font-style: normal; font-variant-ligatures: normal; font-variant-caps: normal; letter-spacing: normal; orphans: 2; text-align: left; text-indent: 0px; text-transform: none; white-space: normal; widows: 2; word-spacing: 0px; -webkit-text-stroke-width: 0px; text-decoration-thickness: initial; text-decoration-style: initial; text-decoration-color: initial;"><strong><font color="#ff0000" data-keep-original-tag="false" data-original-attrs="{&quot;style&quot;:&quot;&quot;}">Extend the work on histogram generation to more SQL engines</font></strong></h3> <div style="color: rgb(0, 0, 0); font-family: &quot;Times New Roman&quot;; font-size: medium; font-style: normal; font-variant-ligatures: normal; font-variant-caps: normal; font-weight: 400; letter-spacing: normal; orphans: 2; text-align: left; text-indent: 0px; text-transform: none; white-space: normal; widows: 2; word-spacing: 0px; -webkit-text-stroke-width: 0px; text-decoration-thickness: initial; text-decoration-style: initial; text-decoration-color: initial;"><span style="color: rgb(0, 0, 0); font-family: &quot;Times New Roman&quot;; font-size: medium; font-style: normal; font-variant-ligatures: normal; font-variant-caps: normal; font-weight: 400; letter-spacing: normal; orphans: 2; text-align: left; text-indent: 0px; text-transform: none; white-space: normal; widows: 2; word-spacing: 0px; -webkit-text-stroke-width: 0px; text-decoration-thickness: initial; text-decoration-style: initial; text-decoration-color: initial; display: inline !important; float: none;">You can also use SQL to generate your histograms. The following examples work with minor modifications across different data/database systems and can be easily extended to run on all SQL engines that implement the width_bucket function. Example notebooks:</span></div> <div style="color: rgb(0, 0, 0); font-family: &quot;Times New Roman&quot;; font-size: medium; font-style: normal; font-variant-ligatures: normal; font-variant-caps: normal; font-weight: 400; letter-spacing: normal; orphans: 2; text-align: left; text-indent: 0px; text-transform: none; white-space: normal; widows: 2; word-spacing: 0px; -webkit-text-stroke-width: 0px; text-decoration-thickness: initial; text-decoration-style: initial; text-decoration-color: initial;"> </div> <div style="color: rgb(0, 0, 0); font-family: &quot;Times New Roman&quot;; font-size: medium; font-style: normal; font-variant-ligatures: normal; font-variant-caps: normal; font-weight: 400; letter-spacing: normal; orphans: 2; text-align: left; text-indent: 0px; text-transform: none; white-space: normal; widows: 2; word-spacing: 0px; -webkit-text-stroke-width: 0px; text-decoration-thickness: initial; text-decoration-style: initial; text-decoration-color: initial;"> <ul dir="auto"><li><a data-original-attrs="{&quot;data-original-href&quot;:&quot;https://github.com/LucaCanali/Miscellaneous/blob/master/Spark_Notes/Spark_Histograms/Spark_Histograms/Spark_SQL_Frequency_Histograms.ipynb&quot;}" href="https://github.com/LucaCanali/Miscellaneous/blob/master/Spark_Notes/Spark_Histograms/Spark_SQL_Frequency_Histograms.ipynb"><b>frequency histograms using Spark SQL</b></a></li> <li><a data-original-attrs="{&quot;data-original-href&quot;:&quot;https://github.com/LucaCanali/Miscellaneous/blob/master/Trino_Presto_Jupyter/Trino_histograms.ipynb&quot;}" href="https://github.com/LucaCanali/Miscellaneous/blob/master/Trino_Presto_Jupyter/Trino_histograms.ipynb"><b>frequency histograms using_Trino_or_Presto</b></a></li> <li><a data-original-attrs="{&quot;data-original-href&quot;:&quot;https://github.com/LucaCanali/Miscellaneous/blob/master/PostgreSQL_YugabyteDB_Jupyter/PostgreSQL_histograms.ipynb&quot;}" href="https://github.com/LucaCanali/Miscellaneous/blob/master/PostgreSQL_YugabyteDB_Jupyter/PostgreSQL_histograms.ipynb"><b>frequency histograms using_PostgreSQL</b></a></li> <li><a data-original-attrs="{&quot;data-original-href&quot;:&quot;https://github.com/LucaCanali/Miscellaneous/blob/master/PostgreSQL_YugabyteDB_Jupyter/YugabyteDB_histograms.ipynb&quot;}" href="https://github.com/LucaCanali/Miscellaneous/blob/master/PostgreSQL_YugabyteDB_Jupyter/YugabyteDB_histograms.ipynb"><b>frequency histograms using_YugabyteDB</b></a></li> <li><a data-original-attrs="{&quot;data-original-href&quot;:&quot;https://github.com/LucaCanali/Miscellaneous/blob/master/Oracle_Jupyter/Oracle_histograms.ipynb&quot;}" href="https://github.com/LucaCanali/Miscellaneous/blob/master/Oracle_Jupyter/Oracle_histograms.ipynb"><b>frequency histograms using_Oracle</b></a></li> </ul><div>   </div> <h3 data-original-attrs="{&quot;style&quot;:&quot;&quot;}" style="text-align: left;"><strong><font color="#ff0000" data-keep-original-tag="false" data-original-attrs="{&quot;style&quot;:&quot;&quot;}">References and acknowledgments</font></strong></h3> <div>A note with more details on this work at: <a data-original-attrs="{&quot;data-original-href&quot;:&quot;https://github.com/LucaCanali/Miscellaneous/blob/master/Spark_Notes/Spark_DataFrame_Histograms.md&quot;}" href="https://github.com/LucaCanali/Miscellaneous/blob/master/Spark_Notes/Spark_DataFrame_Histograms.md">https://github.com/LucaCanali/Miscellaneous/blob/master/Spark_Notes/Spark_DataFrame_Histograms.md</a></div> </div> <div style="color: rgb(0, 0, 0); font-family: &quot;Times New Roman&quot;; font-size: medium; font-style: normal; font-variant-ligatures: normal; font-variant-caps: normal; font-weight: 400; letter-spacing: normal; orphans: 2; text-align: left; text-indent: 0px; text-transform: none; white-space: normal; widows: 2; word-spacing: 0px; -webkit-text-stroke-width: 0px; text-decoration-thickness: initial; text-decoration-style: initial; text-decoration-color: initial;">For more complex histogram use cases with Spark see also<span> </span><a data-original-attrs="{&quot;data-original-href&quot;:&quot;https://github.com/histogrammar/histogrammar-python#example-notebooks&quot;}" href="https://github.com/histogrammar/histogrammar-python#example-notebooks">Histogrammar</a></div> <div style="color: rgb(0, 0, 0); font-family: &quot;Times New Roman&quot;; font-size: medium; font-style: normal; font-variant-ligatures: normal; font-variant-caps: normal; font-weight: 400; letter-spacing: normal; orphans: 2; text-align: left; text-indent: 0px; text-transform: none; white-space: normal; widows: 2; word-spacing: 0px; -webkit-text-stroke-width: 0px; text-decoration-thickness: initial; text-decoration-style: initial; text-decoration-color: initial;">  </div> <div style="color: rgb(0, 0, 0); font-family: &quot;Times New Roman&quot;; font-size: medium; font-style: normal; font-variant-ligatures: normal; font-variant-caps: normal; font-weight: 400; letter-spacing: normal; orphans: 2; text-align: left; text-indent: 0px; text-transform: none; white-space: normal; widows: 2; word-spacing: 0px; -webkit-text-stroke-width: 0px; text-decoration-thickness: initial; text-decoration-style: initial; text-decoration-color: initial;">This work has been done in the context of the CERN databases and analytics services and the ATLAS data engineering efforts. Additional thanks to Jim Pivarski for discussions.</div> <div style="color: rgb(0, 0, 0); font-family: &quot;Times New Roman&quot;; font-size: medium; font-style: normal; font-variant-ligatures: normal; font-variant-caps: normal; font-weight: 400; letter-spacing: normal; orphans: 2; text-align: left; text-indent: 0px; text-transform: none; white-space: normal; widows: 2; word-spacing: 0px; -webkit-text-stroke-width: 0px; text-decoration-thickness: initial; text-decoration-style: initial; text-decoration-color: initial;">  </div> <p> </p> </div> </div> <span><a title="View user profile." href="/users/luca-canali" lang="" about="/users/luca-canali" typeof="schema:Person" property="schema:name" datatype="">canali</a></span> <span><time datetime="2022-05-23T20:32:17+02:00" title="Monday, May 23, 2022 - 20:32">Mon, 05/23/2022 - 20:32</time> </span> <div class="field field--name-field-tags field--type-entity-reference field--label-above"> <div class="field--label"><b>Tags</b></div> <div class="field--items"> <div class="field--item"><a href="/tags/apache-spark" hreflang="en">Apache Spark</a></div> </div> </div> <section> <h2>Add new comment</h2> <drupal-render-placeholder callback="comment.lazy_builders:renderForm" arguments="0=node&amp;1=187&amp;2=comment_node_blog_post&amp;3=comment_node_blog_post" token="kH_5-UeEj7R9_nEpCTWS5SwAPQlsAnEbgjoBib7qQVE"></drupal-render-placeholder> </section> Mon, 23 May 2022 18:32:17 +0000 canali 187 at https://db-blog.web.cern.ch Can High Energy Physics Analysis Profit from Apache Spark APIs? https://db-blog.web.cern.ch/node/186 <span>Can High Energy Physics Analysis Profit from Apache Spark APIs?</span> <div class="field field--name-body field--type-text-with-summary field--label-above"> <div class="field--label"><b>Blog article:</b></div> <div class="field--item"><p>We are in a golden age for distributed data processing, with an abundance of tools and solutions emerging from industry and open source. <a href="https://en.wikipedia.org/wiki/Particle_physics">High Energy Physics</a> (HEP) experiments at the <a href="https://home.cern/science/accelerators/large-hadron-collider">LHC </a>stand to profit from all this progress, as they are data-intensive operations with several hundreds of Petabytes of data to collect and process.</p> <p>This post collects a few examples of <b>code</b> and <b>open data</b>, where Apache Spark, a very popular tool in industry and open source, is used for a few simple HEP data analyses. This post aims to be a general overview both for <b>physicists</b> wanting to know more about what Spark can do, and for <b>data</b> <b>scientists</b> wanting to get a feeling of what a HEP data analysis looks like.<br /> The code and discussion here are proposed as a technology exploration and do not reflect any particular official activity by an experiment team.<br /> This post comes with series of <a href="https://github.com/LucaCanali/Miscellaneous/tree/master/Spark_Physics">notebooks at this link</a>.</p> <p><b>TLDR;</b> Apache Spark (PySpark) APIs can easily be used for simple HEP analysis tasks, for example running the analysis in notebook environments, and profiting of a cluster infrastructure for computing power. Complex analyses can be challenging to implement and often require to develop UDF (user defined function) which may increase complexity and reduce performance. Follow this link to an <a href="https://colab.research.google.com/github/LucaCanali/Miscellaneous/blob/master/Spark_Physics/Dimuon_mass_spectrum/Dimuon_mass_spectrum_histogram_Spark_DataFrame_Colab_version.ipynb" target="">example analysis notebook in Colab</a>, where you play with code and open data. </p> <p>   </p> <h2 data-original-attrs="{&quot;style&quot;:&quot;&quot;}" style="text-align:left"><b><font color="#ff0000"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;&quot;}">A High-level view of particle physics analysis</font></font></font></b></h2> <p>The <b>input</b> to the analysis work is a set of files containing <b>event</b> data. For each event, a large set of attributes  is provided, with details on the particles and physical quantities that are associated with it (photons, electrons, muons, jets, etc). Events are what comes from particle collisions collected at a <b>detector</b>, plus all the processing steps in between, to prepare it via reconstruction, calibration, etc. In other cases, event data is generated from simulations.</p> <p>Data is sliced with <b>projection</b> and <b>filter</b> operations, then specific computations are <b>processed</b> for each event of interest. In the final processing steps, data is typically <b>aggregated</b> into one or more histograms. These are the output "plots" with physical quantities of interest.</p> <p>Some good news, for data engines based on DataFrames and/or table abstractions, like Spark or SQL platforms, are: that event data have fixed <b>schemas</b>, moreover they are statistically independent, so you will typically not need to perform joins across events. Engines and data formats for <b>columnar</b> processing are also quite a good fit, as often only a subset of attributes is processed for a given analysis.</p> <p>The <b>hard</b> part, for data processing engines, is that event data is <b>nested</b>, typically containing arrays. Moreover, complex <b>formulas</b> and in some cases algorithms, are needed to process event data, which require high efficiency in <b>CPU</b> utilization. Finally, there are tons of data, and many different tests to be executed to find the "good plot".</p> <p> </p> <h2 data-original-attrs="{&quot;style&quot;:&quot;&quot;}" style="text-align:left"><font color="#ff0000"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;&quot;}">Example analyses: notebooks and open data</font></font></font></h2> <div> </div> <div> <ul><li><b><a href="https://github.com/LucaCanali/Miscellaneous/tree/master/Spark_Physics#1-dimuon-mass-spectrum-analysis" target="">Dimuon mass spectrum analysis l</a><b><a href="https://github.com/LucaCanali/Miscellaneous/tree/master/Spark_Physics#1-dimuon-mass-spectrum-analysis" target="">ink to the notebooks and data</a>:</b></b><br /> This is a sort of "Hello World!" example for High Energy Physics analysis.<br /> It's a dimuon mass spectrum analysis accompanied by open data with <b>6.5 billion events </b>(200 GB). You can find there what HEP data looks like, how to perform simple filters and reduction to histograms, finally rewarded by a nice plot!<br />  </li> <li><a href="https://github.com/LucaCanali/Miscellaneous/tree/master/Spark_Physics#2-hep-analysis-benchmark"><b>HEP benchmark, link to the notebooks and data</b></a>: This implements a series of tasks typical of Particle Physics analysis, organized as a benchmark with an accompanying dataset of 53 million events from CERN open data. You can find there example of how to process array data for HEP using the Spark DataFrame API and and some examples of UDF.<br />  </li> <li><strong><a href="https://github.com/LucaCanali/Miscellaneous/tree/master/Spark_Physics#3-atlas-higgs-boson-analysis---outreach-style">Atlas Higgs boson analysis</a></strong> of the decay channel H - ZZ* - 4lep. This is an outreach-style analysis directly inspired by the original analysis and ATLAS paper for the discovery of the Higgs boson.<br />  </li> <li><a href="https://github.com/LucaCanali/Miscellaneous/tree/master/Spark_Physics#4-lhcb-matter-antimatter-asymmetries-analysis---outreach-style"><b>LHCb matter antimatter asymmetries analysis, outreach-style</b></a>: An analysis meant for outreach by the LHCb experiment, re-implemented using the Apache Spark DataFrame API.<br />  </li> <li>Notes: <ul><li>on <a href="https://github.com/LucaCanali/Miscellaneous/tree/master/Spark_Physics#notes-on-reading-and-converting-data-stored-in-root-format">converting data in ROOT format to Apache Parquet or to ORC</a></li> <li>a few links with <a href="https://github.com/LucaCanali/Miscellaneous/tree/master/Spark_Physics#physics">additional details on the Physics terms and formulas used in the notebooks</a></li> </ul></li> </ul></div> <div> </div> <div> <p><a href="https://blogger.googleusercontent.com/img/a/AVvXsEgC9ZdOPgns5OBgEuftmTZymzro6u60mWyqVB6uPYWA3hQTcE3aH8UGuMUS4OMDGCq0QzDWWoFF08JBrB00_1GNHUrSqA6T1xI5JiCSDn0gkV4S9QvJZRU22Qm-128aerJNTWPpFB_TYAF6R78hLhmgzVx5uZjGC2XIeE8_3rLriFqijK3rlyu3fxg1=s853" imageanchor="1"></a><a data-entity-type="" data-entity-uuid="" href="https://blogger.googleusercontent.com/img/a/AVvXsEgC9ZdOPgns5OBgEuftmTZymzro6u60mWyqVB6uPYWA3hQTcE3aH8UGuMUS4OMDGCq0QzDWWoFF08JBrB00_1GNHUrSqA6T1xI5JiCSDn0gkV4S9QvJZRU22Qm-128aerJNTWPpFB_TYAF6R78hLhmgzVx5uZjGC2XIeE8_3rLriFqijK3rlyu3fxg1=w640-h474"><img alt="" border="0" data-entity-type="" data-entity-uuid="" data-original-height="633" data-original-width="853" height="474" src="https://blogger.googleusercontent.com/img/a/AVvXsEgC9ZdOPgns5OBgEuftmTZymzro6u60mWyqVB6uPYWA3hQTcE3aH8UGuMUS4OMDGCq0QzDWWoFF08JBrB00_1GNHUrSqA6T1xI5JiCSDn0gkV4S9QvJZRU22Qm-128aerJNTWPpFB_TYAF6R78hLhmgzVx5uZjGC2XIeE8_3rLriFqijK3rlyu3fxg1=w640-h474" width="640" /></a></p> <p> </p> </div> <h2 data-original-attrs="{&quot;style&quot;:&quot;&quot;}" style="text-align:left"><font color="#ff0000"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;&quot;}">Lessons learned</font></font></font></h2> <div> </div> <div> <p><b>Apache Spark API for HEP:</b></p> <ul><li>(+) The DataFrame API and Spark SQL work well for structured data like HEP data. Moreover, the key HEP data processing operations are, map, filter, and reduction to histograms, which are well implemented in Spark DataFrame API.</li> <li>(+) Physics datasets consist of a large number (GBs to 100s of TBs) of statistically independent events, which can be processed in parallel. This fits well with the Spark execution model.</li> <li>(+) Lazy evaluation in Spark allows building the analysis from small steps, each in a different piece of code, which helps exploration and allows detailed comments inside the code. All operations will be optimized together at the execution time (when an action is triggered such as fetching the histogram for plotting).</li> <li>(+) The function <i>width_bucket</i> provides an acceptable solution for computing histograms with the DataFrame API and with SQL.</li> <li>(+) Spark DataFrame API and SQL can handle complex data types with arrays and structs. It implements <i>explode </i>and <a href="https://spark.apache.org/docs/latest/api/sql/index.html#posexplode"><i>posexplode </i></a>functions, it has several <a href="https://spark.apache.org/docs/latest/api/sql/search.html?q=array">array functions</a>, it also has <a href="https://databricks.com/blog/2018/11/16/introducing-new-built-in-functions-and-higher-order-functions-for-complex-data-types-in-apache-spark.html">higher order functions</a> specialized for array processing.</li> <li>(-) Spark (3.2 and 3.3) does not implement the SQL <a href="https://cloud.google.com/bigquery/docs/reference/standard-sql/query-syntax#unnest_operator">UNNEST operator</a>. Spark does not have functions to handle natively <a href="https://en.wikipedia.org/wiki/Four-vector">4-vectors</a>.</li> <li>(-) Some of the complex data processing is hard to implement with the DataFrame API or SQL, and requires UDF.</li> </ul><p><b>  </b></p> <p><b>Data formats:</b></p> <ul><li>(+) Spark is optimized (with a vectorized reader) to ingest columnar formats such as Apache Parquet and ORC. This brings to the table performance-enhancing features such as: filter pushdown, min-max filtering with rowgroup and page index statistics, bloom filters. Spark has additional optimizations for handling complex data types (e.g. arrays) with ORC (Spark 3.2) and Parquet (Spark 3.3, see <a href="https://issues.apache.org/jira/browse/SPARK-34863">SPARK-34863</a>).</li> <li>(+/-) The <a href="https://github.com/spark-root/laurelin">Laurelin library</a>  allows reading HEP specialized data format, ROOT. However, this is still experimental and not optimized for performance, rather to be used for format conversion.</li> <li>(+/-) The examples reported here use data in a relatively flat structure (nanoaod format), which plays well with Spark DataFrame API. HEP data with more nested structures, which is common for HEP data in the recent past, introduces additional performance issue when using Spark.</li> <li>(-) The large majority of HEP data is stored in ROOT format at present. This "adds friction" when using tools from industry and open source that do not fully support it.</li> </ul><p>   </p> <p><b>Platform and ecosystem:</b></p> <ul><li>(+) PySpark works well on notebooks. Spark sessions can run locally and on clusters (stand-alone, YARN, Kubernetes) and this makes it a good building block for a data analysis platform. At CERN we have integrated the web analysis service, called <a href="https://swan.web.cern.ch/swan/">SWAN</a>, with Spark services running on YARN and Kubernetes.</li> <li>(+) Spark integrates well with cloud environments. Connectors are available to major object stores, s3 and more. For CERN storage system EOS, there is the <a href="https://github.com/cerndb/hadoop-xrootd">Hadoop-XRootD connector</a>.</li> <li>(+) Spark is a well know platform, with many libraries and integration available. Users like the idea of learning Spark as it is widely used in the industry.</li> <li>(+/-) Hardware resources for physics are made available on HPC systems and on batch systems, some work to use the standalone cluster mode is needed there.</li> </ul><p><b>  </b></p> <p><b>Performance:</b></p> <ul><li>(-) Python UDFs in Spark have improved their performance with the latest releases, but their need to serialize and deserialize the data passed to Python workers can take a considerable hit on performance, even when using Apache Arrow.</li> <li>(-) The state-of-the-art platforms for HEP analysis have large parts written in C/C++ and optimized for performance of numerical computations on HEP data, typically using vectorized computations. Apache Spark (3.2) does not have vectorized execution.</li> <li>(+/-) Using UDF written in Scala via PySpark can be useful to combine performance and advanced features (see benchmark examples Q6 and Q8), however, they add complexity and will require most users to spend time learning how to do this.</li> <li>(+/-) Spark higher-order function for array processing are expressive, but their performance in Apache Spark (3.2) could be improved (compare the 2 solutions to benchmark Q7).</li> </ul><p>   </p> <p>Note: these comments refer to the tests run for this work in 2022, using Apache Spark versions 3.2.1 and 3.3.0.</p> </div> <div>  </div> <div> <h2 data-original-attrs="{&quot;style&quot;:&quot;&quot;}" style="text-align:left"><font color="#ff0000"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;&quot;}">Conclusions</font></font></font></h2> </div> <div>  </div> <div>Apache Spark provides a suitable API, platform, and ecosystem for High Energy Physics data analysis, with some caveats. The examples shown here demonstrate how PySpark on notebooks can be used to write simple analysis code and run it locally or at scale on clusters. Since several years, CERN runs notebooks service integrated with YARN and Kubernetes clusters and cloud storage.</div> <div>Spark DataFrame API works surprisingly well for several simple HEP use cases, notably prodicing histograms at scale. One key caveat is that DataFrame APIs need to be supplemented with the help of user defined functions (UDF) for most the complex real-world cases. The performance of UDFs, in particular when written in Python, are a concern. Also a concern is the current need to read/convert files in ROOT format, as Spark is rather optimized for data formats common in industry, like Apache Parquet and ORC. </div> <div>What is reported here complements previous work on using Apache Spark for data reduction at scale and data preparation for a ML tasks (see references below).</div> <div>Additional work is needed both on the HEP and Apache Spark sides to bring Apache Spark up-to-speed with specialized HEP analysis software in their optimization domain (see links below in related work).</div> <div>  </div> <h2 data-original-attrs="{&quot;style&quot;:&quot;&quot;}" style="text-align:left"><font color="#ff0000"><font data-keep-original-tag="false"><font data-original-attrs="{&quot;style&quot;:&quot;&quot;}">Related work and acknowledgments</font></font></font></h2> <p> </p> <p dir="auto"><a href="https://root.cern.ch/">ROOT</a> is the reference platform for running HEP data analysis, using C++ and also Python bindings. Its current evolution implements the dataframe abstraction, with "RDataframe"  and integrates with Apache Spark and Dask to scale out computations.</p> <p dir="auto"><a href="https://coffeateam.github.io/coffea/">Coffea</a>, <a href="https://awkward-array.readthedocs.io/">Awkward Array</a>, <a href="https://uproot.readthedocs.io/">Uproot</a>, <a href="https://iris-hep.org/projects/servicex">ServiceX</a>, are components of a suite of Python libraries and packages to build a HEP data analysis platform. The platform is integrated with Dask and Apache Spark, Parsl, and Work Queue Executor for scaling out computations.</p> <p dir="auto">The <a href="https://github.com/spark-root/laurelin">Laurelin library</a> integrates with Apache Spark for reading ROOT files (by Andrew Melo). The <a href="https://github.com/cerndb/hadoop-xrootd">Hadoop-XRootD connector</a> integrates with Apache Spark to access "the root:// filesystem" (by the CERN Hadoop and Spark service).</p> <p dir="auto"> </p> <p dir="auto">The work on implementing the HEP benchmark with Apache Spark reported here, stems from:</p> <ul><li>The <a href="https://github.com/iris-hep/adl-benchmarks-index">IRIS-HEP benchmark</a> specifications and solutions linked there.</li> <li> <p>The article and related code: <a href="https://arxiv.org/abs/2104.12615" rel="nofollow">Evaluating Query Languages and Systems for High-Energy Physics Data</a>.</p> </li> </ul><p dir="auto"> </p> <p dir="auto">Previous work on the topic of using Apache Spark for physics, for ML data preparation and data reduction at scale, include:</p> <ul><li><a href="https://rdcu.be/b4Wk9">Machine Learning Pipelines with Modern Big Data Tools for High Energy Physics</a>, Matteo Migliorini, Riccardo Castellotti, Luca Canali, Marco Zanetti, <i>Comput Softw Big Sci</i> <b>4, </b>8 (2020).</li> <li><a href="https://doi.org/10.1051/epjconf/201921406030">Using Big Data Technologies for HEP Analysis</a>, M. Cremonesi <i>et al.</i>, EPJ Web of Conferences 214, 06030 (2019)</li> <li><a href="https://iopscience.iop.org/article/10.1088/1742-6596/1085/4/042030/meta">CMS Analysis and Data Reduction with Apache Spark</a>, O. Gutsche et al. 2018 J. Phys.: Conf. Ser.1085 042030</li> <li><a href="https://ieeexplore.ieee.org/document/8605741">Big Data Tools and Cloud Services for High Energy Physics Analysis in TOTEM Experiment</a>, V. Avati <i>et al.</i>, 2018, Proceeding of: 2018 IEEE/ACM International Conference on Utility and Cloud Computing Companion (UCC Companion)</li> </ul><p> </p> <p dir="auto">Many thanks go to Jim Pivarski, Lindsey Gray, Andrew Melo, Lukas Heinrich, Gordon Watts, Ghislain Fourny, Ingo Müller, for discussions. To Ruslan Dautkhanov and Hyukjin Kwon from Databricks for their support and work with mapInArrow, see <a href="https://issues.apache.org/jira/browse/SPARK-37227">SPARK-37227</a> and <a href="https://issues.apache.org/jira/browse/SPARK-30153">SPARK-30153</a>. To the Hadoop and Spark team and the <a href="https://swan.web.cern.ch/swan/">SWAN</a> (platform for web-based analysis) team at CERN, in particular Riccardo Castellotti.</p> <p dir="auto">This work was done in the context of the Hadoop, Spark, and SWAN services at CERN, and of the data engineering efforts with the ATLAS experiment.</p> <p dir="auto"> </p> </div> </div> <span><a title="View user profile." href="/users/luca-canali" lang="" about="/users/luca-canali" typeof="schema:Person" property="schema:name" datatype="">canali</a></span> <span><time datetime="2022-03-10T15:21:14+01:00" title="Thursday, March 10, 2022 - 15:21">Thu, 03/10/2022 - 15:21</time> </span> <div class="field field--name-field-tags field--type-entity-reference field--label-above"> <div class="field--label"><b>Tags</b></div> <div class="field--items"> <div class="field--item"><a href="/tags/apache-spark" hreflang="en">Apache Spark</a></div> <div class="field--item"><a href="/tags/physics" hreflang="en">Physics</a></div> </div> </div> <section> <h2>Add new comment</h2> <drupal-render-placeholder callback="comment.lazy_builders:renderForm" arguments="0=node&amp;1=186&amp;2=comment_node_blog_post&amp;3=comment_node_blog_post" token="WzqUni3dmjnFHVTPCFRfUg5uspfEtV2l03VsoiWHMM4"></drupal-render-placeholder> </section> Thu, 10 Mar 2022 14:21:14 +0000 canali 186 at https://db-blog.web.cern.ch