Category Archives: All About Software Technology

Scala On Spark – Word-pair Count

So far, the few programming examples in the SoS (Scala on Spark) blog series have all centered around DataFrames. In this blog post, I would like to give an example on Spark’s RDD (resilient distributed data), which is an immutable distributed collection of data that can be processed via functional transformations (e.g. map, filter, reduce).

The main difference between the RDD and DataFrame APIs is that the former provides more granular low-level functionality whereas the latter is equipped with powerful SQL-style functions to process table-form data. Note that even though a DataFrame is in table form with named columns, the underlying JVM only treats each row of the data a generic untyped object. As a side note, Spark also supports another data abstraction called Dataset, which is a distributed collection of strongly-typed objects.

Back to the RDD world. In this programming exercise, our goal is to count the number of occurrences of every distinct pair of consecutive words in a text file. In essence, for every given distinct word in a text file we’re going to count the number of occurrences of all distinct words following the word. As a trivial example, if the text is “I am what I am”, the result should be (i, am) = 2, (what, i) = 1, (am, what) = 1.

For illustration purpose, let’s assemble a small piece of text as follows and save it in a file, say in a Hadoop HDFS file system:

This is line one.
And this is line two.
Is this line three?
This is another line.
And this is yet another line!
Line one and line two are similar.
But line two and line three are not similar!
And line three and line four are not similar.
But line four and line five are similar!

Simple word count

As a warm-up exercise, let’s perform a hello-world word count, which simply reports the count of every distinct word in a text file. Using the ‘textFile()’ method in SparkContext, which serves as the entry point for every program to be able to access resources on a Spark cluster, we load the content from the HDFS file:

// Count occurrences of distinct words
val wordCountRDD = sc.textFile("hdfs://path/to/textfile").
  flatMap( _.split("""[\s,.;:!?]+""") ).
  map( _.toLowerCase ).
  map( (_, 1) ).
  reduceByKey( _ + _ ).
  sortBy( z => (z._2, z._1), ascending = false )

Viewed as a collection of lines (delimited by carriage returns), we first use ‘flatMap’ to split each line of the text by punctuations into an array of words then flatten the arrays. Note that ‘_.split()’ is just a Scala short-hand for ‘line => line.split()’.

Next, all words are lowercased (to disregard cases) with the transformation ‘word => word.toLowerCase’, followed by a map transformation ‘word => (word, 1)’ for tallying. Using ‘reduceByKey’, the reduction transformation ‘(total, count) => total + count’ (short-handed as ‘(_ + _)’) for each key transforms every word into a tuple of (word, totalcount). The final sorting is just for ordering the result by count.

Since the dataset is small, we can ‘collect’ the result data to see the output:

wordCountRDD.collect.foreach{
  case (a, b) => println(f"$a%10s" + "  : " + f"$b%4s")
}

      line  :   13
       and  :    7
      this  :    5
        is  :    5
   similar  :    4
       are  :    4
       two  :    3
     three  :    3
       one  :    2
       not  :    2
      four  :    2
       but  :    2
   another  :    2
       yet  :    1
      five  :    1

On a related note, Spark’s ‘reduceByKey()’ along with a couple of other ‘xxxxByKey()’ functions are handy tools for this kind of key-value pair transformations. Had they not been provided, one would have to do it with a little more hand-crafting work like:

  groupBy( _._1 ).mapValues( _.map(_._2).sum )

  // OR

  foldLeft( Map[String, Int]() )( (acc, x) => 
    acc + (x._1 -> (acc.getOrElse(x._1, 0) + x._2) )
  )

Word-pair count

Now, let’s move onto the main topic of this blog post – counting distinct pairs of consecutive words:

import org.apache.spark.mllib.rdd.RDDFunctions._

// Count occurrences of distinct word pairs
val wordPairCountRDD = sc.textFile("hdfs://path/to/textfile").
  flatMap( _.split("""[\s,.;:!?]+""") ).
  map( _.toLowerCase ).
  sliding(2).
  map{ case Array(x, y) => ((x, y), 1) }.
  reduceByKey( _ + _ ).
  sortBy( z => (z._2, z._1._1, z._1._2), ascending = false )

Even though the required logic for counting word pairs is apparently more complex than that of counting individual words, the necessary transformations look only slightly different. It’s partly due to how compositions of modularized functions can make complex data transformations look seemingly simple in a functional programming language like Scala. Another key factor in this case is the availability of the powerful ‘sliding(n)’ function, which transforms a collection of elements into sliding windows each in the form of an array of size ‘n’. For example, applying sliding(2) to a sequence of words “apples”, “and”, “oranges” would result in Array(“apples”, “and”) and Array(“and”, “oranges”).

Scanning through the compositional functions, the split by punctuations and lowercasing do exactly the same thing as in the hello-world word count case. Next, ‘sliding(2)’ generates sliding window of word pairs each stored in an array. The subsequent ‘map’ each of the word-pair arrays into a key/value tuple with the word-pair-tuple being the key and 1 being the count value.

Similar to the reduction transformation in the hello-world word count case, ‘reduceByKey()’ generates count for each word pair. Result is then sorted by count, 1st word in word-pair, 2nd word in word-pair. Output of the word-pair count using ‘collect’ is as follows:

wordPairCountRDD.collect.foreach{
  case ((a, b), c) => println(f"$a%10s" + "    -> " + f"$b%10s" + "  : " + f"$c%4s")
}

       and    ->       line  :    5
      this    ->         is  :    4
      line    ->        two  :    3
      line    ->      three  :    3
   similar    ->        but  :    2
       one    ->        and  :    2
       not    ->    similar  :    2
      line    ->        one  :    2
      line    ->       four  :    2
        is    ->       line  :    2
       but    ->       line  :    2
       are    ->    similar  :    2
       are    ->        not  :    2
   another    ->       line  :    2
       and    ->       this  :    2
       yet    ->    another  :    1
       two    ->         is  :    1
       two    ->        are  :    1
       two    ->        and  :    1
     three    ->       this  :    1
     three    ->        are  :    1
     three    ->        and  :    1
      this    ->       line  :    1
   similar    ->        and  :    1
      line    ->       line  :    1
      line    ->       five  :    1
      line    ->        and  :    1
        is    ->        yet  :    1
        is    ->       this  :    1
        is    ->    another  :    1
      four    ->        are  :    1
      four    ->        and  :    1
      five    ->        are  :    1

Creating a word-pair count method

The above word-pair counting snippet can be repurposed to serve as a general method for counting a specific word-pair in a text file:

import org.apache.spark.SparkContext
import org.apache.spark.mllib.rdd.RDDFunctions._

def wordPairCount(word1: String, word2: String, filePath: String)(implicit sc: SparkContext) =
  sc.textFile(filePath).
    flatMap( _.split("""[\s,.;:!?]+""") ).
    map( _.toLowerCase ).
    sliding(2).
    collect{ case Array(`word1`, `word2`) => ((word1, word2), 1) }.
    reduceByKey( _ + _ )

It’s worth noting that Scala’s collect method (not to be confused with Spark’s RDD ‘collect’ method) has now replaced method ‘map’ in the previous snippet. It’s because we’re now interested in counting only the specific word-pair word1 and word2, thus requiring the inherent filtering functionality from method ‘collect’. Also note that in the ‘case’ statement the pair of words are enclosed in backticks to refer to the passed-in words, rather than arbitrary pattern-matching variables.

To use the word-pair count method, simply provide the pair of consecutive words and the file path as parameters, along with the SparkContext to be passed in an implicit parameter. For example:

implicit val sc = SparkContext.getOrCreate

wordPairCount("line", "two", "hdfs://path/to/textfile")
// res1: org.apache.spark.rdd.RDD[((String, String), Int)] = ShuffledRDD[56] at reduceByKey at :42

res1.collect
// res2: Array[((String, String), Int)] = Array(((line,two),3))

Scala On Spark – Streak

This is yet another programming example in my Scala-on-Spark blog series. Again, while it starts with the same minuscule weather data used in previous examples of the blog series, it can be viewed as an independent programming exercise.

In this example, we’re going to create a table that shows the streaks of consecutive months with non-zero precipitation.

Result should be similar to the following:

+-------+-------+--------------+------+
|station|year_mo|monthly_precip|streak|
+-------+-------+--------------+------+
|    100|2017-07|           0.5|     1|
|    100|2017-08|           2.0|     2|
|    100|2017-11|           1.5|     1|
|    100|2017-12|           3.5|     2|
|    115|2017-07|           1.0|     1|
|    115|2017-10|           1.5|     1|
|    115|2017-11|           3.0|     2|
|    115|2017-12|           4.5|     3|
+-------+-------+--------------+------+

We’ll explore using Spark’s window functions in this example. As a side note, some of the previous examples in the blog series could be resolved using window functions as well. By means of aggregating over partitioned sliding windows of data, Spark’s window functions readily perform certain kinds of complex aggregations which would otherwise require repetitive nested groupings. They are similar to how PostgreSQL’s window functions work.

Now, let’s load up the same old minuscule weather data.

import java.sql.Date

// DataFrame columns:
//   Weather Station ID
//   Start Date of a half-month period
//   Temperature High (in Fahrenheit) over the period
//   Temperature Low (in Fahrenheit) over the period
//   Total Precipitation (in inches) over the period
val weatherDataDF = Seq(
  (100, Date.valueOf("2017-07-01"), 75, 59, 0.0),
  (100, Date.valueOf("2017-07-16"), 77, 59, 0.5),
  (100, Date.valueOf("2017-08-01"), 80, 63, 1.0),
  (100, Date.valueOf("2017-08-16"), 78, 62, 1.0),
  (100, Date.valueOf("2017-09-01"), 74, 59, 0.0),
  (100, Date.valueOf("2017-09-16"), 72, 57, 0.0),
  (100, Date.valueOf("2017-10-01"), 68, 54, 0.0),
  (100, Date.valueOf("2017-10-16"), 66, 54, 0.0),
  (100, Date.valueOf("2017-11-01"), 64, 50, 0.5),
  (100, Date.valueOf("2017-11-16"), 61, 48, 1.0),
  (100, Date.valueOf("2017-12-01"), 59, 46, 2.0),
  (100, Date.valueOf("2017-12-16"), 57, 45, 1.5),
  (115, Date.valueOf("2017-07-01"), 76, 57, 0.0),
  (115, Date.valueOf("2017-07-16"), 76, 56, 1.0),
  (115, Date.valueOf("2017-08-01"), 78, 57, 0.0),
  (115, Date.valueOf("2017-08-16"), 81, 57, 0.0),
  (115, Date.valueOf("2017-09-01"), 77, 54, 0.0),
  (115, Date.valueOf("2017-09-16"), 72, 50, 0.0),
  (115, Date.valueOf("2017-10-01"), 65, 45, 0.0),
  (115, Date.valueOf("2017-10-16"), 59, 40, 1.5),
  (115, Date.valueOf("2017-11-01"), 55, 37, 1.0),
  (115, Date.valueOf("2017-11-16"), 52, 35, 2.0),
  (115, Date.valueOf("2017-12-01"), 45, 30, 3.0),
  (115, Date.valueOf("2017-12-16"), 41, 28, 1.5)
).toDF("station", "start_date", "temp_high", "temp_low", "total_precip")

First, create a DataFrame of precipitation by weather station and month and filter it to consist of only months with positive precipitation.

import org.apache.spark.sql.functions._
import org.apache.spark.sql.expressions.Window

val positivePrecipDF = weatherDataDF.groupBy($"station", last_day($"start_date").as("mo_end")).
  agg(sum($"total_precip").as("monthly_precip")).
  where($"monthly_precip" > 0)

positivePrecipDF.orderBy($"station", $"mo_end").show
// +-------+----------+--------------+
// |station|    mo_end|monthly_precip|
// +-------+----------+--------------+
// |    100|2017-07-31|           0.5|
// |    100|2017-08-31|           2.0|
// |    100|2017-11-30|           1.5|
// |    100|2017-12-31|           3.5|
// |    115|2017-07-31|           1.0|
// |    115|2017-10-31|           1.5|
// |    115|2017-11-30|           3.0|
// |    115|2017-12-31|           4.5|
// +-------+----------+--------------+

Next, using window function, we capture sequences of row numbers ordered by month over partitions by weather station. For each row, we then use an UDF to calculate the base date by dating back from the corresponding month of the row in accordance with the row number. As shown in the following table, these base dates help trace chunks of contiguous months back to their common base dates.

// UDF to calculate the last day of the month which is n months prior to the input date
def baseDate = udf{ (d: java.sql.Date, n: Long) =>
  val base = d.toLocalDate.plusMonths(-n)
  java.sql.Date.valueOf(base.withDayOfMonth(base.lengthOfMonth))
}

val rankedDF = positivePrecipDF.withColumn("row_num", row_number.over(
  Window.partitionBy($"station").orderBy($"mo_end")
)).withColumn(
  "baseDate", baseDate($"mo_end", $"row_num")
)

rankedDF.show
// +-------+----------+--------------+-------+----------+
// |station|    mo_end|monthly_precip|row_num|  baseDate|
// +-------+----------+--------------+-------+----------+
// |    115|2017-07-31|           1.0|      1|2017-06-30|
// |    115|2017-10-31|           1.5|      2|2017-08-31|
// |    115|2017-11-30|           3.0|      3|2017-08-31|
// |    115|2017-12-31|           4.5|      4|2017-08-31|
// |    100|2017-07-31|           0.5|      1|2017-06-30|
// |    100|2017-08-31|           2.0|      2|2017-06-30|
// |    100|2017-11-30|           1.5|      3|2017-08-31|
// |    100|2017-12-31|           3.5|      4|2017-08-31|
// +-------+----------+--------------+-------+----------+

Finally, we apply another row-number window function, but this time, over partitions by weather station as well as base date. This partitioning allows contiguous common base dates to generate new row numbers as the wanted streaks.

val streakDF = rankedDF.select(
  $"station", $"mo_end", $"monthly_precip", row_number.over(
    Window.partitionBy($"station", $"baseDate").orderBy($"mo_end")
  ).as("streak")
).withColumn(
  "year_mo", concat(year($"mo_end"), lit("-"), lpad(month($"mo_end"), 2, "0"))
).select(
  $"station", $"year_mo", $"monthly_precip", $"streak"
).orderBy($"station", $"mo_end")

streakDF.show
// +-------+-------+--------------+------+
// |station|year_mo|monthly_precip|streak|
// +-------+-------+--------------+------+
// |    100|2017-07|           0.5|     1|
// |    100|2017-08|           2.0|     2|
// |    100|2017-11|           1.5|     1|
// |    100|2017-12|           3.5|     2|
// |    115|2017-07|           1.0|     1|
// |    115|2017-10|           1.5|     1|
// |    115|2017-11|           3.0|     2|
// |    115|2017-12|           4.5|     3|
// +-------+-------+--------------+------+

Using the same logic flow, we can also generate similar streak reports for temperature high/low (e.g. streak of temperature high above 75F). I’ll leave that as exercise for the readers.

Scala On Spark – Sum Over Periods

This is another programming example in my Scala-on-Spark blog series. While it uses the same minuscule weather data created in the first example of the blog series, it can be viewed as an independent programming exercise.

In this example, we want a table of total precipitation over custom past periods by weather stations. The specific periods in this example are the previous month, previous 3 months, and all previous months. We have data from July through December, and let’s say it’s now January hence the previous month is December.

The result should be like this:

+-------+---------------+---------------+---------------+
|station|precip_prev_1mo|precip_prev_3mo|precip_prev_all|
+-------+---------------+---------------+---------------+
|    100|            3.5|            5.0|            7.5|
|    115|            4.5|            9.0|           10.0|
+-------+---------------+---------------+---------------+

User-defined functions (UDF) will be used in this example. Spark’s UDF supplements its API by allowing the vast library of Scala (or any of the other supported languages) functions to be used. That said, a method from Spark’s API should be picked over an UDF of same functionality as the former would likely perform more optimally.

First, let’s load up the said weather data.

import java.sql.Date

// DataFrame columns:
//   Weather Station ID
//   Start Date of a half-month period
//   Temperature High (in Fahrenheit) over the period
//   Temperature Low (in Fahrenheit) over the period
//   Total Precipitation (in inches) over the period
val weatherDataDF = Seq(
  (100, Date.valueOf("2017-07-01"), 75, 59, 0.0),
  (100, Date.valueOf("2017-07-16"), 77, 59, 0.5),
  (100, Date.valueOf("2017-08-01"), 80, 63, 1.0),
  (100, Date.valueOf("2017-08-16"), 78, 62, 1.0),
  (100, Date.valueOf("2017-09-01"), 74, 59, 0.0),
  (100, Date.valueOf("2017-09-16"), 72, 57, 0.0),
  (100, Date.valueOf("2017-10-01"), 68, 54, 0.0),
  (100, Date.valueOf("2017-10-16"), 66, 54, 0.0),
  (100, Date.valueOf("2017-11-01"), 64, 50, 0.5),
  (100, Date.valueOf("2017-11-16"), 61, 48, 1.0),
  (100, Date.valueOf("2017-12-01"), 59, 46, 2.0),
  (100, Date.valueOf("2017-12-16"), 57, 45, 1.5),
  (115, Date.valueOf("2017-07-01"), 76, 57, 0.0),
  (115, Date.valueOf("2017-07-16"), 76, 56, 1.0),
  (115, Date.valueOf("2017-08-01"), 78, 57, 0.0),
  (115, Date.valueOf("2017-08-16"), 81, 57, 0.0),
  (115, Date.valueOf("2017-09-01"), 77, 54, 0.0),
  (115, Date.valueOf("2017-09-16"), 72, 50, 0.0),
  (115, Date.valueOf("2017-10-01"), 65, 45, 0.0),
  (115, Date.valueOf("2017-10-16"), 59, 40, 1.5),
  (115, Date.valueOf("2017-11-01"), 55, 37, 1.0),
  (115, Date.valueOf("2017-11-16"), 52, 35, 2.0),
  (115, Date.valueOf("2017-12-01"), 45, 30, 3.0),
  (115, Date.valueOf("2017-12-16"), 41, 28, 1.5)
).toDF("station", "start_date", "temp_high", "temp_low", "total_precip")

We first create a DataFrame of precipitation by weather station and month, each with the number of months that lag the current month.

import org.apache.spark.sql.functions._

val monthlyPrecipDF = weatherDataDF.groupBy($"station", last_day($"start_date").as("mo_end")).
  agg(sum($"total_precip").as("monthly_precip")).
  withColumn("mo_lag", months_between(last_day(current_date), $"mo_end")).
  orderBy($"station", $"mo_end")

monthlyPrecipDF.show
// +-------+----------+--------------+------+
// |station|    mo_end|monthly_precip|mo_lag|
// +-------+----------+--------------+------+
// |    100|2017-12-31|           3.5|   1.0|
// |    100|2017-11-30|           1.5|   2.0|
// |    100|2017-10-31|           0.0|   3.0|
// |    100|2017-09-30|           0.0|   4.0|
// |    100|2017-08-31|           2.0|   5.0|
// |    100|2017-07-31|           0.5|   6.0|
// |    115|2017-12-31|           4.5|   1.0|
// |    115|2017-11-30|           3.0|   2.0|
// |    115|2017-10-31|           1.5|   3.0|
// |    115|2017-09-30|           0.0|   4.0|
// |    115|2017-08-31|           0.0|   5.0|
// |    115|2017-07-31|           1.0|   6.0|
// +-------+----------+--------------+------+

Next, we combine the list of months-lagged with monthly precipitation by means of a UDF to create a map column. To do that, we use Scala’s zip method within the UDF to create a list of tuples from the two input lists and convert the resulting list into a map.

// UDF to combine 2 array-type columns to map
def arraysToMap = udf(
  (a: Seq[Double], b: Seq[Double]) => (a zip b).toMap
)

val precipMapDF = monthlyPrecipDF.groupBy("station").agg(
  collect_list($"mo_lag").as("mo_lag_list"),
  collect_list($"monthly_precip").as("precip_list")
).select(
  $"station", arraysToMap($"mo_lag_list", $"precip_list").as("mo_precip_map")
)

precipMapDF.show(truncate=false)
// +-------+---------------------------------------------------------------------------+
// |station|mo_precip_map                                                              |
// +-------+---------------------------------------------------------------------------+
// |115    |Map(5.0 -> 0.0, 1.0 -> 4.5, 6.0 -> 1.0, 2.0 -> 3.0, 3.0 -> 1.5, 4.0 -> 0.0)|
// |100    |Map(5.0 -> 2.0, 1.0 -> 3.5, 6.0 -> 0.5, 2.0 -> 1.5, 3.0 -> 0.0, 4.0 -> 0.0)|
// +-------+---------------------------------------------------------------------------+

Note that the map content might look different depending on when it is generated, as the months-lagged is relative to the current month when the application is run.

Using another UDF to sum precipitation counting backward from the previous months based on the number of months lagged, we create the result DataFrame.

// UDF to aggregate map values for keys less than or equal to x (0 for all)
def aggMapValues = udf(
  (m: Map[Double, Double], x: Double) =>
    if (x > 0) m.map{ case (k, v) => if (k <= x) v else 0 }.sum else
      m.map{ case (k, v) => v }.sum
)

val customPrecipDF = precipMapDF.
  withColumn( "precip_prev_1mo", aggMapValues($"mo_precip_map", lit(1)) ).
  withColumn( "precip_prev_3mo", aggMapValues($"mo_precip_map", lit(3)) ).
  withColumn( "precip_prev_all", aggMapValues($"mo_precip_map", lit(0)) ).
  select( $"station", $"precip_prev_1mo", $"precip_prev_3mo", $"precip_prev_all" )

customPrecipDF.show
// +-------+---------------+---------------+---------------+
// |station|precip_prev_1mo|precip_prev_3mo|precip_prev_all|
// +-------+---------------+---------------+---------------+
// |    115|            4.5|            9.0|           10.0|
// |    100|            3.5|            5.0|            7.5|
// +-------+---------------+---------------+---------------+

Again, note that the months-lagged is relative to the current month when the application is executed, hence the months-lagged parameters for the aggMapValues UDF should be adjusted accordingly.

We can use similar approach to come up with a table for temperature high/low over the custom periods. Below are the steps for creating the result table for temperature high.

val monthlyHighDF = weatherDataDF.groupBy($"station", last_day($"start_date").as("mo_end")).
  agg(max($"temp_high").as("monthly_high")).
  withColumn("mo_lag", months_between(last_day(current_date), $"mo_end"))

monthlyHighDF.orderBy($"station", $"mo_end").show
// +-------+----------+------------+------+
// |station|    mo_end|monthly_high|mo_lag|
// +-------+----------+------------+------+
// |    100|2017-07-31|          77|   6.0|
// |    100|2017-08-31|          80|   5.0|
// |    100|2017-09-30|          74|   4.0|
// |    100|2017-10-31|          68|   3.0|
// |    100|2017-11-30|          64|   2.0|
// |    100|2017-12-31|          59|   1.0|
// |    115|2017-07-31|          76|   6.0|
// |    115|2017-08-31|          81|   5.0|
// |    115|2017-09-30|          77|   4.0|
// |    115|2017-10-31|          65|   3.0|
// |    115|2017-11-30|          55|   2.0|
// |    115|2017-12-31|          45|   1.0|
// +-------+----------+------------+------+

import org.apache.spark.sql.types.DoubleType

val tempHighMapDF = monthlyHighDF.groupBy("station").agg(
  collect_list($"mo_lag").as("mo_lag_list"),
  collect_list($"monthly_high".cast(DoubleType)).as("temp_high_list")
).select(
  $"station", arraysToMap($"mo_lag_list", $"temp_high_list").as("mo_high_map")
)

tempHighMapDF.show(truncate=false)

// UDF to aggregate map values for keys less than or equal to x (0 for all)
def aggMapValues = udf(
  (m: Map[Double, Double], x: Double) =>
    if (x > 0) m.map{ case (k, v) => if (k <= x) v else 0 }.max else
      m.map{ case (k, v) => v }.max
)

val customTempHighDF = tempHighMapDF.
  withColumn( "high_prev_1mo", aggMapValues($"mo_high_map", lit(1)) ).
  withColumn( "high_prev_3mo", aggMapValues($"mo_high_map", lit(3)) ).
  withColumn( "high_prev_all", aggMapValues($"mo_high_map", lit(0)) ).
  select( $"station", $"high_prev_1mo", $"high_prev_3mo", $"high_prev_all" )

customTempHighDF.show
// +-------+-------------+-------------+-------------+
// |station|high_prev_1mo|high_prev_3mo|high_prev_all|
// +-------+-------------+-------------+-------------+
// |    115|         45.0|         65.0|         81.0|
// |    100|         59.0|         68.0|         80.0|
// +-------+-------------+-------------+-------------+

I’ll leave creating the temperature low result table as a programming exercise for the readers. Note that rather than calculating temperature high and low separately, one could aggregate both of them together in some of the steps with little code change. For those who are up for a slightly more challenging exercise, both temperature high and low data can in fact be transformed together in every step of the way.