Bulk processing of AEMO MMS data with a custom Spark Data Source

Background

AEMO is the Australian Energy Market Operator. It makes available a well organised database for market participants to track bids, demand, generation and other market functions. This database is known as the MMS (Market Management System Data Model).

Electricity researchers, retailers, distributors and others use this data to get insights and manage their business.

Public MMS data is freely available:

The traditional approach to make use of MMS datasets is to load them into an RDBMS. The volume, and variety of data can make this difficult, although some helper tools do exist. However loading a large history of granular data for analysis, even for a particular dataset is also a common business requirement.

Apache Spark (an alternative to traditional RDBMS) has a natural advantage in being able to read and process large datasets in parallel, particularly for analytics.

Can it be used here?

Challenges

The AEMO CSV format used to populate MMS allows there to be multiple reports in a single file.

Furthermore files are frequently compressed in Zip format. This usually means pre-processing is required – e.g. before reading in as text or CSV.

Whilst the underlying files are comma separated, the number of columns in each row also varies in a given file due to:

  • Different record types (Comment, Information or Data)
  • Different report schemas (each having a different column set)
AEMO MMS Data Model CSV structure

Here is a snippet from a sample file:

C,SETP.WORLD,DVD_DISPATCH_UNIT_SCADA,AEMO,PUBLIC,2021/10/07,00:00:05,0000000350353790,,0000000350353744
I,DISPATCH,UNIT_SCADA,1,SETTLEMENTDATE,DUID,SCADAVALUE
D,DISPATCH,UNIT_SCADA,1,"2021/09/01 00:05:00",BARCSF1,0
D,DISPATCH,UNIT_SCADA,1,"2021/09/01 00:05:00",BUTLERSG,9.499998
D,DISPATCH,UNIT_SCADA,1,"2021/09/01 00:05:00",CAPTL_WF,47.048004
...lots more rows...
C,"END OF REPORT",3368947

This file structure presents some specific challenges for parsing with Spark and thus being able to derive useful insights from the underlying data.

Issue #1 – reading too many rows in a file (even for a single report) can cause out of memory issues

Issue #2 – naively reading just the data (D) rows misses file and report header information, such as column names

Issue #3 – parsing full files can result in unnecessary data being read, when only a subset is needed

Solution

SparkMMS is a custom data reader implemented in Java using Apache Spark’s DataSource V2 API.

It can be used to efficiently read AEMO MMS files in bulk.

Input:

SparkMMS takes a glob path, which means it can read multiple files based on a file pattern – e.g. to read all dispatch related zip files from a monthly archive:

/tmp/Data_Archive/Wholesale_Electricity/MMSDM/2021/MMSDM_2021_09/MMSDM_Historical_Data_SQLLoader/DATA/PUBLIC_DVD_DISPATCH*.zip

Output:

Spark MMS creates a Spark dataframe with chunks of rows related to each specific report type across all input files. The data rows are nested in the “data” column of the dataframe. The file header, report headers (including column names) and data rows are also preserved:

>>> df.printSchema()
root
 |-- original_filename: string (nullable = false)
 |-- system: string (nullable = false)
 |-- report_id: string (nullable = false)
 |-- report_from: string (nullable = false)
 |-- report_to: string (nullable = false)
 |-- publish_datetime: timestamp (nullable = false)
 |-- id1: string (nullable = false)
 |-- id2: string (nullable = false)
 |-- id3: string (nullable = false)
 |-- report_type: string (nullable = false)
 |-- report_subtype: string (nullable = false)
 |-- report_version: integer (nullable = false)
 |-- column_headers: array (nullable = false)
 |    |-- element: string (containsNull = true)
 |-- data: array (nullable = false)
 |    |-- element: array (containsNull = true)
 |    |    |-- element: string (containsNull = true)

This structure makes it easy to do further processing of the data and means no information is lost when reading files in parallel:

Other features:

  • Reads both .CSV and .zip
  • Automatically splits large files into multiple partitions
  • Extracts useful metadata from raw files, including column headers
  • Supports multiple report schemas / versions
  • Supports predicate pushdown – skips reports within a file if not selected
  • Column pruning – reads of only a subset of data from raw files, if columns not selected
  • Can read from cloud storage (e.g. Azure Blob storage, Amazon S3, Databricks DBFS)

Demo

These steps show the SparkMMS custom reader in action using Azure Databricks:

Note: Databricks is a paid cloud based Data Lake / ML platform. Alternatively, see source code for a demonstration running Spark MMS locally on a single node.

Prerequisites

  1. Download the library:
    https://github.com/niftimus/SparkMMS/releases/download/v0.1/SparkMMS-0.1-SNAPSHOT.jar
  2. Start a Databricks cluster – e.g.:

    Note: Select Runtime 9.1 LTS for compatibility
  3. Add the SparkMMS library to the cluster via Cluster > Libraries > Install New > Drag and Drop Jar:

Using SparkMMS

1. Define helper functions. At runtime, these create MMS report specific dataframe definitions (with correct per-report column headings) and also create temporary tables to streamline querying via SQL:

# Get a new dataframe with the schema of a single report type
def getReport(df, report_type, report_subtype, report_version):
    from pyspark.sql.functions import explode
    df = df.where(f"report_type = '{report_type}' and report_subtype = '{report_subtype}' and report_version = {report_version}")
    tmpDF = df.select("column_headers", explode(df.data).alias("datarow"))
    
    colHeaders = df.select("column_headers").first().column_headers
    
    for idx, colName in enumerate(colHeaders):
        tmpDF = tmpDF.withColumn(colName, tmpDF.datarow[idx])
    
    tmpDF = tmpDF.drop("column_headers").drop("datarow")    
    
    return tmpDF

# Register all reports available in the dataframe as temporary view in the metastore
def registerAllReports(df=df):
    tmpDF = df.select("report_type","report_subtype","report_version")
    tmpDF = tmpDF.dropDuplicates()
    
    reports = tmpDF.collect()
    
    for r in reports:
        tmpReportDF = getReport(df,r.report_type,r.report_subtype,r.report_version)
        tmpReportDF.createOrReplaceTempView(f"{r.report_type}_{r.report_subtype}_{r.report_version}")

2. Create a temporary directory and download sample data from AEMO (15mb zipped, 191mb unzipped):

%sh
cd /dbfs/
mkdir tmp
cd tmp
wget https://nemweb.com.au/Data_Archive/Wholesale_Electricity/MMSDM/2021/MMSDM_2021_09/MMSDM_Historical_Data_SQLLoader/DATA/PUBLIC_DVD_DISPATCH_UNIT_SCADA_202109010000.zip

Note – there is no need to unzip the file.

3. Read raw data into a Spark dataframe using SparkMMS:

Notes:

  • Option maxRowsPerPartition tells the reader to create each partition with a maximum of 50,000 report data rows. All report rows will be read, however some will be in different partitions for performance reasons.
  • Option minSplitFilesize tells the reader not to bother splitting files smaller than 1,000,000 bytes, which improves performance.
df = (
    spark 
    .read
    .format("com.analyticsanvil.SparkMMS")
    .option("fileName", "/tmp/PUBLIC_DVD_DISPATCH_UNIT_SCADA_202109010000.zip")
    .option("maxRowsPerPartition","50000")
    .option("minSplitFilesize","1000000")
    .load()
)

4. Validate that the dataframe contains rows:

df.show()

Example output:

Note: Optionally here we can also run df.cache() to improve performance in subsequent steps.

5. Register each report found in the raw file(s) as a temporary table and then validate the output:

registerAllReports(df)

After the above command, a single temp table is registered because our file only contained one report:
Report type: DISPATCH
Report sub-type: UNIT_SCADA
Version: 1



Note: If we selected more files in step 2 above we would see more temp tables above.

Now query the temp table and check the data:

6. Finally, we can create a view on top of the temporary table(s) with further calculations or data-type conversions – for example:

%sql
-- Create a temporary view with expected data types
CREATE OR REPLACE TEMPORARY VIEW vw_dispatch_unit_scada_1
AS
SELECT
  to_timestamp(REPLACE(SETTLEMENTDATE,'"',''), 'yyyy/MM/dd HH:mm:ss') AS dispatch_time, -- Strip quote characters from SETTLEMENTDATE and convert to native timestamp type
  DUID AS generator,
  CAST(SCADAVALUE AS DOUBLE) AS generation_MW -- Convert to numeric
FROM dispatch_unit_scada_1;

…and then perform charting, aggregations. For example, charting the average generation in MW for three generation units (Coal, Wind, Solar) in September 2021:

Conclusion

Apache Spark provides a convenient way to process large datasets in parallel once data is available in a structured format.

AEMO’s MMS data model data is vast and varied, so keeping all data loaded in an online platform for eternity can be an expensive option. Occasionally, however, a use case may arise which relies on having a long period of historical data available to query.

SparkMMS demonstrates a convenient way to process raw files in bulk, with no pre-processing or manual schema design. In some organisations, historical files may be available on cloud / local storage, even if data has been archived from an RDBMS. Therefore, custom readers like SparkMMS may be a convenient option to explore for ad-hoc use cases, as an alternative to re-loading old data into a relational database.

Files

References

Leave a comment