Skip to content

Instantly share code, notes, and snippets.

@vinothchandar
Last active August 31, 2021 17:11
Show Gist options
  • Save vinothchandar/593b19c47bea2406b9a8a9aaed30775a to your computer and use it in GitHub Desktop.
Save vinothchandar/593b19c47bea2406b9a8a9aaed30775a to your computer and use it in GitHub Desktop.
hyperspace - demo

TL;DR :

  • Was exploring if hyperspace can be used an alternative for our record/bloom indexes
  • For the needle-in-a-haystack search i.e a single id out of all the records, hyperspace also seems to be not very effective atm (might not be suprising given the covered indexes recommendations so far).
  • Our old workhorse BLOOM_INDEX still significantly outperforms. But we should really step on the gas for RFC-15 like efforts/RFC-08 to make this much faster

https://microsoft.github.io/hyperspace/docs/ug-quick-start-guide/

~/bin/spark-3.0.0-bin-hadoop2.7/bin/spark-shell   --conf 'spark.serializer=org.apache.spark.serializer.KryoSerializer' --driver-memory 8g --packages com.microsoft.hyperspace:hyperspace-core_2.12:0.1.0

val part100Path = "file:///Volumes/HUDIDATA/input-data/amazon-reviews-100-parts"
val df100 = spark.read.parquet(part100Path)
df100.registerTempTable("amazon_reviews_100_parts")

import com.microsoft.hyperspace._
val hs = new Hyperspace(spark)
import com.microsoft.hyperspace.index._


+--------------+
|     review_id|
+--------------+
|R38YR2K3RQVUT6|
|R1UE9PRDNPVWJN|
|R2T5TIOI92JDOA|
| RY7UKOQOZ1NA9|
| R1LJ65G8LY6L6|
| ROQTM343YUPY5|
|R160R9P9BRK8J6|
| R30ZKF6EPTV76|
|R2Q93ZF9K7BERL|
|R2UG8JB73C003W|
|R1NX7L8FAZFL6T|
| R3RJQHNPYINS1|
| R5Z19IT94F27U|
|R1C1X93D1TPIVY|
|R2AZ4P431BHSXD|
|R1G30L7BW96HH9|
|R2Q05M51VX6P14|
| RL9AZUSVJC16M|
|R119E7G9JQDDO5|
|R36I5SKSR7V0WK|
+--------------+

@vinothchandar
Copy link
Author

vinothchandar commented Jul 20, 2020

Hudi Bloom Index

import org.apache.hudi.QuickstartUtils._
import scala.collection.JavaConversions._
import org.apache.spark.sql.SaveMode._
import org.apache.hudi.DataSourceReadOptions._
import org.apache.hudi.DataSourceWriteOptions._
import org.apache.hudi.config.HoodieWriteConfig._

val tableName = "hudi_trips_cow"
val basePath = "file:///Volumes/HUDIDATA/input-data/amazon-reviews-hudi"
val dataGen = new DataGenerator

df100.write.format("hudi").
  option(PRECOMBINE_FIELD_OPT_KEY, "review_id").
  option(RECORDKEY_FIELD_OPT_KEY, "review_id").
  option(PARTITIONPATH_FIELD_OPT_KEY, "product_category").
  option(TABLE_NAME, "amazon_reviews_hudi").
  option(OPERATION_OPT_KEY,"bulk_insert").
  option("hoodie.bloom.index.filter.type", "DYNAMIC_V0").
  option("hoodie.bulkinsert.shuffle.parallelism", 100).
  option("hoodie.parquet.compression.codec", "snappy").
  mode(Overwrite).
  save(basePath)
val jsc = new org.apache.spark.api.java.JavaSparkContext(spark.sparkContext)
import org.apache.hudi.config._;
import org.apache.hudi.common.model._

val cfg =  HoodieWriteConfig.newBuilder().withPath("file:///Volumes/HUDIDATA/input-data/amazon-reviews-hudi").withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(org.apache.hudi.index.HoodieIndex.IndexType.GLOBAL_BLOOM).build()).build()
val readClient = new org.apache.hudi.client.HoodieReadClient(jsc, cfg)
readClient.checkExists(jsc.parallelize(java.util.Arrays.asList(new HoodieKey("ROQTM343YUPY5", null)), 1)).collect()

Just sample full search using parquet format, takes about 63 seconds

scala> val sparkDF = spark.read.format("parquet").load("file:///Volumes/HUDIDATA/input-data/amazon-reviews-hudi/*/*")
sparkDF: org.apache.spark.sql.DataFrame = [_hoodie_commit_time: string, _hoodie_commit_seqno: string ... 19 more fields]

scala> sparkDF.filter("review_id = 'ROQTM343YUPY5'").show
+-------------------+--------------------+------------------+----------------------+--------------------+-----------+-----------+-------------+----------+--------------+--------------------+-----------+-------------+-----------+----+-----------------+--------------------+--------------------+-----------+----+----------------+
|_hoodie_commit_time|_hoodie_commit_seqno|_hoodie_record_key|_hoodie_partition_path|   _hoodie_file_name|marketplace|customer_id|    review_id|product_id|product_parent|       product_title|star_rating|helpful_votes|total_votes|vine|verified_purchase|     review_headline|         review_body|review_date|year|product_category|
+-------------------+--------------------+------------------+----------------------+--------------------+-----------+-----------+-------------+----------+--------------+--------------------+-----------+-------------+-----------+----+-----------------+--------------------+--------------------+-----------+----+----------------+
|     20200719210452|20200719210452_22...|     ROQTM343YUPY5|                 Books|f05cf503-6f55-4d5...|         US|   29124476|ROQTM343YUPY5|0374528373|     569503661|The Brothers Kara...|          5|            0|          1|   N|                N|Despite all his f...|This is indeed on...| 2015-06-25|2015|           Books|
+-------------------+--------------------+------------------+----------------------+--------------------+-----------+-----------+-------------+----------+--------------+--------------------+-----------+-------------+-----------+----+-----------------+--------------------+--------------------+-----------+----+----------------+


scala> new java.util.Date()
res33: java.util.Date = Mon Jul 20 00:31:23 PDT 2020

scala> hudiDF.filter("review_id = 'ROQTM343YUPY5'").show
+-------------------+--------------------+------------------+----------------------+--------------------+-----------+-----------+-------------+----------+--------------+--------------------+-----------+-------------+-----------+----+-----------------+--------------------+--------------------+-----------+----+----------------+
|_hoodie_commit_time|_hoodie_commit_seqno|_hoodie_record_key|_hoodie_partition_path|   _hoodie_file_name|marketplace|customer_id|    review_id|product_id|product_parent|       product_title|star_rating|helpful_votes|total_votes|vine|verified_purchase|     review_headline|         review_body|review_date|year|product_category|
+-------------------+--------------------+------------------+----------------------+--------------------+-----------+-----------+-------------+----------+--------------+--------------------+-----------+-------------+-----------+----+-----------------+--------------------+--------------------+-----------+----+----------------+
|     20200719210452|20200719210452_22...|     ROQTM343YUPY5|                 Books|f05cf503-6f55-4d5...|         US|   29124476|ROQTM343YUPY5|0374528373|     569503661|The Brothers Kara...|          5|            0|          1|   N|                N|Despite all his f...|This is indeed on...| 2015-06-25|2015|           Books|
+-------------------+--------------------+------------------+----------------------+--------------------+-----------+-----------+-------------+----------+--------------+--------------------+-----------+-------------+-----------+----+-----------------+--------------------+--------------------+-----------+----+----------------+


scala> new java.util.Date()
res35: java.util.Date = Mon Jul 20 00:32:42 PDT 2020

image

Lookup using BLOOM_INDEX, supplying the Books partitionPath

scala> readClient.checkExists(jsc.parallelize(java.util.Arrays.asList(new HoodieKey("ROQTM343YUPY5", "Books")), 1)).collect()
res9: java.util.List[(org.apache.hudi.common.model.HoodieKey, org.apache.hudi.common.util.Option[String])] = [(HoodieKey { recordKey=ROQTM343YUPY5 partitionPath=Books},Option{val=(Books,f05cf503-6f55-4d54-ad7d-e16cfe41aa2c-0)})]

scala>

image

Searching across all the partitions

val allProductCategories = hudiDF.select("product_category").distinct().map(r => r.getString(0)).collect
val keys = allProductCategories.map(c => new HoodieKey("ROQTM343YUPY5", c)).toList
readClient.checkExists(jsc.parallelize(keys, 1)).collect()
val locations = readClient.checkExists(jsc.parallelize(keys, 1)).collect()
scala> locations.foreach(println)
(HoodieKey { recordKey=ROQTM343YUPY5 partitionPath=Lawn_and_Garden},Option{val=null})
(HoodieKey { recordKey=ROQTM343YUPY5 partitionPath=Digital_Music_Purchase},Option{val=null})
(HoodieKey { recordKey=ROQTM343YUPY5 partitionPath=Grocery},Option{val=null})
(HoodieKey { recordKey=ROQTM343YUPY5 partitionPath=Mobile_Apps},Option{val=null})
(HoodieKey { recordKey=ROQTM343YUPY5 partitionPath=Baby},Option{val=null})
(HoodieKey { recordKey=ROQTM343YUPY5 partitionPath=Musical_Instruments},Option{val=null})
(HoodieKey { recordKey=ROQTM343YUPY5 partitionPath=Watches},Option{val=null})
(HoodieKey { recordKey=ROQTM343YUPY5 partitionPath=Video_Games},Option{val=null})
(HoodieKey { recordKey=ROQTM343YUPY5 partitionPath=Books},Option{val=(Books,f05cf503-6f55-4d54-ad7d-e16cfe41aa2c-0)})
(HoodieKey { recordKey=ROQTM343YUPY5 partitionPath=Digital_Ebook_Purchase},Option{val=null})
(HoodieKey { recordKey=ROQTM343YUPY5 partitionPath=Outdoors},Option{val=null})
(HoodieKey { recordKey=ROQTM343YUPY5 partitionPath=Shoes},Option{val=null})
(HoodieKey { recordKey=ROQTM343YUPY5 partitionPath=Health_&_Personal_Care},Option{val=null})
(HoodieKey { recordKey=ROQTM343YUPY5 partitionPath=Gift_Card},Option{val=null})
(HoodieKey { recordKey=ROQTM343YUPY5 partitionPath=Jewelry},Option{val=null})
(HoodieKey { recordKey=ROQTM343YUPY5 partitionPath=PC},Option{val=null})
(HoodieKey { recordKey=ROQTM343YUPY5 partitionPath=Furniture},Option{val=null})
(HoodieKey { recordKey=ROQTM343YUPY5 partitionPath=Beauty},Option{val=null})
(HoodieKey { recordKey=ROQTM343YUPY5 partitionPath=Wireless},Option{val=null})
(HoodieKey { recordKey=ROQTM343YUPY5 partitionPath=Luggage},Option{val=null})
(HoodieKey { recordKey=ROQTM343YUPY5 partitionPath=Toys},Option{val=null})
(HoodieKey { recordKey=ROQTM343YUPY5 partitionPath=Home_Improvement},Option{val=null})
(HoodieKey { recordKey=ROQTM343YUPY5 partitionPath=Major_Appliances},Option{val=null})
(HoodieKey { recordKey=ROQTM343YUPY5 partitionPath=Kitchen},Option{val=null})
(HoodieKey { recordKey=ROQTM343YUPY5 partitionPath=Digital_Software},Option{val=null})
(HoodieKey { recordKey=ROQTM343YUPY5 partitionPath=Apparel},Option{val=null})
(HoodieKey { recordKey=ROQTM343YUPY5 partitionPath=Sports},Option{val=null})
(HoodieKey { recordKey=ROQTM343YUPY5 partitionPath=Tools},Option{val=null})
(HoodieKey { recordKey=ROQTM343YUPY5 partitionPath=Home},Option{val=null})
(HoodieKey { recordKey=ROQTM343YUPY5 partitionPath=Mobile_Electronics},Option{val=null})
(HoodieKey { recordKey=ROQTM343YUPY5 partitionPath=Digital_Video_Games},Option{val=null})
(HoodieKey { recordKey=ROQTM343YUPY5 partitionPath=Home_Entertainment},Option{val=null})
(HoodieKey { recordKey=ROQTM343YUPY5 partitionPath=Pet_Products},Option{val=null})
(HoodieKey { recordKey=ROQTM343YUPY5 partitionPath=Automotive},Option{val=null})
(HoodieKey { recordKey=ROQTM343YUPY5 partitionPath=Digital_Video_Download},Option{val=null})
(HoodieKey { recordKey=ROQTM343YUPY5 partitionPath=Electronics},Option{val=null})
(HoodieKey { recordKey=ROQTM343YUPY5 partitionPath=Personal_Care_Appliances},Option{val=null})
(HoodieKey { recordKey=ROQTM343YUPY5 partitionPath=Office_Products},Option{val=null})
(HoodieKey { recordKey=ROQTM343YUPY5 partitionPath=Music},Option{val=null})
(HoodieKey { recordKey=ROQTM343YUPY5 partitionPath=Software},Option{val=null})
(HoodieKey { recordKey=ROQTM343YUPY5 partitionPath=Camera},Option{val=null})
(HoodieKey { recordKey=ROQTM343YUPY5 partitionPath=Video_DVD},Option{val=null})
(HoodieKey { recordKey=ROQTM343YUPY5 partitionPath=Video},Option{val=null})

It found the one key in approx 26 seconds!

image

20 Seconds of that in reading all the bloom filter/ranges. (here is where RFC-15 is going to rock)

image

and the remaining to check the 1 matched file.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment