From 8a61514db7b163724d5dce15cf469a61bcf2c1a5 Mon Sep 17 00:00:00 2001 From: osopardo1 Date: Tue, 10 May 2022 14:47:55 +0200 Subject: [PATCH 01/76] First naive implementation of convert to qbeast --- .../spark/delta/IndexStatusBuilder.scala | 4 +- .../commands/ConvertToQbeastCommand.scala | 33 ++++++++++ .../PartiallyConvertToQbeastCommand.scala | 28 +++++++++ .../spark/utils/ConvertToQbeastTest.scala | 62 +++++++++++++++++++ 4 files changed, 126 insertions(+), 1 deletion(-) create mode 100644 src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala create mode 100644 src/main/scala/io/qbeast/spark/internal/commands/PartiallyConvertToQbeastCommand.scala create mode 100644 src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala diff --git a/src/main/scala/io/qbeast/spark/delta/IndexStatusBuilder.scala b/src/main/scala/io/qbeast/spark/delta/IndexStatusBuilder.scala index 9f5e7c0fa..899702205 100644 --- a/src/main/scala/io/qbeast/spark/delta/IndexStatusBuilder.scala +++ b/src/main/scala/io/qbeast/spark/delta/IndexStatusBuilder.scala @@ -56,7 +56,9 @@ private[delta] class IndexStatusBuilder( import spark.implicits._ val ndims: Int = rev.transformations.size - revisionFiles + // TODO some files may not include metadata + val filesWithMetadata = revisionFiles.where("tags is not null") + filesWithMetadata .groupBy(TagColumns.cube) .agg( weight(min(TagColumns.maxWeight)).as("maxWeight"), diff --git a/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala b/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala new file mode 100644 index 000000000..a8bbd050e --- /dev/null +++ b/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala @@ -0,0 +1,33 @@ +package io.qbeast.spark.internal.commands + +import org.apache.http.annotation.Experimental +import org.apache.spark.qbeast.config.DEFAULT_CUBE_SIZE +import org.apache.spark.sql.{Row, SparkSession} +import org.apache.spark.sql.execution.command.RunnableCommand + +@Experimental +case class ConvertToQbeastCommand( + path: String, + fileFormat: String, + columnsToIndex: Seq[String], + cubeSize: Int = DEFAULT_CUBE_SIZE) + extends RunnableCommand { + + private val acceptedFormats = Seq("parquet", "delta") + + override def run(sparkSession: SparkSession): Seq[Row] = { + // TODO very basic mechanism for converting to qbeast + val options = + Map("columnsToIndex" -> columnsToIndex.mkString(","), "cubeSize" -> cubeSize.toString) + if (acceptedFormats.contains(fileFormat)) { + val df = sparkSession.read.format(fileFormat).load(path) + df.write.format("qbeast").mode("overwrite").options(options).save(path) + } else { + throw new UnsupportedOperationException(s"Unsupported file format: $fileFormat") + } + + Seq.empty + + } + +} diff --git a/src/main/scala/io/qbeast/spark/internal/commands/PartiallyConvertToQbeastCommand.scala b/src/main/scala/io/qbeast/spark/internal/commands/PartiallyConvertToQbeastCommand.scala new file mode 100644 index 000000000..ed07bc61f --- /dev/null +++ b/src/main/scala/io/qbeast/spark/internal/commands/PartiallyConvertToQbeastCommand.scala @@ -0,0 +1,28 @@ +package io.qbeast.spark.internal.commands + +import jdk.jfr.Experimental +import org.apache.spark.qbeast.config.DEFAULT_CUBE_SIZE +import org.apache.spark.sql.{Row, SparkSession} +import org.apache.spark.sql.execution.command.RunnableCommand + +@Experimental +case class PartiallyConvertToQbeastCommand( + path: String, + fileFormat: String, + columnsToIndex: Seq[String], + cubeSize: Int = DEFAULT_CUBE_SIZE) + extends RunnableCommand { + + override def run(sparkSession: SparkSession): Seq[Row] = { + + fileFormat match { + case "delta" => // + case "parquet" => // + case _ => throw new UnsupportedOperationException(s"Unsupported file format: $fileFormat") + } + // TODO + Seq.empty + + } + +} diff --git a/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala b/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala new file mode 100644 index 000000000..7dfe42b45 --- /dev/null +++ b/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala @@ -0,0 +1,62 @@ +package io.qbeast.spark.utils + +import io.qbeast.spark.QbeastIntegrationTestSpec +import io.qbeast.spark.internal.commands.ConvertToQbeastCommand +import org.apache.spark.sql.AnalysisException + +class ConvertToQbeastTest extends QbeastIntegrationTestSpec { + + "ConvertToQbeast" should "convert a Parquet Table to a Qbeast Table" in withSparkAndTmpDir( + (spark, tmpDir) => { + val df = loadTestData(spark) + df.write.mode("overwrite").parquet(tmpDir) + val columnsToIndex = Seq("user_id", "product_id") + + // Run the command + ConvertToQbeastCommand(tmpDir, "parquet", columnsToIndex).run(spark) + + val indexed = spark.read.format("qbeast").load(tmpDir) + indexed.count shouldBe df.count + + }) + + it should "convert a Delta Table to a Qbeast Table" in withSparkAndTmpDir((spark, tmpDir) => { + val df = loadTestData(spark) + df.write.format("delta").save(tmpDir) + val columnsToIndex = Seq("user_id", "product_id") + + // Run the command + ConvertToQbeastCommand(tmpDir, "delta", columnsToIndex).run(spark) + + val indexed = spark.read.format("qbeast").load(tmpDir) + indexed.count shouldBe df.count + + }) + + it should "throw an error when converting another file format" in withSparkAndTmpDir( + (spark, tmpDir) => { + val df = loadTestData(spark) + df.write.mode("overwrite").json(tmpDir) + val columnsToIndex = Seq("user_id", "product_id") + + // Run the command + an[UnsupportedOperationException] shouldBe thrownBy( + ConvertToQbeastCommand(tmpDir, "json", columnsToIndex) + .run(spark)) + }) + + it should "throw an error when the file format does not match" in withSparkAndTmpDir( + (spark, tmpDir) => { + val df = loadTestData(spark) + // write as json + df.write.mode("overwrite").json(tmpDir) + val columnsToIndex = Seq("user_id", "product_id") + + // Run the command + // read as delta + an[AnalysisException] shouldBe thrownBy( + ConvertToQbeastCommand(tmpDir, "delta", columnsToIndex) + .run(spark)) + + }) +} From 99a823b5050875ee23bbe2f354a8ed3b2af5e52c Mon Sep 17 00:00:00 2001 From: osopardo1 Date: Wed, 11 May 2022 09:32:47 +0200 Subject: [PATCH 02/76] Add headers --- .../spark/internal/commands/ConvertToQbeastCommand.scala | 3 +++ .../internal/commands/PartiallyConvertToQbeastCommand.scala | 3 +++ 2 files changed, 6 insertions(+) diff --git a/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala b/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala index a8bbd050e..3876e71a1 100644 --- a/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala +++ b/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala @@ -1,3 +1,6 @@ +/* + * Copyright 2021 Qbeast Analytics, S.L. + */ package io.qbeast.spark.internal.commands import org.apache.http.annotation.Experimental diff --git a/src/main/scala/io/qbeast/spark/internal/commands/PartiallyConvertToQbeastCommand.scala b/src/main/scala/io/qbeast/spark/internal/commands/PartiallyConvertToQbeastCommand.scala index ed07bc61f..3872aec5b 100644 --- a/src/main/scala/io/qbeast/spark/internal/commands/PartiallyConvertToQbeastCommand.scala +++ b/src/main/scala/io/qbeast/spark/internal/commands/PartiallyConvertToQbeastCommand.scala @@ -1,3 +1,6 @@ +/* + * Copyright 2021 Qbeast Analytics, S.L. + */ package io.qbeast.spark.internal.commands import jdk.jfr.Experimental From feb0754b91a6e93267651e98e774ad78af0a0495 Mon Sep 17 00:00:00 2001 From: osopardo1 Date: Thu, 8 Sep 2022 12:50:16 +0200 Subject: [PATCH 03/76] Change RunnableCommand to LeafRunnableCommand --- .../spark/internal/commands/ConvertToQbeastCommand.scala | 4 ++-- .../internal/commands/PartiallyConvertToQbeastCommand.scala | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala b/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala index 3876e71a1..13bb0ff02 100644 --- a/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala +++ b/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala @@ -6,7 +6,7 @@ package io.qbeast.spark.internal.commands import org.apache.http.annotation.Experimental import org.apache.spark.qbeast.config.DEFAULT_CUBE_SIZE import org.apache.spark.sql.{Row, SparkSession} -import org.apache.spark.sql.execution.command.RunnableCommand +import org.apache.spark.sql.execution.command.{LeafRunnableCommand} @Experimental case class ConvertToQbeastCommand( @@ -14,7 +14,7 @@ case class ConvertToQbeastCommand( fileFormat: String, columnsToIndex: Seq[String], cubeSize: Int = DEFAULT_CUBE_SIZE) - extends RunnableCommand { + extends LeafRunnableCommand { private val acceptedFormats = Seq("parquet", "delta") diff --git a/src/main/scala/io/qbeast/spark/internal/commands/PartiallyConvertToQbeastCommand.scala b/src/main/scala/io/qbeast/spark/internal/commands/PartiallyConvertToQbeastCommand.scala index 3872aec5b..a90b98d9c 100644 --- a/src/main/scala/io/qbeast/spark/internal/commands/PartiallyConvertToQbeastCommand.scala +++ b/src/main/scala/io/qbeast/spark/internal/commands/PartiallyConvertToQbeastCommand.scala @@ -6,7 +6,7 @@ package io.qbeast.spark.internal.commands import jdk.jfr.Experimental import org.apache.spark.qbeast.config.DEFAULT_CUBE_SIZE import org.apache.spark.sql.{Row, SparkSession} -import org.apache.spark.sql.execution.command.RunnableCommand +import org.apache.spark.sql.execution.command.{LeafRunnableCommand} @Experimental case class PartiallyConvertToQbeastCommand( @@ -14,7 +14,7 @@ case class PartiallyConvertToQbeastCommand( fileFormat: String, columnsToIndex: Seq[String], cubeSize: Int = DEFAULT_CUBE_SIZE) - extends RunnableCommand { + extends LeafRunnableCommand { override def run(sparkSession: SparkSession): Seq[Row] = { From 5d4e0bca36bbb39b552e45705ac25abc978484ee Mon Sep 17 00:00:00 2001 From: Jiawei Date: Fri, 9 Sep 2022 17:55:12 +0200 Subject: [PATCH 04/76] Access deltaLog/snapshot and modify log --- .../commands/ConvertToQbeastCommand.scala | 69 ++++++++++++++++--- 1 file changed, 60 insertions(+), 9 deletions(-) diff --git a/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala b/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala index 13bb0ff02..d378de4dd 100644 --- a/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala +++ b/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala @@ -3,10 +3,16 @@ */ package io.qbeast.spark.internal.commands +import io.qbeast.core.model.{BroadcastedTableChanges, CubeId, QTableID, Weight} +import io.qbeast.spark.delta.SparkDeltaMetadataManager +import io.qbeast.spark.index.SparkRevisionFactory +import io.qbeast.spark.utils.{State, TagUtils} import org.apache.http.annotation.Experimental import org.apache.spark.qbeast.config.DEFAULT_CUBE_SIZE +import org.apache.spark.sql.delta.DeltaLog import org.apache.spark.sql.{Row, SparkSession} -import org.apache.spark.sql.execution.command.{LeafRunnableCommand} +import org.apache.spark.sql.execution.command.LeafRunnableCommand +import scala.util.matching.Regex @Experimental case class ConvertToQbeastCommand( @@ -16,21 +22,66 @@ case class ConvertToQbeastCommand( cubeSize: Int = DEFAULT_CUBE_SIZE) extends LeafRunnableCommand { - private val acceptedFormats = Seq("parquet", "delta") + private val parquetFormat = "parquet" + private val deltaFormat = "delta" + private val acceptedFormats = Seq(parquetFormat, deltaFormat) + private val numRecordsPattern: Regex = """"numRecords":(\d+),""".r + + def convertParquetToDelta(path: String): Unit = {} override def run(sparkSession: SparkSession): Seq[Row] = { // TODO very basic mechanism for converting to qbeast + assert(acceptedFormats.contains(fileFormat), s"Format $fileFormat not supported.") + + if (fileFormat == parquetFormat) convertParquetToDelta(path) + val options = Map("columnsToIndex" -> columnsToIndex.mkString(","), "cubeSize" -> cubeSize.toString) - if (acceptedFormats.contains(fileFormat)) { - val df = sparkSession.read.format(fileFormat).load(path) - df.write.format("qbeast").mode("overwrite").options(options).save(path) - } else { - throw new UnsupportedOperationException(s"Unsupported file format: $fileFormat") - } - Seq.empty + val deltaLog = DeltaLog.forTable(sparkSession, path) + val snapshot = deltaLog.snapshot + val deltaFiles = snapshot.allFiles.collect() + val revision = + SparkRevisionFactory.createNewRevision(QTableID(path), snapshot.schema, options) + + // Assume all files are AddFiles + val newFiles = deltaFiles + .map(file => { + val elementCount = numRecordsPattern.findFirstMatchIn(file.stats) match { + case Some(matching) => matching.group(1) + case _ => "0" + } + println(elementCount) + file.copy(tags = Map( + TagUtils.cube -> "", + TagUtils.minWeight -> Weight.MinValue.value.toString, + TagUtils.maxWeight -> Weight.MaxValue.value.toString, + TagUtils.state -> State.FLOODED, + TagUtils.revision -> revision.revisionID.toString, + TagUtils.elementCount -> elementCount)) + }) + .toIndexedSeq + + SparkDeltaMetadataManager.updateWithTransaction( + revision.tableID, + snapshot.schema, + append = false) { + val tableChanges = BroadcastedTableChanges( + isNewRevision = false, + isOptimizeOperation = false, + revision, + Set.empty[CubeId], + Set.empty[CubeId], + SparkSession.active.sparkContext.broadcast( + Map(revision.createCubeIdRoot() -> State.FLOODED)), + SparkSession.active.sparkContext.broadcast( + Map(revision.createCubeIdRoot -> Weight.MaxValue))) + + (tableChanges, newFiles) + } +// throw new UnsupportedOperationException(s"Unsupported file format: $fileFormat") + Seq.empty } } From 51c5bb6e7834010140b6ecd78812c42869b2d1ed Mon Sep 17 00:00:00 2001 From: Jiawei Date: Mon, 12 Sep 2022 14:06:45 +0200 Subject: [PATCH 05/76] Add transformers to revision --- .../commands/ConvertToQbeastCommand.scala | 48 ++++++++++++------- 1 file changed, 32 insertions(+), 16 deletions(-) diff --git a/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala b/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala index d378de4dd..bc65bd30b 100644 --- a/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala +++ b/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala @@ -3,15 +3,19 @@ */ package io.qbeast.spark.internal.commands -import io.qbeast.core.model.{BroadcastedTableChanges, CubeId, QTableID, Weight} +import io.qbeast.IISeq +import io.qbeast.core.model._ +import io.qbeast.core.transform._ import io.qbeast.spark.delta.SparkDeltaMetadataManager import io.qbeast.spark.index.SparkRevisionFactory import io.qbeast.spark.utils.{State, TagUtils} import org.apache.http.annotation.Experimental import org.apache.spark.qbeast.config.DEFAULT_CUBE_SIZE import org.apache.spark.sql.delta.DeltaLog -import org.apache.spark.sql.{Row, SparkSession} import org.apache.spark.sql.execution.command.LeafRunnableCommand +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.{Row, SparkSession} + import scala.util.matching.Regex @Experimental @@ -29,37 +33,49 @@ case class ConvertToQbeastCommand( def convertParquetToDelta(path: String): Unit = {} + def initializeRevision(path: String, schema: StructType): Revision = { + val revision = + SparkRevisionFactory.createNewRevision( + QTableID(path), + schema, + Map("columnsToIndex" -> columnsToIndex.mkString(","), "cubeSize" -> cubeSize.toString)) + + val transformations: IISeq[Transformation] = revision.columnTransformers.map { + case LinearTransformer(_, dataType: OrderedDataType) => + LinearTransformation(Int.MinValue, Int.MaxValue, dataType) + case HashTransformer(_, _) => HashTransformation() + } + + revision.copy(transformations = transformations) + } + override def run(sparkSession: SparkSession): Seq[Row] = { // TODO very basic mechanism for converting to qbeast assert(acceptedFormats.contains(fileFormat), s"Format $fileFormat not supported.") if (fileFormat == parquetFormat) convertParquetToDelta(path) - val options = - Map("columnsToIndex" -> columnsToIndex.mkString(","), "cubeSize" -> cubeSize.toString) - val deltaLog = DeltaLog.forTable(sparkSession, path) val snapshot = deltaLog.snapshot val deltaFiles = snapshot.allFiles.collect() - val revision = - SparkRevisionFactory.createNewRevision(QTableID(path), snapshot.schema, options) + val revision = initializeRevision(path, snapshot.schema) - // Assume all files are AddFiles val newFiles = deltaFiles .map(file => { val elementCount = numRecordsPattern.findFirstMatchIn(file.stats) match { case Some(matching) => matching.group(1) case _ => "0" } - println(elementCount) - file.copy(tags = Map( - TagUtils.cube -> "", - TagUtils.minWeight -> Weight.MinValue.value.toString, - TagUtils.maxWeight -> Weight.MaxValue.value.toString, - TagUtils.state -> State.FLOODED, - TagUtils.revision -> revision.revisionID.toString, - TagUtils.elementCount -> elementCount)) + file.copy( + dataChange = true, + tags = Map( + TagUtils.cube -> "", + TagUtils.minWeight -> Weight.MinValue.value.toString, + TagUtils.maxWeight -> Weight.MaxValue.value.toString, + TagUtils.state -> State.FLOODED, + TagUtils.revision -> revision.revisionID.toString, + TagUtils.elementCount -> elementCount)) }) .toIndexedSeq From 9a2236bef470415bd09c87e30eaab7f3c525efe3 Mon Sep 17 00:00:00 2001 From: Jiawei Date: Mon, 12 Sep 2022 14:07:25 +0200 Subject: [PATCH 06/76] Converted tables should be readable using both formates --- .../spark/utils/ConvertToQbeastTest.scala | 48 ++++++++++++++++++- 1 file changed, 47 insertions(+), 1 deletion(-) diff --git a/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala b/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala index 7dfe42b45..b73148698 100644 --- a/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala +++ b/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala @@ -2,10 +2,27 @@ package io.qbeast.spark.utils import io.qbeast.spark.QbeastIntegrationTestSpec import io.qbeast.spark.internal.commands.ConvertToQbeastCommand -import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.{AnalysisException, SparkSession} +import org.apache.spark.sql.delta.DeltaLog class ConvertToQbeastTest extends QbeastIntegrationTestSpec { +// def addCubeTag: UserDefinedFunction = { +// udf((elementCount: String) => +// Map( +// "state" -> "FLOODED", +// "cube" -> "", +// "revision" -> 0, +// "minWeight" -> Int.MinValue.toString, +// "maxWeight" -> Int.MaxValue.toString, +// "elementCount" -> elementCount)) +// } +// + def showFileLog(spark: SparkSession, path: String): Unit = { + val snapshot = DeltaLog.forTable(spark, path).snapshot + snapshot.allFiles.show(false) + } + "ConvertToQbeast" should "convert a Parquet Table to a Qbeast Table" in withSparkAndTmpDir( (spark, tmpDir) => { val df = loadTestData(spark) @@ -59,4 +76,33 @@ class ConvertToQbeastTest extends QbeastIntegrationTestSpec { .run(spark)) }) + +// "A converted delta table" should "be readable using delta" in withSparkAndTmpDir( +// (spark, tmpDir) => { +// val df = loadTestData(spark) +// df.write.format("delta").save(tmpDir) +// showFileLog(spark, tmpDir) +// +// val fileFormat = "delta" +// val columnsToIndex = Seq("user_id", "price", "category_id") +// ConvertToQbeastCommand(tmpDir, fileFormat, columnsToIndex, 500000).run(spark) +// showFileLog(spark, tmpDir) +// +// spark.read.format("delta").load(tmpDir).count shouldBe df.count +// }) +// +// "A converted delta table" should "be readable using qbeast" in withSparkAndTmpDir( +// (spark, tmpDir) => { +// val df = loadTestData(spark) +// df.write.format("delta").save(tmpDir) +// showFileLog(spark, tmpDir) +// +// val fileFormat = "delta" +// val columnsToIndex = Seq("user_id", "price", "category_id") +// ConvertToQbeastCommand(tmpDir, fileFormat, columnsToIndex, 500000).run(spark) +// showFileLog(spark, tmpDir) +// +// spark.read.format("qbeast").load(tmpDir).count shouldBe df.count +// }) + } From 76219b37a706221ea0bc379506b8e21176ecbfbd Mon Sep 17 00:00:00 2001 From: Jiawei Date: Tue, 20 Sep 2022 12:38:18 +0200 Subject: [PATCH 07/76] Add AddFiles with qbeast metadata using append write mode --- .../commands/ConvertToQbeastCommand.scala | 79 +++++++++++-------- 1 file changed, 47 insertions(+), 32 deletions(-) diff --git a/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala b/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala index bc65bd30b..22c15788d 100644 --- a/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala +++ b/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala @@ -11,10 +11,10 @@ import io.qbeast.spark.index.SparkRevisionFactory import io.qbeast.spark.utils.{State, TagUtils} import org.apache.http.annotation.Experimental import org.apache.spark.qbeast.config.DEFAULT_CUBE_SIZE +import org.apache.spark.sql.{Row, SparkSession} import org.apache.spark.sql.delta.DeltaLog import org.apache.spark.sql.execution.command.LeafRunnableCommand import org.apache.spark.sql.types.StructType -import org.apache.spark.sql.{Row, SparkSession} import scala.util.matching.Regex @@ -31,7 +31,22 @@ case class ConvertToQbeastCommand( private val acceptedFormats = Seq(parquetFormat, deltaFormat) private val numRecordsPattern: Regex = """"numRecords":(\d+),""".r - def convertParquetToDelta(path: String): Unit = {} + private val intMinMax = ColumnMinMax(-1e8.toInt, 1e8.toInt) + private val doubleMinMax = ColumnMinMax(-1e10, 1e10) + private val longMinMax = ColumnMinMax(-1e15.toLong, 1e15.toLong) + + private val dataTypeMinMax = Map( + DoubleDataType -> doubleMinMax, + IntegerDataType -> intMinMax, + LongDataType -> longMinMax, + FloatDataType -> doubleMinMax, + DecimalDataType -> doubleMinMax, + TimestampDataType -> longMinMax, + DateDataType -> longMinMax) + + def convertParquetToDelta(path: String): Unit = { + // TODO Convert parquet files to delta + } def initializeRevision(path: String, schema: StructType): Revision = { val revision = @@ -42,7 +57,8 @@ case class ConvertToQbeastCommand( val transformations: IISeq[Transformation] = revision.columnTransformers.map { case LinearTransformer(_, dataType: OrderedDataType) => - LinearTransformation(Int.MinValue, Int.MaxValue, dataType) + val minMax = dataTypeMinMax(dataType) + LinearTransformation(minMax.minValue, minMax.maxValue, dataType) case HashTransformer(_, _) => HashTransformation() } @@ -53,51 +69,50 @@ case class ConvertToQbeastCommand( // TODO very basic mechanism for converting to qbeast assert(acceptedFormats.contains(fileFormat), s"Format $fileFormat not supported.") + // Convert parquet to delta if (fileFormat == parquetFormat) convertParquetToDelta(path) - val deltaLog = DeltaLog.forTable(sparkSession, path) - val snapshot = deltaLog.snapshot - val deltaFiles = snapshot.allFiles.collect() - + // Convert delta to qbeast + val snapshot = DeltaLog.forTable(sparkSession, path).snapshot val revision = initializeRevision(path, snapshot.schema) - - val newFiles = deltaFiles - .map(file => { - val elementCount = numRecordsPattern.findFirstMatchIn(file.stats) match { - case Some(matching) => matching.group(1) - case _ => "0" - } - file.copy( - dataChange = true, - tags = Map( - TagUtils.cube -> "", - TagUtils.minWeight -> Weight.MinValue.value.toString, - TagUtils.maxWeight -> Weight.MaxValue.value.toString, - TagUtils.state -> State.FLOODED, - TagUtils.revision -> revision.revisionID.toString, - TagUtils.elementCount -> elementCount)) - }) - .toIndexedSeq + val root = revision.createCubeIdRoot() + val allFiles = snapshot.allFiles.collect() SparkDeltaMetadataManager.updateWithTransaction( revision.tableID, snapshot.schema, - append = false) { + append = true) { val tableChanges = BroadcastedTableChanges( - isNewRevision = false, + isNewRevision = true, isOptimizeOperation = false, revision, Set.empty[CubeId], Set.empty[CubeId], - SparkSession.active.sparkContext.broadcast( - Map(revision.createCubeIdRoot() -> State.FLOODED)), - SparkSession.active.sparkContext.broadcast( - Map(revision.createCubeIdRoot -> Weight.MaxValue))) - + SparkSession.active.sparkContext.broadcast(Map(root -> State.FLOODED)), + SparkSession.active.sparkContext.broadcast(Map(root -> Weight.MaxValue))) + + val newFiles = allFiles + .map(addFile => { + val elementCount = numRecordsPattern.findFirstMatchIn(addFile.stats) match { + case Some(matching) => matching.group(1) + case _ => "0" + } + addFile.copy( + modificationTime = System.currentTimeMillis(), + tags = Map( + TagUtils.cube -> root.string, + TagUtils.minWeight -> Weight.MinValue.value.toString, + TagUtils.maxWeight -> Weight.MaxValue.value.toString, + TagUtils.state -> State.FLOODED, + TagUtils.revision -> revision.revisionID.toString, + TagUtils.elementCount -> elementCount)) + }) + .toIndexedSeq (tableChanges, newFiles) } // throw new UnsupportedOperationException(s"Unsupported file format: $fileFormat") Seq.empty } + case class ColumnMinMax(minValue: Any, maxValue: Any) {} } From d3d472d684539c47f43c2044ef496196e3311943 Mon Sep 17 00:00:00 2001 From: Jiawei Date: Tue, 20 Sep 2022 12:47:56 +0200 Subject: [PATCH 08/76] Check input format --- .../spark/internal/commands/ConvertToQbeastCommand.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala b/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala index 22c15788d..878ee1e4b 100644 --- a/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala +++ b/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala @@ -67,7 +67,9 @@ case class ConvertToQbeastCommand( override def run(sparkSession: SparkSession): Seq[Row] = { // TODO very basic mechanism for converting to qbeast - assert(acceptedFormats.contains(fileFormat), s"Format $fileFormat not supported.") + if (!acceptedFormats.contains(fileFormat)) { + throw new UnsupportedOperationException(s"Unsupported file format: $fileFormat") + } // Convert parquet to delta if (fileFormat == parquetFormat) convertParquetToDelta(path) @@ -110,7 +112,6 @@ case class ConvertToQbeastCommand( .toIndexedSeq (tableChanges, newFiles) } -// throw new UnsupportedOperationException(s"Unsupported file format: $fileFormat") Seq.empty } From 8f5ffe271d0c181c143c80ceeaf68eee0f5f4c92 Mon Sep 17 00:00:00 2001 From: Jiawei Date: Tue, 20 Sep 2022 12:48:08 +0200 Subject: [PATCH 09/76] Simplify test --- .../spark/utils/ConvertToQbeastTest.scala | 125 +++++++----------- 1 file changed, 46 insertions(+), 79 deletions(-) diff --git a/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala b/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala index b73148698..0ffb3964c 100644 --- a/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala +++ b/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala @@ -2,107 +2,74 @@ package io.qbeast.spark.utils import io.qbeast.spark.QbeastIntegrationTestSpec import io.qbeast.spark.internal.commands.ConvertToQbeastCommand -import org.apache.spark.sql.{AnalysisException, SparkSession} +import org.apache.spark.sql.SparkSession import org.apache.spark.sql.delta.DeltaLog class ConvertToQbeastTest extends QbeastIntegrationTestSpec { + val columnsToIndex: Seq[String] = Seq("user_id", "price", "category_id") -// def addCubeTag: UserDefinedFunction = { -// udf((elementCount: String) => -// Map( -// "state" -> "FLOODED", -// "cube" -> "", -// "revision" -> 0, -// "minWeight" -> Int.MinValue.toString, -// "maxWeight" -> Int.MaxValue.toString, -// "elementCount" -> elementCount)) -// } -// - def showFileLog(spark: SparkSession, path: String): Unit = { + def showFileLog(spark: SparkSession, path: String, truncate: Boolean = true): Unit = { + // scalastyle:off println val snapshot = DeltaLog.forTable(spark, path).snapshot - snapshot.allFiles.show(false) - } - - "ConvertToQbeast" should "convert a Parquet Table to a Qbeast Table" in withSparkAndTmpDir( - (spark, tmpDir) => { - val df = loadTestData(spark) - df.write.mode("overwrite").parquet(tmpDir) - val columnsToIndex = Seq("user_id", "product_id") - // Run the command - ConvertToQbeastCommand(tmpDir, "parquet", columnsToIndex).run(spark) - - val indexed = spark.read.format("qbeast").load(tmpDir) - indexed.count shouldBe df.count - - }) - - it should "convert a Delta Table to a Qbeast Table" in withSparkAndTmpDir((spark, tmpDir) => { - val df = loadTestData(spark) - df.write.format("delta").save(tmpDir) - val columnsToIndex = Seq("user_id", "product_id") + println("AddFiles:") + snapshot.allFiles.show(truncate) - // Run the command - ConvertToQbeastCommand(tmpDir, "delta", columnsToIndex).run(spark) + println("RemoveFiles:") + snapshot.tombstones.show(truncate) + } - val indexed = spark.read.format("qbeast").load(tmpDir) - indexed.count shouldBe df.count + "ConvertToQbeast" should "convert a Parquet Table to a Qbeast Table" in withSparkAndTmpDir( + (spark, tmpDir) => {}) - }) + it should "convert a Delta Table to a Qbeast Table" in withSparkAndTmpDir((spark, tmpDir) => {}) - it should "throw an error when converting another file format" in withSparkAndTmpDir( + it should "throw an error when attempting to convert to unsupported format" in withSparkAndTmpDir( (spark, tmpDir) => { val df = loadTestData(spark) df.write.mode("overwrite").json(tmpDir) - val columnsToIndex = Seq("user_id", "product_id") - // Run the command an[UnsupportedOperationException] shouldBe thrownBy( - ConvertToQbeastCommand(tmpDir, "json", columnsToIndex) - .run(spark)) - }) - - it should "throw an error when the file format does not match" in withSparkAndTmpDir( - (spark, tmpDir) => { - val df = loadTestData(spark) - // write as json - df.write.mode("overwrite").json(tmpDir) - val columnsToIndex = Seq("user_id", "product_id") - - // Run the command - // read as delta - an[AnalysisException] shouldBe thrownBy( - ConvertToQbeastCommand(tmpDir, "delta", columnsToIndex) - .run(spark)) - + ConvertToQbeastCommand(tmpDir, "json", columnsToIndex).run(spark)) }) -// "A converted delta table" should "be readable using delta" in withSparkAndTmpDir( +// it should "throw an error when the file format does not match" in withSparkAndTmpDir( // (spark, tmpDir) => { // val df = loadTestData(spark) -// df.write.format("delta").save(tmpDir) -// showFileLog(spark, tmpDir) +// // write as json +// df.write.mode("overwrite").json(tmpDir) // -// val fileFormat = "delta" -// val columnsToIndex = Seq("user_id", "price", "category_id") -// ConvertToQbeastCommand(tmpDir, fileFormat, columnsToIndex, 500000).run(spark) -// showFileLog(spark, tmpDir) +// // Run the command +// // read as delta +// an[AnalysisException] shouldBe thrownBy( +// ConvertToQbeastCommand(tmpDir, "delta", columnsToIndex) +// .run(spark)) // -// spark.read.format("delta").load(tmpDir).count shouldBe df.count // }) -// -// "A converted delta table" should "be readable using qbeast" in withSparkAndTmpDir( -// (spark, tmpDir) => { -// val df = loadTestData(spark) -// df.write.format("delta").save(tmpDir) -// showFileLog(spark, tmpDir) -// -// val fileFormat = "delta" -// val columnsToIndex = Seq("user_id", "price", "category_id") -// ConvertToQbeastCommand(tmpDir, fileFormat, columnsToIndex, 500000).run(spark) + + "A converted delta table" should "be readable using delta" in withSparkAndTmpDir( + (spark, tmpDir) => { + val df = loadTestData(spark) + df.write.format("delta").save(tmpDir) // showFileLog(spark, tmpDir) -// -// spark.read.format("qbeast").load(tmpDir).count shouldBe df.count -// }) + + val fileFormat = "delta" + ConvertToQbeastCommand(tmpDir, fileFormat, columnsToIndex).run(spark) +// showFileLog(spark, tmpDir, truncate = false) + + spark.read.format("delta").load(tmpDir).count shouldBe df.count + }) + + it should "be readable using qbeast" in withSparkAndTmpDir((spark, tmpDir) => { + val df = loadTestData(spark) + val fileFormat = "delta" + df.write.format(fileFormat).save(tmpDir) +// showFileLog(spark, tmpDir) + + ConvertToQbeastCommand(tmpDir, fileFormat, columnsToIndex).run(spark) +// showFileLog(spark, tmpDir, truncate = false) + + spark.read.format("qbeast").load(tmpDir).count shouldBe df.count + }) } From ac90d08d71dca0d8283f0f84838feb1d021cfe26 Mon Sep 17 00:00:00 2001 From: Jiawei Date: Tue, 20 Sep 2022 12:56:30 +0200 Subject: [PATCH 10/76] Refactor code --- .../commands/ConvertToQbeastCommand.scala | 70 +++++++++++-------- 1 file changed, 41 insertions(+), 29 deletions(-) diff --git a/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala b/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala index 878ee1e4b..e295ee326 100644 --- a/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala +++ b/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala @@ -11,8 +11,9 @@ import io.qbeast.spark.index.SparkRevisionFactory import io.qbeast.spark.utils.{State, TagUtils} import org.apache.http.annotation.Experimental import org.apache.spark.qbeast.config.DEFAULT_CUBE_SIZE +import org.apache.spark.sql.delta.actions.FileAction import org.apache.spark.sql.{Row, SparkSession} -import org.apache.spark.sql.delta.DeltaLog +import org.apache.spark.sql.delta.{DeltaLog, Snapshot} import org.apache.spark.sql.execution.command.LeafRunnableCommand import org.apache.spark.sql.types.StructType @@ -65,6 +66,42 @@ case class ConvertToQbeastCommand( revision.copy(transformations = transformations) } + private def createQbeastActions(snapshot: Snapshot, revision: Revision): IISeq[FileAction] = { + val root = revision.createCubeIdRoot() + val allFiles = snapshot.allFiles.collect() + + allFiles + .map(addFile => { + val elementCount = numRecordsPattern.findFirstMatchIn(addFile.stats) match { + case Some(matching) => matching.group(1) + case _ => "0" + } + addFile.copy( + modificationTime = System.currentTimeMillis(), + tags = Map( + TagUtils.cube -> root.string, + TagUtils.minWeight -> Weight.MinValue.value.toString, + TagUtils.maxWeight -> Weight.MaxValue.value.toString, + TagUtils.state -> State.FLOODED, + TagUtils.revision -> revision.revisionID.toString, + TagUtils.elementCount -> elementCount)) + }) + .toIndexedSeq + } + + private def getTableChanges(revision: Revision): TableChanges = { + val root = revision.createCubeIdRoot() + + BroadcastedTableChanges( + isNewRevision = true, + isOptimizeOperation = false, + revision, + Set.empty[CubeId], + Set.empty[CubeId], + SparkSession.active.sparkContext.broadcast(Map(root -> State.FLOODED)), + SparkSession.active.sparkContext.broadcast(Map(root -> Weight.MaxValue))) + } + override def run(sparkSession: SparkSession): Seq[Row] = { // TODO very basic mechanism for converting to qbeast if (!acceptedFormats.contains(fileFormat)) { @@ -77,39 +114,14 @@ case class ConvertToQbeastCommand( // Convert delta to qbeast val snapshot = DeltaLog.forTable(sparkSession, path).snapshot val revision = initializeRevision(path, snapshot.schema) - val root = revision.createCubeIdRoot() - val allFiles = snapshot.allFiles.collect() SparkDeltaMetadataManager.updateWithTransaction( revision.tableID, snapshot.schema, append = true) { - val tableChanges = BroadcastedTableChanges( - isNewRevision = true, - isOptimizeOperation = false, - revision, - Set.empty[CubeId], - Set.empty[CubeId], - SparkSession.active.sparkContext.broadcast(Map(root -> State.FLOODED)), - SparkSession.active.sparkContext.broadcast(Map(root -> Weight.MaxValue))) - - val newFiles = allFiles - .map(addFile => { - val elementCount = numRecordsPattern.findFirstMatchIn(addFile.stats) match { - case Some(matching) => matching.group(1) - case _ => "0" - } - addFile.copy( - modificationTime = System.currentTimeMillis(), - tags = Map( - TagUtils.cube -> root.string, - TagUtils.minWeight -> Weight.MinValue.value.toString, - TagUtils.maxWeight -> Weight.MaxValue.value.toString, - TagUtils.state -> State.FLOODED, - TagUtils.revision -> revision.revisionID.toString, - TagUtils.elementCount -> elementCount)) - }) - .toIndexedSeq + val tableChanges = getTableChanges(revision) + val newFiles = createQbeastActions(snapshot, revision) + (tableChanges, newFiles) } Seq.empty From b4d1466d56e325abceb9022d40be2b0225521cd2 Mon Sep 17 00:00:00 2001 From: Jiawei Date: Tue, 20 Sep 2022 14:58:47 +0200 Subject: [PATCH 11/76] Add method for parquet to delta conversion --- .../spark/internal/commands/ConvertToQbeastCommand.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala b/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala index e295ee326..0adc2c1ea 100644 --- a/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala +++ b/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala @@ -45,8 +45,8 @@ case class ConvertToQbeastCommand( TimestampDataType -> longMinMax, DateDataType -> longMinMax) - def convertParquetToDelta(path: String): Unit = { - // TODO Convert parquet files to delta + private def convertParquetToDelta(spark: SparkSession, path: String): Unit = { + spark.sql(s"CONVERT TO DELTA $parquetFormat.`$path`") } def initializeRevision(path: String, schema: StructType): Revision = { @@ -109,7 +109,7 @@ case class ConvertToQbeastCommand( } // Convert parquet to delta - if (fileFormat == parquetFormat) convertParquetToDelta(path) + if (fileFormat == parquetFormat) convertParquetToDelta(sparkSession, path) // Convert delta to qbeast val snapshot = DeltaLog.forTable(sparkSession, path).snapshot From 471cb06a7ac93e73cc5d7a060e0d7bd3f86309f3 Mon Sep 17 00:00:00 2001 From: Jiawei Date: Tue, 20 Sep 2022 18:07:47 +0200 Subject: [PATCH 12/76] Reformat tests --- .../spark/utils/ConvertToQbeastTest.scala | 83 ++++++++++++------- 1 file changed, 53 insertions(+), 30 deletions(-) diff --git a/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala b/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala index 0ffb3964c..b47b004f9 100644 --- a/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala +++ b/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala @@ -2,29 +2,39 @@ package io.qbeast.spark.utils import io.qbeast.spark.QbeastIntegrationTestSpec import io.qbeast.spark.internal.commands.ConvertToQbeastCommand -import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.delta.DeltaLog +import org.apache.spark.sql.{DataFrame, SparkSession} class ConvertToQbeastTest extends QbeastIntegrationTestSpec { val columnsToIndex: Seq[String] = Seq("user_id", "price", "category_id") + val dataSize = 99986 // loadTestData(spark).count - def showFileLog(spark: SparkSession, path: String, truncate: Boolean = true): Unit = { - // scalastyle:off println - val snapshot = DeltaLog.forTable(spark, path).snapshot + def convertFormatsFromTo( + sourceFormat: String, + targetFormat: String, + spark: SparkSession, + dir: String, + columnsToIndex: Seq[String] = columnsToIndex): DataFrame = { + val data = loadTestData(spark) + data.write.mode("overwrite").format(sourceFormat).save(dir) - println("AddFiles:") - snapshot.allFiles.show(truncate) + ConvertToQbeastCommand(dir, sourceFormat, columnsToIndex).run(spark) - println("RemoveFiles:") - snapshot.tombstones.show(truncate) + spark.read.format(targetFormat).load(dir) } - "ConvertToQbeast" should "convert a Parquet Table to a Qbeast Table" in withSparkAndTmpDir( - (spark, tmpDir) => {}) + "ConvertToQbeastCommand" should "convert a Delta Table into a Qbeast Table" in withSparkAndTmpDir( + (spark, tmpDir) => { + val qDf = convertFormatsFromTo("delta", "qbeast", spark, tmpDir) + qDf.count shouldBe dataSize + }) - it should "convert a Delta Table to a Qbeast Table" in withSparkAndTmpDir((spark, tmpDir) => {}) + it should "convert a Parquet Table into a Qbeast Table" in + withSparkAndTmpDir((spark, tmpDir) => { + val qDf = convertFormatsFromTo("parquet", "qbeast", spark, tmpDir) + qDf.count shouldBe dataSize + }) - it should "throw an error when attempting to convert to unsupported format" in withSparkAndTmpDir( + it should "throw an error when attempting to convert an unsupported format" in withSparkAndTmpDir( (spark, tmpDir) => { val df = loadTestData(spark) df.write.mode("overwrite").json(tmpDir) @@ -47,29 +57,42 @@ class ConvertToQbeastTest extends QbeastIntegrationTestSpec { // // }) - "A converted delta table" should "be readable using delta" in withSparkAndTmpDir( + it should "throw an error if columnsToIndex are not found in table schema" in withSparkAndTmpDir( (spark, tmpDir) => { - val df = loadTestData(spark) - df.write.format("delta").save(tmpDir) -// showFileLog(spark, tmpDir) + val nonExistentColumns = Seq("a", "b") + an[RuntimeException] shouldBe thrownBy( + convertFormatsFromTo("delta", "qbeast", spark, tmpDir, nonExistentColumns)) + }) - val fileFormat = "delta" - ConvertToQbeastCommand(tmpDir, fileFormat, columnsToIndex).run(spark) -// showFileLog(spark, tmpDir, truncate = false) + it should "should be idempotent" in withSparkAndTmpDir((spark, tmpDir) => {}) - spark.read.format("delta").load(tmpDir).count shouldBe df.count - }) + it should "create correct OTree metrics" in withSparkAndTmpDir((spark, tmpDir) => {}) - it should "be readable using qbeast" in withSparkAndTmpDir((spark, tmpDir) => { - val df = loadTestData(spark) - val fileFormat = "delta" - df.write.format(fileFormat).save(tmpDir) -// showFileLog(spark, tmpDir) + it should "allow correct execution of Analyze and Optimize" in withSparkAndTmpDir( + (spark, tmpDir) => {}) - ConvertToQbeastCommand(tmpDir, fileFormat, columnsToIndex).run(spark) -// showFileLog(spark, tmpDir, truncate = false) + it should "allow correct execution of Compaction" in withSparkAndTmpDir((spark, tmpDir) => {}) - spark.read.format("qbeast").load(tmpDir).count shouldBe df.count + "A converted delta table" should "be readable using delta" in withSparkAndTmpDir( + (spark, tmpDir) => { + val qDf = convertFormatsFromTo("delta", "delta", spark, tmpDir) + qDf.count shouldBe dataSize + }) + + it should "be readable using parquet" in withSparkAndTmpDir((spark, tmpDir) => { + val qDf = convertFormatsFromTo("delta", "parquet", spark, tmpDir) + qDf.count shouldBe dataSize }) +// "A converted parquet table" should "be readable using delta" in withSparkAndTmpDir( +// (spark, tmpDir) => { +// val qDf = convertFormatsFromTo("parquet", "delta", spark, tmpDir) +// qDf.count shouldBe dataSize +// }) +// +// it should "be readable using parquet" in withSparkAndTmpDir((spark, tmpDir) => { +// val qDf = convertFormatsFromTo("parquet", "parquet", spark, tmpDir) +// qDf.count shouldBe dataSize +// }) + } From b22afc08c56d5df5c8924b2beaaf30cd5305e061 Mon Sep 17 00:00:00 2001 From: Jiawei Date: Tue, 20 Sep 2022 18:09:05 +0200 Subject: [PATCH 13/76] Separate metadata tag computation --- .../commands/ConvertToQbeastCommand.scala | 60 ++++++++++++------- 1 file changed, 37 insertions(+), 23 deletions(-) diff --git a/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala b/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala index 0adc2c1ea..0272fd7a7 100644 --- a/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala +++ b/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala @@ -11,7 +11,7 @@ import io.qbeast.spark.index.SparkRevisionFactory import io.qbeast.spark.utils.{State, TagUtils} import org.apache.http.annotation.Experimental import org.apache.spark.qbeast.config.DEFAULT_CUBE_SIZE -import org.apache.spark.sql.delta.actions.FileAction +import org.apache.spark.sql.delta.actions.{AddFile, FileAction} import org.apache.spark.sql.{Row, SparkSession} import org.apache.spark.sql.delta.{DeltaLog, Snapshot} import org.apache.spark.sql.execution.command.LeafRunnableCommand @@ -27,11 +27,6 @@ case class ConvertToQbeastCommand( cubeSize: Int = DEFAULT_CUBE_SIZE) extends LeafRunnableCommand { - private val parquetFormat = "parquet" - private val deltaFormat = "delta" - private val acceptedFormats = Seq(parquetFormat, deltaFormat) - private val numRecordsPattern: Regex = """"numRecords":(\d+),""".r - private val intMinMax = ColumnMinMax(-1e8.toInt, 1e8.toInt) private val doubleMinMax = ColumnMinMax(-1e10, 1e10) private val longMinMax = ColumnMinMax(-1e15.toLong, 1e15.toLong) @@ -45,8 +40,12 @@ case class ConvertToQbeastCommand( TimestampDataType -> longMinMax, DateDataType -> longMinMax) + private def isSupportedFormat(format: String): Boolean = { + format == "parquet" || format == "delta" + } + private def convertParquetToDelta(spark: SparkSession, path: String): Unit = { - spark.sql(s"CONVERT TO DELTA $parquetFormat.`$path`") + spark.sql(s"CONVERT TO DELTA parquet.`$path`") } def initializeRevision(path: String, schema: StructType): Revision = { @@ -67,24 +66,12 @@ case class ConvertToQbeastCommand( } private def createQbeastActions(snapshot: Snapshot, revision: Revision): IISeq[FileAction] = { - val root = revision.createCubeIdRoot() val allFiles = snapshot.allFiles.collect() allFiles .map(addFile => { - val elementCount = numRecordsPattern.findFirstMatchIn(addFile.stats) match { - case Some(matching) => matching.group(1) - case _ => "0" - } - addFile.copy( - modificationTime = System.currentTimeMillis(), - tags = Map( - TagUtils.cube -> root.string, - TagUtils.minWeight -> Weight.MinValue.value.toString, - TagUtils.maxWeight -> Weight.MaxValue.value.toString, - TagUtils.state -> State.FLOODED, - TagUtils.revision -> revision.revisionID.toString, - TagUtils.elementCount -> elementCount)) + val metadataTag = QbeastMetadataExtractor.extractMetadataTag(addFile, revision) + addFile.copy(tags = metadataTag) }) .toIndexedSeq } @@ -104,12 +91,13 @@ case class ConvertToQbeastCommand( override def run(sparkSession: SparkSession): Seq[Row] = { // TODO very basic mechanism for converting to qbeast - if (!acceptedFormats.contains(fileFormat)) { + if (!isSupportedFormat(fileFormat)) { throw new UnsupportedOperationException(s"Unsupported file format: $fileFormat") } + // Make convert to delta idempotent // Convert parquet to delta - if (fileFormat == parquetFormat) convertParquetToDelta(sparkSession, path) + if (fileFormat == "parquet") convertParquetToDelta(sparkSession, path) // Convert delta to qbeast val snapshot = DeltaLog.forTable(sparkSession, path).snapshot @@ -129,3 +117,29 @@ case class ConvertToQbeastCommand( case class ColumnMinMax(minValue: Any, maxValue: Any) {} } + +object QbeastMetadataExtractor { + private val numRecordsPattern: Regex = """"numRecords":(\d+),""".r + private val defaultNumRecord: String = "-1" + + def extractMetadataTag(addFile: AddFile, revision: Revision): Map[String, String] = { + val elementCount = addFile.stats match { + case stats: String => + numRecordsPattern.findFirstMatchIn(stats) match { + case Some(matching) => matching.group(1) + case _ => defaultNumRecord + } + case _ => defaultNumRecord + } + + Map( + TagUtils.cube -> "", + TagUtils.minWeight -> Weight.MinValue.value.toString, + TagUtils.maxWeight -> Weight.MaxValue.value.toString, + TagUtils.state -> State.FLOODED, + TagUtils.revision -> revision.revisionID.toString, + TagUtils.elementCount -> elementCount) + + } + +} From 1de47a813feeb15ab79e3424eba442627066a45b Mon Sep 17 00:00:00 2001 From: Jiawei Date: Wed, 21 Sep 2022 10:03:49 +0200 Subject: [PATCH 14/76] Extract record count from parquet file metadata --- .../commands/ConvertToQbeastCommand.scala | 33 ++++++++++++++----- 1 file changed, 25 insertions(+), 8 deletions(-) diff --git a/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala b/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala index 0272fd7a7..36a9f98b9 100644 --- a/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala +++ b/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala @@ -9,7 +9,11 @@ import io.qbeast.core.transform._ import io.qbeast.spark.delta.SparkDeltaMetadataManager import io.qbeast.spark.index.SparkRevisionFactory import io.qbeast.spark.utils.{State, TagUtils} +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path import org.apache.http.annotation.Experimental +import org.apache.parquet.hadoop.ParquetFileReader +import org.apache.parquet.hadoop.util.HadoopInputFile import org.apache.spark.qbeast.config.DEFAULT_CUBE_SIZE import org.apache.spark.sql.delta.actions.{AddFile, FileAction} import org.apache.spark.sql.{Row, SparkSession} @@ -65,12 +69,15 @@ case class ConvertToQbeastCommand( revision.copy(transformations = transformations) } - private def createQbeastActions(snapshot: Snapshot, revision: Revision): IISeq[FileAction] = { + private def createQbeastActions( + snapshot: Snapshot, + revision: Revision, + path: String): IISeq[FileAction] = { val allFiles = snapshot.allFiles.collect() allFiles .map(addFile => { - val metadataTag = QbeastMetadataExtractor.extractMetadataTag(addFile, revision) + val metadataTag = QbeastMetadataExtractor.extractMetadataTag(addFile, revision, path) addFile.copy(tags = metadataTag) }) .toIndexedSeq @@ -108,28 +115,38 @@ case class ConvertToQbeastCommand( snapshot.schema, append = true) { val tableChanges = getTableChanges(revision) - val newFiles = createQbeastActions(snapshot, revision) + val newFiles = createQbeastActions(snapshot, revision, path) (tableChanges, newFiles) } Seq.empty } - case class ColumnMinMax(minValue: Any, maxValue: Any) {} } +case class ColumnMinMax(minValue: Any, maxValue: Any) + object QbeastMetadataExtractor { private val numRecordsPattern: Regex = """"numRecords":(\d+),""".r - private val defaultNumRecord: String = "-1" - def extractMetadataTag(addFile: AddFile, revision: Revision): Map[String, String] = { + def extractParquetFileCount(parquetFilePath: String): String = { + val path = new Path(parquetFilePath) + val file = HadoopInputFile.fromPath(path, new Configuration()) + val reader = ParquetFileReader.open(file) + reader.getRecordCount.toString + } + + def extractMetadataTag( + addFile: AddFile, + revision: Revision, + root: String): Map[String, String] = { val elementCount = addFile.stats match { case stats: String => numRecordsPattern.findFirstMatchIn(stats) match { case Some(matching) => matching.group(1) - case _ => defaultNumRecord + case _ => extractParquetFileCount(root + "/" + addFile.path) } - case _ => defaultNumRecord + case _ => extractParquetFileCount(root + "/" + addFile.path) } Map( From d435fc7668de123ad758d00524d6702852282995 Mon Sep 17 00:00:00 2001 From: Jiawei Date: Wed, 21 Sep 2022 10:05:18 +0200 Subject: [PATCH 15/76] Add test for parquet to qbeast conversion, test index metrics resulted from conversion --- .../spark/utils/ConvertToQbeastTest.scala | 31 ++++++++++++------- 1 file changed, 19 insertions(+), 12 deletions(-) diff --git a/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala b/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala index b47b004f9..8426e3dc5 100644 --- a/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala +++ b/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala @@ -1,6 +1,6 @@ package io.qbeast.spark.utils -import io.qbeast.spark.QbeastIntegrationTestSpec +import io.qbeast.spark.{QbeastIntegrationTestSpec, QbeastTable} import io.qbeast.spark.internal.commands.ConvertToQbeastCommand import org.apache.spark.sql.{DataFrame, SparkSession} @@ -60,13 +60,21 @@ class ConvertToQbeastTest extends QbeastIntegrationTestSpec { it should "throw an error if columnsToIndex are not found in table schema" in withSparkAndTmpDir( (spark, tmpDir) => { val nonExistentColumns = Seq("a", "b") + an[RuntimeException] shouldBe thrownBy( convertFormatsFromTo("delta", "qbeast", spark, tmpDir, nonExistentColumns)) }) it should "should be idempotent" in withSparkAndTmpDir((spark, tmpDir) => {}) - it should "create correct OTree metrics" in withSparkAndTmpDir((spark, tmpDir) => {}) + it should "create correct OTree metrics" in withSparkAndTmpDir((spark, tmpDir) => { + convertFormatsFromTo("delta", "qbeast", spark, tmpDir) + + val metrics = QbeastTable.forPath(spark, tmpDir).getIndexMetrics() + + metrics.elementCount shouldBe dataSize + metrics.cubeCount shouldBe 1 + }) it should "allow correct execution of Analyze and Optimize" in withSparkAndTmpDir( (spark, tmpDir) => {}) @@ -84,15 +92,14 @@ class ConvertToQbeastTest extends QbeastIntegrationTestSpec { qDf.count shouldBe dataSize }) -// "A converted parquet table" should "be readable using delta" in withSparkAndTmpDir( -// (spark, tmpDir) => { -// val qDf = convertFormatsFromTo("parquet", "delta", spark, tmpDir) -// qDf.count shouldBe dataSize -// }) -// -// it should "be readable using parquet" in withSparkAndTmpDir((spark, tmpDir) => { -// val qDf = convertFormatsFromTo("parquet", "parquet", spark, tmpDir) -// qDf.count shouldBe dataSize -// }) + "A converted parquet table" should "be readable using delta" in withSparkAndTmpDir( + (spark, tmpDir) => { + val qDf = convertFormatsFromTo("parquet", "delta", spark, tmpDir) + qDf.count shouldBe dataSize + }) + it should "be readable using parquet" in withSparkAndTmpDir((spark, tmpDir) => { + val qDf = convertFormatsFromTo("parquet", "parquet", spark, tmpDir) + qDf.count shouldBe dataSize + }) } From 0d264066ea209fbaeb55b3da7bbff05ef487ee45 Mon Sep 17 00:00:00 2001 From: Jiawei Date: Wed, 21 Sep 2022 10:59:22 +0200 Subject: [PATCH 16/76] Test String data type --- src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala b/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala index 8426e3dc5..ba14f1ee8 100644 --- a/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala +++ b/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala @@ -5,7 +5,7 @@ import io.qbeast.spark.internal.commands.ConvertToQbeastCommand import org.apache.spark.sql.{DataFrame, SparkSession} class ConvertToQbeastTest extends QbeastIntegrationTestSpec { - val columnsToIndex: Seq[String] = Seq("user_id", "price", "category_id") + val columnsToIndex: Seq[String] = Seq("user_id", "price", "event_type") val dataSize = 99986 // loadTestData(spark).count def convertFormatsFromTo( From 6f92dcc618c7f6fb97ad11504242af6db4e1728a Mon Sep 17 00:00:00 2001 From: Jiawei Date: Wed, 21 Sep 2022 14:19:56 +0200 Subject: [PATCH 17/76] Make command idempotent --- .../commands/ConvertToQbeastCommand.scala | 42 +++++++++++++++---- 1 file changed, 35 insertions(+), 7 deletions(-) diff --git a/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala b/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala index 36a9f98b9..27ed7bc86 100644 --- a/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala +++ b/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala @@ -6,7 +6,7 @@ package io.qbeast.spark.internal.commands import io.qbeast.IISeq import io.qbeast.core.model._ import io.qbeast.core.transform._ -import io.qbeast.spark.delta.SparkDeltaMetadataManager +import io.qbeast.spark.delta.{DeltaQbeastLog, SparkDeltaMetadataManager} import io.qbeast.spark.index.SparkRevisionFactory import io.qbeast.spark.utils.{State, TagUtils} import org.apache.hadoop.conf.Configuration @@ -48,6 +48,29 @@ case class ConvertToQbeastCommand( format == "parquet" || format == "delta" } + /** + * Check if the provided table is already a qbeast table by examining the RevisionIDs + * @param sparkSession SparkSession to use + * @return + */ + private def isQbeastTable(sparkSession: SparkSession): Boolean = { + // The first revisionID for the converted tables is 0, + // while for those written in the conventional fashion is 1. + val deltaLog = DeltaLog.forTable(sparkSession, path) + val qbeastSnapshot = DeltaQbeastLog(deltaLog).qbeastSnapshot + + qbeastSnapshot.existsRevision(0) || qbeastSnapshot.existsRevision(1) + } + + // scalastyle:off println + def logConsole(line: String): Unit = println(line) + // scalastyle:on println + + /** + * Convert the parquet table using ConvertToDeltaCommand from Delta Lake + * @param spark SparkSession to use + * @param path table path for the parquet table + */ private def convertParquetToDelta(spark: SparkSession, path: String): Unit = { spark.sql(s"CONVERT TO DELTA parquet.`$path`") } @@ -83,7 +106,7 @@ case class ConvertToQbeastCommand( .toIndexedSeq } - private def getTableChanges(revision: Revision): TableChanges = { + private def getTableChanges(revision: Revision, sparkSession: SparkSession): TableChanges = { val root = revision.createCubeIdRoot() BroadcastedTableChanges( @@ -92,16 +115,20 @@ case class ConvertToQbeastCommand( revision, Set.empty[CubeId], Set.empty[CubeId], - SparkSession.active.sparkContext.broadcast(Map(root -> State.FLOODED)), - SparkSession.active.sparkContext.broadcast(Map(root -> Weight.MaxValue))) + sparkSession.sparkContext.broadcast(Map(root -> State.FLOODED)), + sparkSession.sparkContext.broadcast(Map(root -> Weight.MaxValue))) } override def run(sparkSession: SparkSession): Seq[Row] = { - // TODO very basic mechanism for converting to qbeast if (!isSupportedFormat(fileFormat)) { throw new UnsupportedOperationException(s"Unsupported file format: $fileFormat") } - // Make convert to delta idempotent + + // Make convert to qbeast idempotent + if (isQbeastTable(sparkSession)) { + logConsole("The table you are trying to convert is already a qbeast table") + return Seq.empty + } // Convert parquet to delta if (fileFormat == "parquet") convertParquetToDelta(sparkSession, path) @@ -114,11 +141,12 @@ case class ConvertToQbeastCommand( revision.tableID, snapshot.schema, append = true) { - val tableChanges = getTableChanges(revision) + val tableChanges = getTableChanges(revision, sparkSession) val newFiles = createQbeastActions(snapshot, revision, path) (tableChanges, newFiles) } + Seq.empty } From 0208678c8f8a66e2ac19c8ef44ae78f2e77d9c16 Mon Sep 17 00:00:00 2001 From: Jiawei Date: Wed, 21 Sep 2022 16:32:08 +0200 Subject: [PATCH 18/76] Test command idempotence --- .../spark/utils/ConvertToQbeastTest.scala | 33 ++++++++++++++++--- 1 file changed, 29 insertions(+), 4 deletions(-) diff --git a/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala b/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala index ba14f1ee8..0f1c306f6 100644 --- a/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala +++ b/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala @@ -2,15 +2,17 @@ package io.qbeast.spark.utils import io.qbeast.spark.{QbeastIntegrationTestSpec, QbeastTable} import io.qbeast.spark.internal.commands.ConvertToQbeastCommand +import org.apache.spark.sql.delta.DeltaLog import org.apache.spark.sql.{DataFrame, SparkSession} +import org.scalatest.PrivateMethodTester -class ConvertToQbeastTest extends QbeastIntegrationTestSpec { +class ConvertToQbeastTest extends QbeastIntegrationTestSpec with PrivateMethodTester { val columnsToIndex: Seq[String] = Seq("user_id", "price", "event_type") val dataSize = 99986 // loadTestData(spark).count def convertFormatsFromTo( sourceFormat: String, - targetFormat: String, + readFormat: String, spark: SparkSession, dir: String, columnsToIndex: Seq[String] = columnsToIndex): DataFrame = { @@ -19,9 +21,11 @@ class ConvertToQbeastTest extends QbeastIntegrationTestSpec { ConvertToQbeastCommand(dir, sourceFormat, columnsToIndex).run(spark) - spark.read.format(targetFormat).load(dir) + spark.read.format(readFormat).load(dir) } + val privateIsQbeast: PrivateMethod[Boolean] = PrivateMethod[Boolean]('isQbeastTable) + "ConvertToQbeastCommand" should "convert a Delta Table into a Qbeast Table" in withSparkAndTmpDir( (spark, tmpDir) => { val qDf = convertFormatsFromTo("delta", "qbeast", spark, tmpDir) @@ -65,7 +69,28 @@ class ConvertToQbeastTest extends QbeastIntegrationTestSpec { convertFormatsFromTo("delta", "qbeast", spark, tmpDir, nonExistentColumns)) }) - it should "should be idempotent" in withSparkAndTmpDir((spark, tmpDir) => {}) + "ConvertToQbeastCommand's idempotence" should "not try to convert a converted table" in + withSparkAndTmpDir((spark, tmpDir) => { + convertFormatsFromTo("parquet", "qbeast", spark, tmpDir) + ConvertToQbeastCommand(tmpDir, "parquet", columnsToIndex).run(spark) + + val df = spark.read.format("qbeast").load(tmpDir) + val deltaLog = DeltaLog.forTable(spark, tmpDir) + + df.count shouldBe dataSize + // Converting parquet to delta creates snapshot version 0, and its + // conversion to qbeast creates snapshot version 1. If the second + // conversion gets executed, it'd produce a snapshot version 2 + deltaLog.snapshot.version shouldBe 1 + }) + + it should "not try to convert a qbeast table" in withSparkAndTmpDir((spark, tmpDir) => { + val data = loadTestData(spark) + writeTestData(data, columnsToIndex, 50000, tmpDir) + + ConvertToQbeastCommand(tmpDir, "parquet", columnsToIndex) invokePrivate privateIsQbeast( + spark) shouldBe true + }) it should "create correct OTree metrics" in withSparkAndTmpDir((spark, tmpDir) => { convertFormatsFromTo("delta", "qbeast", spark, tmpDir) From 909bc11b722c2d5f8c924dab8f8594f9308156a2 Mon Sep 17 00:00:00 2001 From: Jiawei Date: Wed, 21 Sep 2022 18:19:36 +0200 Subject: [PATCH 19/76] Test converting a partitioned delta table --- .../spark/utils/ConvertToQbeastTest.scala | 55 ++++++++++++------- 1 file changed, 35 insertions(+), 20 deletions(-) diff --git a/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala b/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala index 0f1c306f6..ab38cbe0a 100644 --- a/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala +++ b/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala @@ -26,8 +26,8 @@ class ConvertToQbeastTest extends QbeastIntegrationTestSpec with PrivateMethodTe val privateIsQbeast: PrivateMethod[Boolean] = PrivateMethod[Boolean]('isQbeastTable) - "ConvertToQbeastCommand" should "convert a Delta Table into a Qbeast Table" in withSparkAndTmpDir( - (spark, tmpDir) => { + "ConvertToQbeastCommand" should "convert a Delta Table into a Qbeast Table" in + withSparkAndTmpDir((spark, tmpDir) => { val qDf = convertFormatsFromTo("delta", "qbeast", spark, tmpDir) qDf.count shouldBe dataSize }) @@ -38,6 +38,21 @@ class ConvertToQbeastTest extends QbeastIntegrationTestSpec with PrivateMethodTe qDf.count shouldBe dataSize }) + it should "convert a partitioned delta table" in withSparkAndTmpDir((spark, tmpDir) => { + val tmpDir = "/tmp/test/" + val data = loadTestData(spark) + data.write + .mode("overwrite") + .partitionBy("event_type") + .format("delta") + .save(tmpDir) + + ConvertToQbeastCommand(tmpDir, "delta", columnsToIndex).run(spark) + val convertedTable = spark.read.format("qbeast").load(tmpDir) + + convertedTable.count shouldBe dataSize + }) + it should "throw an error when attempting to convert an unsupported format" in withSparkAndTmpDir( (spark, tmpDir) => { val df = loadTestData(spark) @@ -69,6 +84,20 @@ class ConvertToQbeastTest extends QbeastIntegrationTestSpec with PrivateMethodTe convertFormatsFromTo("delta", "qbeast", spark, tmpDir, nonExistentColumns)) }) + it should "create correct OTree metrics" in withSparkAndTmpDir((spark, tmpDir) => { + convertFormatsFromTo("delta", "qbeast", spark, tmpDir) + + val metrics = QbeastTable.forPath(spark, tmpDir).getIndexMetrics() + + metrics.elementCount shouldBe dataSize + metrics.cubeCount shouldBe 1 + }) + + it should "allow correct execution of Analyze and Optimize" in withSparkAndTmpDir( + (spark, tmpDir) => {}) + + it should "allow correct execution of Compaction" in withSparkAndTmpDir((spark, tmpDir) => {}) + "ConvertToQbeastCommand's idempotence" should "not try to convert a converted table" in withSparkAndTmpDir((spark, tmpDir) => { convertFormatsFromTo("parquet", "qbeast", spark, tmpDir) @@ -92,20 +121,6 @@ class ConvertToQbeastTest extends QbeastIntegrationTestSpec with PrivateMethodTe spark) shouldBe true }) - it should "create correct OTree metrics" in withSparkAndTmpDir((spark, tmpDir) => { - convertFormatsFromTo("delta", "qbeast", spark, tmpDir) - - val metrics = QbeastTable.forPath(spark, tmpDir).getIndexMetrics() - - metrics.elementCount shouldBe dataSize - metrics.cubeCount shouldBe 1 - }) - - it should "allow correct execution of Analyze and Optimize" in withSparkAndTmpDir( - (spark, tmpDir) => {}) - - it should "allow correct execution of Compaction" in withSparkAndTmpDir((spark, tmpDir) => {}) - "A converted delta table" should "be readable using delta" in withSparkAndTmpDir( (spark, tmpDir) => { val qDf = convertFormatsFromTo("delta", "delta", spark, tmpDir) @@ -117,14 +132,14 @@ class ConvertToQbeastTest extends QbeastIntegrationTestSpec with PrivateMethodTe qDf.count shouldBe dataSize }) - "A converted parquet table" should "be readable using delta" in withSparkAndTmpDir( + "A converted parquet table" should "be readable using parquet" in withSparkAndTmpDir( (spark, tmpDir) => { - val qDf = convertFormatsFromTo("parquet", "delta", spark, tmpDir) + val qDf = convertFormatsFromTo("parquet", "parquet", spark, tmpDir) qDf.count shouldBe dataSize }) - it should "be readable using parquet" in withSparkAndTmpDir((spark, tmpDir) => { - val qDf = convertFormatsFromTo("parquet", "parquet", spark, tmpDir) + it should "be readable using delta" in withSparkAndTmpDir((spark, tmpDir) => { + val qDf = convertFormatsFromTo("parquet", "delta", spark, tmpDir) qDf.count shouldBe dataSize }) } From c30c6e8da09ea1b4bacf69476303674b64e3d48e Mon Sep 17 00:00:00 2001 From: Jiawei Date: Wed, 21 Sep 2022 18:20:10 +0200 Subject: [PATCH 20/76] Use global path parameter --- .../internal/commands/ConvertToQbeastCommand.scala | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala b/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala index 27ed7bc86..173482c25 100644 --- a/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala +++ b/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala @@ -44,8 +44,8 @@ case class ConvertToQbeastCommand( TimestampDataType -> longMinMax, DateDataType -> longMinMax) - private def isSupportedFormat(format: String): Boolean = { - format == "parquet" || format == "delta" + private def isSupportedFormat: Boolean = { + fileFormat == "parquet" || fileFormat == "delta" } /** @@ -69,9 +69,8 @@ case class ConvertToQbeastCommand( /** * Convert the parquet table using ConvertToDeltaCommand from Delta Lake * @param spark SparkSession to use - * @param path table path for the parquet table */ - private def convertParquetToDelta(spark: SparkSession, path: String): Unit = { + private def convertParquetToDelta(spark: SparkSession): Unit = { spark.sql(s"CONVERT TO DELTA parquet.`$path`") } @@ -120,7 +119,7 @@ case class ConvertToQbeastCommand( } override def run(sparkSession: SparkSession): Seq[Row] = { - if (!isSupportedFormat(fileFormat)) { + if (!isSupportedFormat) { throw new UnsupportedOperationException(s"Unsupported file format: $fileFormat") } @@ -131,7 +130,7 @@ case class ConvertToQbeastCommand( } // Convert parquet to delta - if (fileFormat == "parquet") convertParquetToDelta(sparkSession, path) + if (fileFormat == "parquet") convertParquetToDelta(sparkSession) // Convert delta to qbeast val snapshot = DeltaLog.forTable(sparkSession, path).snapshot From 304c991c70921e40e2d88a768b439845f7148e7f Mon Sep 17 00:00:00 2001 From: Jiawei Date: Thu, 22 Sep 2022 10:20:42 +0200 Subject: [PATCH 21/76] Consider cases where the provided fileFormat doesn't match with real format of the table --- .../commands/ConvertToQbeastCommand.scala | 7 +++- .../spark/utils/ConvertToQbeastTest.scala | 37 ++++++++++++------- 2 files changed, 29 insertions(+), 15 deletions(-) diff --git a/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala b/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala index 173482c25..4b560e4d7 100644 --- a/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala +++ b/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala @@ -31,6 +31,9 @@ case class ConvertToQbeastCommand( cubeSize: Int = DEFAULT_CUBE_SIZE) extends LeafRunnableCommand { + private val parquetFormat = "parquet" + private val deltaFormat = "delta" + private val intMinMax = ColumnMinMax(-1e8.toInt, 1e8.toInt) private val doubleMinMax = ColumnMinMax(-1e10, 1e10) private val longMinMax = ColumnMinMax(-1e15.toLong, 1e15.toLong) @@ -45,7 +48,7 @@ case class ConvertToQbeastCommand( DateDataType -> longMinMax) private def isSupportedFormat: Boolean = { - fileFormat == "parquet" || fileFormat == "delta" + fileFormat == parquetFormat || fileFormat == deltaFormat } /** @@ -130,7 +133,7 @@ case class ConvertToQbeastCommand( } // Convert parquet to delta - if (fileFormat == "parquet") convertParquetToDelta(sparkSession) + if (fileFormat != deltaFormat) convertParquetToDelta(sparkSession) // Convert delta to qbeast val snapshot = DeltaLog.forTable(sparkSession, path).snapshot diff --git a/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala b/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala index ab38cbe0a..1e97ca1e7 100644 --- a/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala +++ b/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala @@ -2,6 +2,7 @@ package io.qbeast.spark.utils import io.qbeast.spark.{QbeastIntegrationTestSpec, QbeastTable} import io.qbeast.spark.internal.commands.ConvertToQbeastCommand +import org.apache.spark.SparkException import org.apache.spark.sql.delta.DeltaLog import org.apache.spark.sql.{DataFrame, SparkSession} import org.scalatest.PrivateMethodTester @@ -62,19 +63,29 @@ class ConvertToQbeastTest extends QbeastIntegrationTestSpec with PrivateMethodTe ConvertToQbeastCommand(tmpDir, "json", columnsToIndex).run(spark)) }) -// it should "throw an error when the file format does not match" in withSparkAndTmpDir( -// (spark, tmpDir) => { -// val df = loadTestData(spark) -// // write as json -// df.write.mode("overwrite").json(tmpDir) -// -// // Run the command -// // read as delta -// an[AnalysisException] shouldBe thrownBy( -// ConvertToQbeastCommand(tmpDir, "delta", columnsToIndex) -// .run(spark)) -// -// }) + it should "throw an error when trying to convert an unsupported format as parquet" in + withSparkAndTmpDir((spark, tmpDir) => { + // Write as json + val df = loadTestData(spark) + df.write.mode("overwrite").json(tmpDir) + + // Run the command reading as parquet + an[SparkException] shouldBe thrownBy( + ConvertToQbeastCommand(tmpDir, "parquet", columnsToIndex) + .run(spark)) + }) + + it should "throw an error when trying to convert an unsupported format as delta" in + withSparkAndTmpDir((spark, tmpDir) => { + // Write as json + val df = loadTestData(spark) + df.write.mode("overwrite").json(tmpDir) + + // Run the command reading as delta + an[IllegalArgumentException] shouldBe thrownBy( + ConvertToQbeastCommand(tmpDir, "delta", columnsToIndex) + .run(spark)) + }) it should "throw an error if columnsToIndex are not found in table schema" in withSparkAndTmpDir( (spark, tmpDir) => { From 72e40aead29dd1b7bb5d65c3fb16bc56f0589a39 Mon Sep 17 00:00:00 2001 From: Jiawei Date: Thu, 22 Sep 2022 15:34:36 +0200 Subject: [PATCH 22/76] Add format inferrence, conversion for partitioned parquet files, inferr schema for partitioned parquet files --- .../commands/ConvertToQbeastCommand.scala | 133 +++++++++++++----- .../spark/utils/ConvertToQbeastTest.scala | 109 +++++++------- 2 files changed, 147 insertions(+), 95 deletions(-) diff --git a/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala b/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala index 4b560e4d7..af87d31b6 100644 --- a/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala +++ b/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala @@ -8,6 +8,7 @@ import io.qbeast.core.model._ import io.qbeast.core.transform._ import io.qbeast.spark.delta.{DeltaQbeastLog, SparkDeltaMetadataManager} import io.qbeast.spark.index.SparkRevisionFactory +import io.qbeast.spark.internal.commands.ConvertToQbeastCommand.{dataTypeToName, extractQbeastTag} import io.qbeast.spark.utils.{State, TagUtils} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path @@ -19,20 +20,30 @@ import org.apache.spark.sql.delta.actions.{AddFile, FileAction} import org.apache.spark.sql.{Row, SparkSession} import org.apache.spark.sql.delta.{DeltaLog, Snapshot} import org.apache.spark.sql.execution.command.LeafRunnableCommand -import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.types.{ + BooleanType, + DateType, + DecimalType, + DoubleType, + FloatType, + IntegerType, + LongType, + StringType, + StructType, + TimestampType +} import scala.util.matching.Regex @Experimental case class ConvertToQbeastCommand( path: String, - fileFormat: String, columnsToIndex: Seq[String], - cubeSize: Int = DEFAULT_CUBE_SIZE) + cubeSize: Int = DEFAULT_CUBE_SIZE, + partitionColumns: Seq[String] = Seq.empty) extends LeafRunnableCommand { - private val parquetFormat = "parquet" - private val deltaFormat = "delta" + private val isPartitioned: Boolean = partitionColumns.nonEmpty private val intMinMax = ColumnMinMax(-1e8.toInt, 1e8.toInt) private val doubleMinMax = ColumnMinMax(-1e10, 1e10) @@ -47,22 +58,27 @@ case class ConvertToQbeastCommand( TimestampDataType -> longMinMax, DateDataType -> longMinMax) - private def isSupportedFormat: Boolean = { - fileFormat == parquetFormat || fileFormat == deltaFormat - } - - /** - * Check if the provided table is already a qbeast table by examining the RevisionIDs - * @param sparkSession SparkSession to use - * @return - */ - private def isQbeastTable(sparkSession: SparkSession): Boolean = { - // The first revisionID for the converted tables is 0, - // while for those written in the conventional fashion is 1. + private def resolveTableFormat(sparkSession: SparkSession): (String, StructType) = { val deltaLog = DeltaLog.forTable(sparkSession, path) val qbeastSnapshot = DeltaQbeastLog(deltaLog).qbeastSnapshot - - qbeastSnapshot.existsRevision(0) || qbeastSnapshot.existsRevision(1) + val schema = deltaLog.snapshot.schema + + val isDelta = deltaLog.tableExists + // The first revisionID for a converted table is 0, + // while for one that's written in the conventional fashion is 1. + val isQbeast = + isDelta && (qbeastSnapshot.existsRevision(0) || qbeastSnapshot.existsRevision(1)) + + if (isQbeast) { + ("qbeast", schema) + } else if (isDelta) { + ("delta", schema) + } else if (isPartitioned) { + ("parquet", sparkSession.read.parquet(path).schema) + } else { + // If parquet is not partitioned, schema.isEmpty but we don't need it + ("parquet", schema) + } } // scalastyle:off println @@ -73,23 +89,40 @@ case class ConvertToQbeastCommand( * Convert the parquet table using ConvertToDeltaCommand from Delta Lake * @param spark SparkSession to use */ - private def convertParquetToDelta(spark: SparkSession): Unit = { - spark.sql(s"CONVERT TO DELTA parquet.`$path`") + private def convertParquetToDelta(spark: SparkSession, schema: StructType): Unit = { + if (isPartitioned) { + assert(schema.nonEmpty, "Empty schema") + assert( + partitionColumns.forall(schema.names.contains), + s"""Partition column not found in schema. + |Partition columns: $partitionColumns, + |schema: $schema""".stripMargin) + + val colsAndTypes = + partitionColumns.map(colName => { + val sqlTypeName = dataTypeToName(colName, schema) + colName + " " + sqlTypeName + }) + spark.sql( + s"CONVERT TO DELTA parquet.`$path` PARTITIONED BY (${colsAndTypes.mkString(", ")})") + } else { + spark.sql(s"CONVERT TO DELTA parquet.`$path`") + } } - def initializeRevision(path: String, schema: StructType): Revision = { + private def initializeRevision(schema: StructType): Revision = { val revision = SparkRevisionFactory.createNewRevision( QTableID(path), schema, Map("columnsToIndex" -> columnsToIndex.mkString(","), "cubeSize" -> cubeSize.toString)) - val transformations: IISeq[Transformation] = revision.columnTransformers.map { + val transformations = revision.columnTransformers.map { case LinearTransformer(_, dataType: OrderedDataType) => val minMax = dataTypeMinMax(dataType) LinearTransformation(minMax.minValue, minMax.maxValue, dataType) case HashTransformer(_, _) => HashTransformation() - } + }.toIndexedSeq revision.copy(transformations = transformations) } @@ -102,7 +135,7 @@ case class ConvertToQbeastCommand( allFiles .map(addFile => { - val metadataTag = QbeastMetadataExtractor.extractMetadataTag(addFile, revision, path) + val metadataTag = extractQbeastTag(addFile, revision, path) addFile.copy(tags = metadataTag) }) .toIndexedSeq @@ -122,22 +155,22 @@ case class ConvertToQbeastCommand( } override def run(sparkSession: SparkSession): Seq[Row] = { - if (!isSupportedFormat) { - throw new UnsupportedOperationException(s"Unsupported file format: $fileFormat") - } - - // Make convert to qbeast idempotent - if (isQbeastTable(sparkSession)) { - logConsole("The table you are trying to convert is already a qbeast table") - return Seq.empty + val (fileFormat, sourceSchema) = resolveTableFormat(sparkSession) + + fileFormat match { + // Idempotent conversion + case "qbeast" => + logConsole("The table you are trying to convert is already a qbeast table") + return Seq.empty + // Convert parquet to delta + case "parquet" => convertParquetToDelta(sparkSession, sourceSchema) + // delta, do nothing + case _ => } - // Convert parquet to delta - if (fileFormat != deltaFormat) convertParquetToDelta(sparkSession) - // Convert delta to qbeast val snapshot = DeltaLog.forTable(sparkSession, path).snapshot - val revision = initializeRevision(path, snapshot.schema) + val revision = initializeRevision(snapshot.schema) SparkDeltaMetadataManager.updateWithTransaction( revision.tableID, @@ -156,7 +189,7 @@ case class ConvertToQbeastCommand( case class ColumnMinMax(minValue: Any, maxValue: Any) -object QbeastMetadataExtractor { +object ConvertToQbeastCommand { private val numRecordsPattern: Regex = """"numRecords":(\d+),""".r def extractParquetFileCount(parquetFilePath: String): String = { @@ -166,7 +199,7 @@ object QbeastMetadataExtractor { reader.getRecordCount.toString } - def extractMetadataTag( + def extractQbeastTag( addFile: AddFile, revision: Revision, root: String): Map[String, String] = { @@ -189,4 +222,26 @@ object QbeastMetadataExtractor { } + private def dataTypeToName(columnName: String, schema: StructType): String = { + val dataType = schema(columnName).dataType + dataType match { + // case _: ArrayType => "ARRAY" + // case _: BinaryType => "BINARY" + case _: BooleanType => "BOOLEAN" + // case _: ByteType => "TINYINT" + case _: DateType => "DATE" + case _: DecimalType => "DECIMAL" + case _: DoubleType => "DOUBLE" + case _: FloatType => "FLOAT" + case _: IntegerType => "INT" + case _: LongType => "BIGINT" + // case _: MapType => "MAP" + // case _: ShortType => "SMALLINT" + case _: StringType => "STRING" + // case _: StructType => "STRUCT" + case _: TimestampType => "TIMESTAMP" + case _ => throw new RuntimeException(s"$dataType is not supported") + } + } + } diff --git a/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala b/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala index 1e97ca1e7..c4c307cf3 100644 --- a/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala +++ b/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala @@ -20,71 +20,74 @@ class ConvertToQbeastTest extends QbeastIntegrationTestSpec with PrivateMethodTe val data = loadTestData(spark) data.write.mode("overwrite").format(sourceFormat).save(dir) - ConvertToQbeastCommand(dir, sourceFormat, columnsToIndex).run(spark) + ConvertToQbeastCommand(dir, columnsToIndex).run(spark) spark.read.format(readFormat).load(dir) } - val privateIsQbeast: PrivateMethod[Boolean] = PrivateMethod[Boolean]('isQbeastTable) - - "ConvertToQbeastCommand" should "convert a Delta Table into a Qbeast Table" in + "ConvertToQbeastCommand" should "convert a delta table into a qbeast table" in withSparkAndTmpDir((spark, tmpDir) => { - val qDf = convertFormatsFromTo("delta", "qbeast", spark, tmpDir) - qDf.count shouldBe dataSize + val convertedTable = convertFormatsFromTo("delta", "qbeast", spark, tmpDir) + + convertedTable.count shouldBe dataSize }) - it should "convert a Parquet Table into a Qbeast Table" in + it should "convert a PARTITIONED delta table into a qbeast table" in withSparkAndTmpDir((spark, tmpDir) => { - val qDf = convertFormatsFromTo("parquet", "qbeast", spark, tmpDir) - qDf.count shouldBe dataSize + // Use a reduced dataset since partitionBy is more expensive to run + val data = loadTestData(spark).limit(10000) + val partitionColumns = Seq("event_type") + data.write + .mode("overwrite") + .partitionBy(partitionColumns: _*) + .format("delta") + .save(tmpDir) + + // Convert a partitioned delta table to a qbeast table + ConvertToQbeastCommand(tmpDir, columnsToIndex, partitionColumns = partitionColumns) + .run(spark) + + val convertedTable = + spark.read.format("qbeast").load(tmpDir) + + convertedTable.count shouldBe 10000 }) - it should "convert a partitioned delta table" in withSparkAndTmpDir((spark, tmpDir) => { - val tmpDir = "/tmp/test/" - val data = loadTestData(spark) - data.write - .mode("overwrite") - .partitionBy("event_type") - .format("delta") - .save(tmpDir) - - ConvertToQbeastCommand(tmpDir, "delta", columnsToIndex).run(spark) - val convertedTable = spark.read.format("qbeast").load(tmpDir) - - convertedTable.count shouldBe dataSize - }) - - it should "throw an error when attempting to convert an unsupported format" in withSparkAndTmpDir( - (spark, tmpDir) => { - val df = loadTestData(spark) - df.write.mode("overwrite").json(tmpDir) + it should "convert a parquet table into a qbeast table" in + withSparkAndTmpDir((spark, tmpDir) => { + val convertedTable = convertFormatsFromTo("parquet", "qbeast", spark, tmpDir) - an[UnsupportedOperationException] shouldBe thrownBy( - ConvertToQbeastCommand(tmpDir, "json", columnsToIndex).run(spark)) + convertedTable.count shouldBe dataSize }) - it should "throw an error when trying to convert an unsupported format as parquet" in + it should "convert a PARTITIONED parquet table into a qbeast table" in withSparkAndTmpDir((spark, tmpDir) => { - // Write as json - val df = loadTestData(spark) - df.write.mode("overwrite").json(tmpDir) - - // Run the command reading as parquet - an[SparkException] shouldBe thrownBy( - ConvertToQbeastCommand(tmpDir, "parquet", columnsToIndex) - .run(spark)) + // Use a reduced dataset since partitionBy is more expensive to run + val data = loadTestData(spark).limit(10000) + val partitionColumns = Seq("event_type") + data.write + .mode("overwrite") + .partitionBy(partitionColumns: _*) + .format("parquet") + .save(tmpDir) + + // Conversion: Partitioned parquet -> delta -> qbeast + ConvertToQbeastCommand(tmpDir, columnsToIndex, partitionColumns = partitionColumns) + .run(spark) + + val convertedTable = + spark.read.format("qbeast").load(tmpDir) + + convertedTable.count shouldBe 10000 }) - it should "throw an error when trying to convert an unsupported format as delta" in - withSparkAndTmpDir((spark, tmpDir) => { - // Write as json + it should "throw an error when attempting to convert an unsupported format" in withSparkAndTmpDir( + (spark, tmpDir) => { val df = loadTestData(spark) df.write.mode("overwrite").json(tmpDir) - // Run the command reading as delta - an[IllegalArgumentException] shouldBe thrownBy( - ConvertToQbeastCommand(tmpDir, "delta", columnsToIndex) - .run(spark)) + an[SparkException] shouldBe thrownBy( + ConvertToQbeastCommand(tmpDir, columnsToIndex).run(spark)) }) it should "throw an error if columnsToIndex are not found in table schema" in withSparkAndTmpDir( @@ -111,8 +114,10 @@ class ConvertToQbeastTest extends QbeastIntegrationTestSpec with PrivateMethodTe "ConvertToQbeastCommand's idempotence" should "not try to convert a converted table" in withSparkAndTmpDir((spark, tmpDir) => { + // csv -> parquet -> delta -> qbeast convertFormatsFromTo("parquet", "qbeast", spark, tmpDir) - ConvertToQbeastCommand(tmpDir, "parquet", columnsToIndex).run(spark) + // qbeast -> qbeast + ConvertToQbeastCommand(tmpDir, columnsToIndex).run(spark) val df = spark.read.format("qbeast").load(tmpDir) val deltaLog = DeltaLog.forTable(spark, tmpDir) @@ -124,15 +129,7 @@ class ConvertToQbeastTest extends QbeastIntegrationTestSpec with PrivateMethodTe deltaLog.snapshot.version shouldBe 1 }) - it should "not try to convert a qbeast table" in withSparkAndTmpDir((spark, tmpDir) => { - val data = loadTestData(spark) - writeTestData(data, columnsToIndex, 50000, tmpDir) - - ConvertToQbeastCommand(tmpDir, "parquet", columnsToIndex) invokePrivate privateIsQbeast( - spark) shouldBe true - }) - - "A converted delta table" should "be readable using delta" in withSparkAndTmpDir( + "A converted Delta table" should "be readable using delta" in withSparkAndTmpDir( (spark, tmpDir) => { val qDf = convertFormatsFromTo("delta", "delta", spark, tmpDir) qDf.count shouldBe dataSize From f009a5458fbd7929b749237d483a55e9671493c8 Mon Sep 17 00:00:00 2001 From: Jiawei Date: Thu, 22 Sep 2022 16:04:14 +0200 Subject: [PATCH 23/76] Add comment --- .../commands/ConvertToQbeastCommand.scala | 81 ++++++++++++++----- 1 file changed, 60 insertions(+), 21 deletions(-) diff --git a/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala b/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala index af87d31b6..368497867 100644 --- a/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala +++ b/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala @@ -8,7 +8,11 @@ import io.qbeast.core.model._ import io.qbeast.core.transform._ import io.qbeast.spark.delta.{DeltaQbeastLog, SparkDeltaMetadataManager} import io.qbeast.spark.index.SparkRevisionFactory -import io.qbeast.spark.internal.commands.ConvertToQbeastCommand.{dataTypeToName, extractQbeastTag} +import io.qbeast.spark.internal.commands.ConvertToQbeastCommand.{ + dataTypeMinMax, + dataTypeToName, + extractQbeastTag +} import io.qbeast.spark.utils.{State, TagUtils} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path @@ -45,19 +49,13 @@ case class ConvertToQbeastCommand( private val isPartitioned: Boolean = partitionColumns.nonEmpty - private val intMinMax = ColumnMinMax(-1e8.toInt, 1e8.toInt) - private val doubleMinMax = ColumnMinMax(-1e10, 1e10) - private val longMinMax = ColumnMinMax(-1e15.toLong, 1e15.toLong) - - private val dataTypeMinMax = Map( - DoubleDataType -> doubleMinMax, - IntegerDataType -> intMinMax, - LongDataType -> longMinMax, - FloatDataType -> doubleMinMax, - DecimalDataType -> doubleMinMax, - TimestampDataType -> longMinMax, - DateDataType -> longMinMax) - + /** + * Format inference for the input table. If partition columns are provided, + * the format is assumed to be parquet. Any unsupported format is considered + * as parquet and is detected when trying to convert it into delta. + * @param sparkSession SparkSession to use + * @return + */ private def resolveTableFormat(sparkSession: SparkSession): (String, StructType) = { val deltaLog = DeltaLog.forTable(sparkSession, path) val qbeastSnapshot = DeltaQbeastLog(deltaLog).qbeastSnapshot @@ -74,9 +72,10 @@ case class ConvertToQbeastCommand( } else if (isDelta) { ("delta", schema) } else if (isPartitioned) { + // Partitioned parquet, table schema is required for its conversion into delta ("parquet", sparkSession.read.parquet(path).schema) } else { - // If parquet is not partitioned, schema.isEmpty but we don't need it + // Parquet, or any other unsupported format, schema.isEmpty but we don't need it ("parquet", schema) } } @@ -86,7 +85,8 @@ case class ConvertToQbeastCommand( // scalastyle:on println /** - * Convert the parquet table using ConvertToDeltaCommand from Delta Lake + * Convert the parquet table using ConvertToDeltaCommand from Delta Lake. + * Any unsupported format will cause a SparkException error. * @param spark SparkSession to use */ private def convertParquetToDelta(spark: SparkSession, schema: StructType): Unit = { @@ -110,6 +110,12 @@ case class ConvertToQbeastCommand( } } + /** + * Initialize Revision for table conversion. + * The smallest RevisionID for a converted table is 0. + * @param schema table schema + * @return + */ private def initializeRevision(schema: StructType): Revision = { val revision = SparkRevisionFactory.createNewRevision( @@ -187,11 +193,27 @@ case class ConvertToQbeastCommand( } -case class ColumnMinMax(minValue: Any, maxValue: Any) - object ConvertToQbeastCommand { private val numRecordsPattern: Regex = """"numRecords":(\d+),""".r + private val intMinMax = ColumnMinMax(-1e8.toInt, 1e8.toInt) + private val doubleMinMax = ColumnMinMax(-1e10, 1e10) + private val longMinMax = ColumnMinMax(-1e15.toLong, 1e15.toLong) + + private val dataTypeMinMax = Map( + DoubleDataType -> doubleMinMax, + IntegerDataType -> intMinMax, + LongDataType -> longMinMax, + FloatDataType -> doubleMinMax, + DecimalDataType -> doubleMinMax, + TimestampDataType -> longMinMax, + DateDataType -> longMinMax) + + /** + * Extract record count from a parquet file metadata. + * @param parquetFilePath target parquet file path + * @return + */ def extractParquetFileCount(parquetFilePath: String): String = { val path = new Path(parquetFilePath) val file = HadoopInputFile.fromPath(path, new Configuration()) @@ -199,17 +221,26 @@ object ConvertToQbeastCommand { reader.getRecordCount.toString } + /** + * Extract Qbeast metadata for an AddFile. + * @param addFile AddFile to be converted into a qbeast block for the root + * @param revision the conversion revision to use, revisionID = 0 + * @param tablePath path of the table + * @return + */ def extractQbeastTag( addFile: AddFile, revision: Revision, - root: String): Map[String, String] = { + tablePath: String): Map[String, String] = { val elementCount = addFile.stats match { case stats: String => numRecordsPattern.findFirstMatchIn(stats) match { case Some(matching) => matching.group(1) - case _ => extractParquetFileCount(root + "/" + addFile.path) + // stats does not contain record count, proceed extraction using parquet metadata + case _ => extractParquetFileCount(tablePath + "/" + addFile.path) } - case _ => extractParquetFileCount(root + "/" + addFile.path) + // AddFile entries with no 'stats' field, proceed extraction using parquet metadata + case _ => extractParquetFileCount(tablePath + "/" + addFile.path) } Map( @@ -222,6 +253,12 @@ object ConvertToQbeastCommand { } + /** + * Convert a Spark data type into a Sql data type. Used to convert partitioned parquet tables + * @param columnName, name of the column whose data type is of our interest + * @param schema table schema of the partitioned parquet table + * @return + */ private def dataTypeToName(columnName: String, schema: StructType): String = { val dataType = schema(columnName).dataType dataType match { @@ -245,3 +282,5 @@ object ConvertToQbeastCommand { } } + +case class ColumnMinMax(minValue: Any, maxValue: Any) From daf4454dd59c08208ae1e97ca2cf8c275cd28914 Mon Sep 17 00:00:00 2001 From: Jiawei Date: Thu, 22 Sep 2022 18:11:25 +0200 Subject: [PATCH 24/76] Test extraction of numRecords when AddFile stats is corrupted --- .../spark/utils/ConvertToQbeastTest.scala | 25 +++++++++++++++++++ 1 file changed, 25 insertions(+) diff --git a/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala b/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala index c4c307cf3..f599ebf93 100644 --- a/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala +++ b/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala @@ -1,5 +1,7 @@ package io.qbeast.spark.utils +import io.qbeast.core.model.QTableID +import io.qbeast.spark.index.SparkRevisionFactory import io.qbeast.spark.{QbeastIntegrationTestSpec, QbeastTable} import io.qbeast.spark.internal.commands.ConvertToQbeastCommand import org.apache.spark.SparkException @@ -107,6 +109,29 @@ class ConvertToQbeastTest extends QbeastIntegrationTestSpec with PrivateMethodTe metrics.cubeCount shouldBe 1 }) + it should "extract elementCount from file metadata if AddFile has corrupted stats" in + withSparkAndTmpDir((spark, tmpDir) => { + val data = loadTestData(spark).limit(500) + data + .coalesce(1) + .write + .mode("overwrite") + .format("delta") + .save(tmpDir) + + val snapshot = DeltaLog.forTable(spark, tmpDir).snapshot + val qbeastTag = ConvertToQbeastCommand.extractQbeastTag( + snapshot.allFiles.first().copy(stats = "{this is a corrupt stats string}"), + SparkRevisionFactory.createNewRevision( + QTableID(tmpDir), + snapshot.schema, + Map("columnsToIndex" -> columnsToIndex.mkString(","), "cubeSize" -> "5000000")), + tmpDir) + + val countFromMetadata = qbeastTag(TagUtils.elementCount).toInt + countFromMetadata shouldBe 500 + }) + it should "allow correct execution of Analyze and Optimize" in withSparkAndTmpDir( (spark, tmpDir) => {}) From c8efff5c4408c13f1637cb24c16aa3621a602366 Mon Sep 17 00:00:00 2001 From: Jiawei Date: Thu, 22 Sep 2022 18:11:58 +0200 Subject: [PATCH 25/76] Remove unnecessary assertions --- .../spark/internal/commands/ConvertToQbeastCommand.scala | 7 ------- 1 file changed, 7 deletions(-) diff --git a/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala b/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala index 368497867..16b091cbc 100644 --- a/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala +++ b/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala @@ -91,13 +91,6 @@ case class ConvertToQbeastCommand( */ private def convertParquetToDelta(spark: SparkSession, schema: StructType): Unit = { if (isPartitioned) { - assert(schema.nonEmpty, "Empty schema") - assert( - partitionColumns.forall(schema.names.contains), - s"""Partition column not found in schema. - |Partition columns: $partitionColumns, - |schema: $schema""".stripMargin) - val colsAndTypes = partitionColumns.map(colName => { val sqlTypeName = dataTypeToName(colName, schema) From 389401d81ac60e213e324ab1fb17781963c5b2d8 Mon Sep 17 00:00:00 2001 From: Jiawei Date: Fri, 23 Sep 2022 11:15:48 +0200 Subject: [PATCH 26/76] Spark data type name extraction for only supported partition data types --- .../commands/ConvertToQbeastCommand.scala | 56 ++++++------------- 1 file changed, 16 insertions(+), 40 deletions(-) diff --git a/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala b/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala index 16b091cbc..3343f678d 100644 --- a/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala +++ b/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala @@ -10,8 +10,8 @@ import io.qbeast.spark.delta.{DeltaQbeastLog, SparkDeltaMetadataManager} import io.qbeast.spark.index.SparkRevisionFactory import io.qbeast.spark.internal.commands.ConvertToQbeastCommand.{ dataTypeMinMax, - dataTypeToName, - extractQbeastTag + extractQbeastTag, + sparkToSqlTypeNames } import io.qbeast.spark.utils.{State, TagUtils} import org.apache.hadoop.conf.Configuration @@ -24,18 +24,7 @@ import org.apache.spark.sql.delta.actions.{AddFile, FileAction} import org.apache.spark.sql.{Row, SparkSession} import org.apache.spark.sql.delta.{DeltaLog, Snapshot} import org.apache.spark.sql.execution.command.LeafRunnableCommand -import org.apache.spark.sql.types.{ - BooleanType, - DateType, - DecimalType, - DoubleType, - FloatType, - IntegerType, - LongType, - StringType, - StructType, - TimestampType -} +import org.apache.spark.sql.types.StructType import scala.util.matching.Regex @@ -93,7 +82,8 @@ case class ConvertToQbeastCommand( if (isPartitioned) { val colsAndTypes = partitionColumns.map(colName => { - val sqlTypeName = dataTypeToName(colName, schema) + val typeName = schema(colName).dataType.typeName + val sqlTypeName = sparkToSqlTypeNames.getOrElse(typeName, typeName) colName + " " + sqlTypeName }) spark.sql( @@ -247,32 +237,18 @@ object ConvertToQbeastCommand { } /** - * Convert a Spark data type into a Sql data type. Used to convert partitioned parquet tables - * @param columnName, name of the column whose data type is of our interest - * @param schema table schema of the partitioned parquet table - * @return + * Map a Spark data type name to a Spark SQL type name. Currently only the following + * are supported for conversion. Any other data type are inferred as String after + * being used as Partition Columns. Timestamp but this is not supported as a partition + * column by delta conversion. */ - private def dataTypeToName(columnName: String, schema: StructType): String = { - val dataType = schema(columnName).dataType - dataType match { - // case _: ArrayType => "ARRAY" - // case _: BinaryType => "BINARY" - case _: BooleanType => "BOOLEAN" - // case _: ByteType => "TINYINT" - case _: DateType => "DATE" - case _: DecimalType => "DECIMAL" - case _: DoubleType => "DOUBLE" - case _: FloatType => "FLOAT" - case _: IntegerType => "INT" - case _: LongType => "BIGINT" - // case _: MapType => "MAP" - // case _: ShortType => "SMALLINT" - case _: StringType => "STRING" - // case _: StructType => "STRUCT" - case _: TimestampType => "TIMESTAMP" - case _ => throw new RuntimeException(s"$dataType is not supported") - } - } + private val sparkToSqlTypeNames = Map( + "integer" -> "INT", + "double" -> "DOUBLE", + "long" -> "BIGINT", + "date" -> "DATE", + "timestamp" -> "TIMESTAMP", + "string" -> "STRING") } From 1a4e5d52d4717be921e62445399271b725090366 Mon Sep 17 00:00:00 2001 From: Jiawei Date: Fri, 23 Sep 2022 11:16:55 +0200 Subject: [PATCH 27/76] Test supported partition data types --- .../spark/utils/ConvertToQbeastTest.scala | 106 ++++++++++++------ 1 file changed, 69 insertions(+), 37 deletions(-) diff --git a/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala b/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala index f599ebf93..b91106925 100644 --- a/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala +++ b/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala @@ -8,6 +8,7 @@ import org.apache.spark.SparkException import org.apache.spark.sql.delta.DeltaLog import org.apache.spark.sql.{DataFrame, SparkSession} import org.scalatest.PrivateMethodTester +import org.apache.spark.sql.functions.col class ConvertToQbeastTest extends QbeastIntegrationTestSpec with PrivateMethodTester { val columnsToIndex: Seq[String] = Seq("user_id", "price", "event_type") @@ -37,7 +38,7 @@ class ConvertToQbeastTest extends QbeastIntegrationTestSpec with PrivateMethodTe it should "convert a PARTITIONED delta table into a qbeast table" in withSparkAndTmpDir((spark, tmpDir) => { // Use a reduced dataset since partitionBy is more expensive to run - val data = loadTestData(spark).limit(10000) + val data = loadTestData(spark).limit(1000) val partitionColumns = Seq("event_type") data.write .mode("overwrite") @@ -52,7 +53,7 @@ class ConvertToQbeastTest extends QbeastIntegrationTestSpec with PrivateMethodTe val convertedTable = spark.read.format("qbeast").load(tmpDir) - convertedTable.count shouldBe 10000 + convertedTable.count shouldBe 1000 }) it should "convert a parquet table into a qbeast table" in @@ -65,7 +66,7 @@ class ConvertToQbeastTest extends QbeastIntegrationTestSpec with PrivateMethodTe it should "convert a PARTITIONED parquet table into a qbeast table" in withSparkAndTmpDir((spark, tmpDir) => { // Use a reduced dataset since partitionBy is more expensive to run - val data = loadTestData(spark).limit(10000) + val data = loadTestData(spark).limit(1000) val partitionColumns = Seq("event_type") data.write .mode("overwrite") @@ -80,7 +81,7 @@ class ConvertToQbeastTest extends QbeastIntegrationTestSpec with PrivateMethodTe val convertedTable = spark.read.format("qbeast").load(tmpDir) - convertedTable.count shouldBe 10000 + convertedTable.count shouldBe 1000 }) it should "throw an error when attempting to convert an unsupported format" in withSparkAndTmpDir( @@ -109,29 +110,6 @@ class ConvertToQbeastTest extends QbeastIntegrationTestSpec with PrivateMethodTe metrics.cubeCount shouldBe 1 }) - it should "extract elementCount from file metadata if AddFile has corrupted stats" in - withSparkAndTmpDir((spark, tmpDir) => { - val data = loadTestData(spark).limit(500) - data - .coalesce(1) - .write - .mode("overwrite") - .format("delta") - .save(tmpDir) - - val snapshot = DeltaLog.forTable(spark, tmpDir).snapshot - val qbeastTag = ConvertToQbeastCommand.extractQbeastTag( - snapshot.allFiles.first().copy(stats = "{this is a corrupt stats string}"), - SparkRevisionFactory.createNewRevision( - QTableID(tmpDir), - snapshot.schema, - Map("columnsToIndex" -> columnsToIndex.mkString(","), "cubeSize" -> "5000000")), - tmpDir) - - val countFromMetadata = qbeastTag(TagUtils.elementCount).toInt - countFromMetadata shouldBe 500 - }) - it should "allow correct execution of Analyze and Optimize" in withSparkAndTmpDir( (spark, tmpDir) => {}) @@ -144,10 +122,10 @@ class ConvertToQbeastTest extends QbeastIntegrationTestSpec with PrivateMethodTe // qbeast -> qbeast ConvertToQbeastCommand(tmpDir, columnsToIndex).run(spark) - val df = spark.read.format("qbeast").load(tmpDir) + val convertedTable = spark.read.format("qbeast").load(tmpDir) val deltaLog = DeltaLog.forTable(spark, tmpDir) - df.count shouldBe dataSize + convertedTable.count shouldBe dataSize // Converting parquet to delta creates snapshot version 0, and its // conversion to qbeast creates snapshot version 1. If the second // conversion gets executed, it'd produce a snapshot version 2 @@ -156,23 +134,77 @@ class ConvertToQbeastTest extends QbeastIntegrationTestSpec with PrivateMethodTe "A converted Delta table" should "be readable using delta" in withSparkAndTmpDir( (spark, tmpDir) => { - val qDf = convertFormatsFromTo("delta", "delta", spark, tmpDir) - qDf.count shouldBe dataSize + val convertedTable = convertFormatsFromTo("delta", "delta", spark, tmpDir) + convertedTable.count shouldBe dataSize }) it should "be readable using parquet" in withSparkAndTmpDir((spark, tmpDir) => { - val qDf = convertFormatsFromTo("delta", "parquet", spark, tmpDir) - qDf.count shouldBe dataSize + val convertedTable = convertFormatsFromTo("delta", "parquet", spark, tmpDir) + convertedTable.count shouldBe dataSize }) "A converted parquet table" should "be readable using parquet" in withSparkAndTmpDir( (spark, tmpDir) => { - val qDf = convertFormatsFromTo("parquet", "parquet", spark, tmpDir) - qDf.count shouldBe dataSize + val convertedTable = convertFormatsFromTo("parquet", "parquet", spark, tmpDir) + convertedTable.count shouldBe dataSize }) it should "be readable using delta" in withSparkAndTmpDir((spark, tmpDir) => { - val qDf = convertFormatsFromTo("parquet", "delta", spark, tmpDir) - qDf.count shouldBe dataSize + val convertedTable = convertFormatsFromTo("parquet", "delta", spark, tmpDir) + convertedTable.count shouldBe dataSize }) + + "extractQbeastTag" should + "extract elementCount from file metadata if AddFile has corrupted stats" in + withSparkAndTmpDir((spark, tmpDir) => { + val data = loadTestData(spark).limit(500) + data + .coalesce(1) + .write + .mode("overwrite") + .format("delta") + .save(tmpDir) + + val snapshot = DeltaLog.forTable(spark, tmpDir).snapshot + val qbeastTag = ConvertToQbeastCommand.extractQbeastTag( + snapshot.allFiles.first().copy(stats = "{this is a corrupt stats string}"), + SparkRevisionFactory.createNewRevision( + QTableID(tmpDir), + snapshot.schema, + Map("columnsToIndex" -> columnsToIndex.mkString(","), "cubeSize" -> "5000000")), + tmpDir) + + val countFromMetadata = qbeastTag(TagUtils.elementCount).toInt + countFromMetadata shouldBe 500 + }) + + "dataTypeToName" should + "be able to convert data types: Int, Double, and Long" in withSparkAndTmpDir( + (spark, tmpDir) => { + val data = loadTestData(spark).limit(20) + val partitionedData = data + .withColumn("stringType", col("event_type")) + .withColumn("integerType", col("user_id").cast("INT")) + .withColumn("doubleType", col("price").cast("DOUBLE")) + .withColumn("longType", col("category_id")) + .withColumn("dateType", col("event_time").cast("DATE")) + .drop(data.columns: _*) + + // integerType, doubleType, longType, dataType. Can't all columns for partitioning + val partitionColumns = partitionedData.columns.tail + partitionedData.write + .mode("overwrite") + .partitionBy(partitionColumns: _*) + .parquet(tmpDir) + + // Partitioned parquet -> delta -> qbeast + ConvertToQbeastCommand( + tmpDir, + Seq("doubleType", "integerType", "longType"), + partitionColumns = partitionColumns) + .run(spark) + + val convertedTable = spark.read.format("qbeast").load(tmpDir) + convertedTable.count shouldBe 20 + }) } From f5858f962ad75cd1128835021be033cfea6675a5 Mon Sep 17 00:00:00 2001 From: Jiawei Date: Fri, 23 Sep 2022 12:18:36 +0200 Subject: [PATCH 28/76] Secure return type --- .../commands/ConvertToQbeastCommand.scala | 33 +++++++++---------- 1 file changed, 16 insertions(+), 17 deletions(-) diff --git a/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala b/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala index 3343f678d..1a0ffa7b8 100644 --- a/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala +++ b/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala @@ -150,7 +150,7 @@ case class ConvertToQbeastCommand( // Idempotent conversion case "qbeast" => logConsole("The table you are trying to convert is already a qbeast table") - return Seq.empty + return Seq.empty[Row] // Convert parquet to delta case "parquet" => convertParquetToDelta(sparkSession, sourceSchema) // delta, do nothing @@ -171,7 +171,7 @@ case class ConvertToQbeastCommand( (tableChanges, newFiles) } - Seq.empty + Seq.empty[Row] } } @@ -192,6 +192,20 @@ object ConvertToQbeastCommand { TimestampDataType -> longMinMax, DateDataType -> longMinMax) + /** + * Map a Spark data type name to a Spark SQL type name. Currently only the following + * are supported for conversion. Any other data type are inferred as String after + * being used as Partition Columns. Timestamp but this is not supported as a partition + * column by delta conversion. + */ + private val sparkToSqlTypeNames = Map( + "integer" -> "INT", + "double" -> "DOUBLE", + "long" -> "BIGINT", + "date" -> "DATE", + "timestamp" -> "TIMESTAMP", + "string" -> "STRING") + /** * Extract record count from a parquet file metadata. * @param parquetFilePath target parquet file path @@ -233,23 +247,8 @@ object ConvertToQbeastCommand { TagUtils.state -> State.FLOODED, TagUtils.revision -> revision.revisionID.toString, TagUtils.elementCount -> elementCount) - } - /** - * Map a Spark data type name to a Spark SQL type name. Currently only the following - * are supported for conversion. Any other data type are inferred as String after - * being used as Partition Columns. Timestamp but this is not supported as a partition - * column by delta conversion. - */ - private val sparkToSqlTypeNames = Map( - "integer" -> "INT", - "double" -> "DOUBLE", - "long" -> "BIGINT", - "date" -> "DATE", - "timestamp" -> "TIMESTAMP", - "string" -> "STRING") - } case class ColumnMinMax(minValue: Any, maxValue: Any) From 02e9100b5f936a909a1e2b3a3ecc661477e43566 Mon Sep 17 00:00:00 2001 From: Jiawei Date: Fri, 23 Sep 2022 17:01:45 +0200 Subject: [PATCH 29/76] Use wider default column min max to avoid out-of-scope points --- .../spark/internal/commands/ConvertToQbeastCommand.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala b/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala index 1a0ffa7b8..f529900b1 100644 --- a/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala +++ b/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala @@ -179,9 +179,9 @@ case class ConvertToQbeastCommand( object ConvertToQbeastCommand { private val numRecordsPattern: Regex = """"numRecords":(\d+),""".r - private val intMinMax = ColumnMinMax(-1e8.toInt, 1e8.toInt) - private val doubleMinMax = ColumnMinMax(-1e10, 1e10) - private val longMinMax = ColumnMinMax(-1e15.toLong, 1e15.toLong) + private val intMinMax = ColumnMinMax(Int.MinValue, Int.MaxValue) + private val doubleMinMax = ColumnMinMax(-1e308, 1e307) + private val longMinMax = ColumnMinMax(Long.MinValue, Long.MaxValue) private val dataTypeMinMax = Map( DoubleDataType -> doubleMinMax, @@ -243,7 +243,7 @@ object ConvertToQbeastCommand { Map( TagUtils.cube -> "", TagUtils.minWeight -> Weight.MinValue.value.toString, - TagUtils.maxWeight -> Weight.MaxValue.value.toString, + TagUtils.maxWeight -> (Weight.MaxValue.value - 1).toString, TagUtils.state -> State.FLOODED, TagUtils.revision -> revision.revisionID.toString, TagUtils.elementCount -> elementCount) From c1c16d5fffa7e91905d3eb9c394722f6418d248a Mon Sep 17 00:00:00 2001 From: Jiawei Date: Fri, 23 Sep 2022 21:12:23 +0200 Subject: [PATCH 30/76] Add tests for Analyze, and Compaction on a converted qbeast table --- .../spark/utils/ConvertToQbeastTest.scala | 42 +++++++++++++++---- 1 file changed, 34 insertions(+), 8 deletions(-) diff --git a/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala b/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala index b91106925..0f6457096 100644 --- a/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala +++ b/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala @@ -1,10 +1,11 @@ package io.qbeast.spark.utils -import io.qbeast.core.model.QTableID +import io.qbeast.core.model.{CubeId, QTableID} import io.qbeast.spark.index.SparkRevisionFactory import io.qbeast.spark.{QbeastIntegrationTestSpec, QbeastTable} import io.qbeast.spark.internal.commands.ConvertToQbeastCommand -import org.apache.spark.SparkException +import org.apache.spark.{SparkConf, SparkException} +import org.apache.spark.qbeast.config.DEFAULT_CUBE_SIZE import org.apache.spark.sql.delta.DeltaLog import org.apache.spark.sql.{DataFrame, SparkSession} import org.scalatest.PrivateMethodTester @@ -19,11 +20,12 @@ class ConvertToQbeastTest extends QbeastIntegrationTestSpec with PrivateMethodTe readFormat: String, spark: SparkSession, dir: String, - columnsToIndex: Seq[String] = columnsToIndex): DataFrame = { + columnsToIndex: Seq[String] = columnsToIndex, + desiredCubeSize: Int = DEFAULT_CUBE_SIZE): DataFrame = { val data = loadTestData(spark) data.write.mode("overwrite").format(sourceFormat).save(dir) - ConvertToQbeastCommand(dir, columnsToIndex).run(spark) + ConvertToQbeastCommand(dir, columnsToIndex, desiredCubeSize).run(spark) spark.read.format(readFormat).load(dir) } @@ -110,10 +112,34 @@ class ConvertToQbeastTest extends QbeastIntegrationTestSpec with PrivateMethodTe metrics.cubeCount shouldBe 1 }) - it should "allow correct execution of Analyze and Optimize" in withSparkAndTmpDir( - (spark, tmpDir) => {}) + "Analyzing a converted qbeast table" should "return the root node" in + withSparkAndTmpDir((spark, tmpDir) => { + convertFormatsFromTo("parquet", "qbeast", spark, tmpDir, desiredCubeSize = 50000) + val qbeastTable = QbeastTable.forPath(spark, tmpDir) + val announcedCubes = qbeastTable.analyze() + announcedCubes shouldBe Seq(CubeId.root(columnsToIndex.size).string) + }) + + "Compacting a converted qbeast table" should "reduce root node block count" in + withExtendedSparkAndTmpDir( + new SparkConf() + .set("spark.qbeast.compact.minFileSize", "1") + .set("spark.qbeast.compact.maxFileSize", "2000000")) { (spark, tmpDir) => + { + convertFormatsFromTo("parquet", "qbeast", spark, tmpDir, desiredCubeSize = 50000) + val qbeastTable = QbeastTable.forPath(spark, tmpDir) + val root = CubeId.root(columnsToIndex.size) + + val rootBlockCountBefore = qbeastTable.getIndexMetrics().cubeStatuses(root).files.size + + qbeastTable.compact() + + val rootBlockCountAfter = qbeastTable.getIndexMetrics().cubeStatuses(root).files.size + + rootBlockCountAfter shouldBe <(rootBlockCountBefore) - it should "allow correct execution of Compaction" in withSparkAndTmpDir((spark, tmpDir) => {}) + } + } "ConvertToQbeastCommand's idempotence" should "not try to convert a converted table" in withSparkAndTmpDir((spark, tmpDir) => { @@ -179,7 +205,7 @@ class ConvertToQbeastTest extends QbeastIntegrationTestSpec with PrivateMethodTe }) "dataTypeToName" should - "be able to convert data types: Int, Double, and Long" in withSparkAndTmpDir( + "be able to convert data types: Int, Double, Long, and Date" in withSparkAndTmpDir( (spark, tmpDir) => { val data = loadTestData(spark).limit(20) val partitionedData = data From 25a3956b59400431eec4bb647ab48b8d91e07c5d Mon Sep 17 00:00:00 2001 From: Jiawei Date: Fri, 23 Sep 2022 21:36:27 +0200 Subject: [PATCH 31/76] Use table path from tablID --- .../commands/ConvertToQbeastCommand.scala | 26 +++++++------------ .../spark/utils/ConvertToQbeastTest.scala | 3 +-- 2 files changed, 11 insertions(+), 18 deletions(-) diff --git a/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala b/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala index f529900b1..2fca3b8ca 100644 --- a/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala +++ b/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala @@ -116,15 +116,12 @@ case class ConvertToQbeastCommand( revision.copy(transformations = transformations) } - private def createQbeastActions( - snapshot: Snapshot, - revision: Revision, - path: String): IISeq[FileAction] = { + private def createQbeastActions(snapshot: Snapshot, revision: Revision): IISeq[FileAction] = { val allFiles = snapshot.allFiles.collect() allFiles .map(addFile => { - val metadataTag = extractQbeastTag(addFile, revision, path) + val metadataTag = extractQbeastTag(addFile, revision) addFile.copy(tags = metadataTag) }) .toIndexedSeq @@ -166,7 +163,7 @@ case class ConvertToQbeastCommand( snapshot.schema, append = true) { val tableChanges = getTableChanges(revision, sparkSession) - val newFiles = createQbeastActions(snapshot, revision, path) + val newFiles = createQbeastActions(snapshot, revision) (tableChanges, newFiles) } @@ -211,9 +208,8 @@ object ConvertToQbeastCommand { * @param parquetFilePath target parquet file path * @return */ - def extractParquetFileCount(parquetFilePath: String): String = { - val path = new Path(parquetFilePath) - val file = HadoopInputFile.fromPath(path, new Configuration()) + def extractParquetFileCount(parquetFilePath: Path): String = { + val file = HadoopInputFile.fromPath(parquetFilePath, new Configuration()) val reader = ParquetFileReader.open(file) reader.getRecordCount.toString } @@ -222,22 +218,20 @@ object ConvertToQbeastCommand { * Extract Qbeast metadata for an AddFile. * @param addFile AddFile to be converted into a qbeast block for the root * @param revision the conversion revision to use, revisionID = 0 - * @param tablePath path of the table * @return */ - def extractQbeastTag( - addFile: AddFile, - revision: Revision, - tablePath: String): Map[String, String] = { + def extractQbeastTag(addFile: AddFile, revision: Revision): Map[String, String] = { val elementCount = addFile.stats match { case stats: String => numRecordsPattern.findFirstMatchIn(stats) match { case Some(matching) => matching.group(1) // stats does not contain record count, proceed extraction using parquet metadata - case _ => extractParquetFileCount(tablePath + "/" + addFile.path) + case _ => + extractParquetFileCount(new Path(revision.tableID.id, addFile.path)) } // AddFile entries with no 'stats' field, proceed extraction using parquet metadata - case _ => extractParquetFileCount(tablePath + "/" + addFile.path) + case _ => + extractParquetFileCount(new Path(revision.tableID.id, addFile.path)) } Map( diff --git a/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala b/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala index 0f6457096..92df3defc 100644 --- a/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala +++ b/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala @@ -197,8 +197,7 @@ class ConvertToQbeastTest extends QbeastIntegrationTestSpec with PrivateMethodTe SparkRevisionFactory.createNewRevision( QTableID(tmpDir), snapshot.schema, - Map("columnsToIndex" -> columnsToIndex.mkString(","), "cubeSize" -> "5000000")), - tmpDir) + Map("columnsToIndex" -> columnsToIndex.mkString(","), "cubeSize" -> "5000000"))) val countFromMetadata = qbeastTag(TagUtils.elementCount).toInt countFromMetadata shouldBe 500 From 56baac94ade7328fbe87f20c0ecc14f664114221 Mon Sep 17 00:00:00 2001 From: Jiawei Date: Fri, 23 Sep 2022 22:12:09 +0200 Subject: [PATCH 32/76] Add test for optimization on a converted table --- .../spark/utils/ConvertToQbeastTest.scala | 45 +++++++++++++++++++ 1 file changed, 45 insertions(+) diff --git a/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala b/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala index 92df3defc..ddc90e974 100644 --- a/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala +++ b/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala @@ -120,6 +120,51 @@ class ConvertToQbeastTest extends QbeastIntegrationTestSpec with PrivateMethodTe announcedCubes shouldBe Seq(CubeId.root(columnsToIndex.size).string) }) + "Optimizing a converted qbeast table" should "preserve table size" in + withSparkAndTmpDir((spark, tmpDir) => { + convertFormatsFromTo("parquet", "qbeast", spark, tmpDir, desiredCubeSize = 50000) + val qbeastTable = QbeastTable.forPath(spark, tmpDir) + + qbeastTable.analyze() + qbeastTable.optimize() + + val convertedTable = spark.read.format("qbeast").load(tmpDir) + val optimizedConvertedTableSize = qbeastTable + .getIndexMetrics() + .cubeStatuses + .values + .map(status => { + // Read non-replicated files + val filesToRead = status.files.filter(_.state != State.REPLICATED) + filesToRead.map(_.elementCount).sum + }) + .sum + + convertedTable.count shouldBe dataSize + optimizedConvertedTableSize shouldBe dataSize + + }) + + it should "preserve sampling accuracy" in withSparkAndTmpDir((spark, tmpDir) => { + convertFormatsFromTo("parquet", "qbeast", spark, tmpDir, desiredCubeSize = 50000) + val qbeastTable = QbeastTable.forPath(spark, tmpDir) + + qbeastTable.analyze() + qbeastTable.optimize() + + val convertedTable = spark.read.format("qbeast").load(tmpDir) + val tolerance = 0.01 + List(0.1, 0.2, 0.5, 0.7, 0.99).foreach(f => { + val sampleSize = convertedTable + .sample(withReplacement = false, f) + .count() + .toDouble + + val margin = dataSize * f * tolerance + sampleSize shouldBe (dataSize * f) +- margin + }) + }) + "Compacting a converted qbeast table" should "reduce root node block count" in withExtendedSparkAndTmpDir( new SparkConf() From 6171063df9e5bd2bf4a3ac2c45ffffec4d538993 Mon Sep 17 00:00:00 2001 From: Jiawei Date: Wed, 28 Sep 2022 17:00:55 +0200 Subject: [PATCH 33/76] Remove command --- .../PartiallyConvertToQbeastCommand.scala | 31 ------------------- 1 file changed, 31 deletions(-) delete mode 100644 src/main/scala/io/qbeast/spark/internal/commands/PartiallyConvertToQbeastCommand.scala diff --git a/src/main/scala/io/qbeast/spark/internal/commands/PartiallyConvertToQbeastCommand.scala b/src/main/scala/io/qbeast/spark/internal/commands/PartiallyConvertToQbeastCommand.scala deleted file mode 100644 index a90b98d9c..000000000 --- a/src/main/scala/io/qbeast/spark/internal/commands/PartiallyConvertToQbeastCommand.scala +++ /dev/null @@ -1,31 +0,0 @@ -/* - * Copyright 2021 Qbeast Analytics, S.L. - */ -package io.qbeast.spark.internal.commands - -import jdk.jfr.Experimental -import org.apache.spark.qbeast.config.DEFAULT_CUBE_SIZE -import org.apache.spark.sql.{Row, SparkSession} -import org.apache.spark.sql.execution.command.{LeafRunnableCommand} - -@Experimental -case class PartiallyConvertToQbeastCommand( - path: String, - fileFormat: String, - columnsToIndex: Seq[String], - cubeSize: Int = DEFAULT_CUBE_SIZE) - extends LeafRunnableCommand { - - override def run(sparkSession: SparkSession): Seq[Row] = { - - fileFormat match { - case "delta" => // - case "parquet" => // - case _ => throw new UnsupportedOperationException(s"Unsupported file format: $fileFormat") - } - // TODO - Seq.empty - - } - -} From 486711fab769d495b641a835add4c47ee5b19294 Mon Sep 17 00:00:00 2001 From: Jiawei Date: Mon, 16 Jan 2023 18:06:03 +0100 Subject: [PATCH 34/76] Add staging files during read --- .../io/qbeast/spark/delta/OTreeIndex.scala | 36 +++++++++++++------ 1 file changed, 25 insertions(+), 11 deletions(-) diff --git a/src/main/scala/io/qbeast/spark/delta/OTreeIndex.scala b/src/main/scala/io/qbeast/spark/delta/OTreeIndex.scala index d6fe79c79..c2b8d6108 100644 --- a/src/main/scala/io/qbeast/spark/delta/OTreeIndex.scala +++ b/src/main/scala/io/qbeast/spark/delta/OTreeIndex.scala @@ -6,15 +6,15 @@ package io.qbeast.spark.delta import io.qbeast.core.model.QbeastBlock import io.qbeast.spark.index.query.{QueryExecutor, QuerySpecBuilder} import org.apache.hadoop.fs.{FileStatus, Path} +import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.expressions.{Expression, GenericInternalRow} -import org.apache.spark.sql.delta.Snapshot +import org.apache.spark.sql.delta.{DeltaLog, Snapshot} +import org.apache.spark.sql.delta.actions.AddFile import org.apache.spark.sql.delta.files.TahoeLogFileIndex import org.apache.spark.sql.execution.datasources.{FileIndex, PartitionDirectory} import org.apache.spark.sql.types.StructType -import org.apache.spark.sql.SparkSession import java.net.URI -import org.apache.spark.sql.delta.DeltaLog /** * FileIndex to prune files @@ -49,11 +49,23 @@ case class OTreeIndex(index: TahoeLogFileIndex) extends FileIndex { queryExecutor.execute() } + private def stagingFiles: Seq[FileStatus] = { + snapshot.allFiles.where("tags IS NULL").collect().map { a: AddFile => + new FileStatus( + /* length */ a.size, + /* isDir */ false, + /* blockReplication */ 0, + /* blockSize */ 1, + /* modificationTime */ a.modificationTime, + absolutePath(a.path)) + } + } + override def listFiles( partitionFilters: Seq[Expression], dataFilters: Seq[Expression]): Seq[PartitionDirectory] = { - val fileStats = matchingBlocks(partitionFilters, dataFilters).map { qbeastBlock => + val qbeastFileStats = matchingBlocks(partitionFilters, dataFilters).map { qbeastBlock => new FileStatus( /* length */ qbeastBlock.size, /* isDir */ false, @@ -62,6 +74,8 @@ case class OTreeIndex(index: TahoeLogFileIndex) extends FileIndex { /* modificationTime */ qbeastBlock.modificationTime, absolutePath(qbeastBlock.path)) }.toArray + val stagingStats = stagingFiles + val fileStats = qbeastFileStats ++ stagingStats Seq(PartitionDirectory(new GenericInternalRow(Array.empty[Any]), fileStats)) @@ -80,17 +94,17 @@ case class OTreeIndex(index: TahoeLogFileIndex) extends FileIndex { override def partitionSchema: StructType = index.partitionSchema } -/** - * Object OTreeIndex to create a new OTreeIndex - * @param sparkSession the spark session - * @param path the path to the delta log - * @return the OTreeIndex - */ object OTreeIndex { def apply(spark: SparkSession, path: Path): OTreeIndex = { val deltaLog = DeltaLog.forTable(spark, path) - val tahoe = TahoeLogFileIndex(spark, deltaLog, path, deltaLog.snapshot, Seq.empty, false) + val tahoe = TahoeLogFileIndex( + spark, + deltaLog, + path, + deltaLog.snapshot, + Seq.empty, + isTimeTravelQuery = false) OTreeIndex(tahoe) } From 116ae7758902fdab9ba9ecffaa59fb44555fe30b Mon Sep 17 00:00:00 2001 From: Jiawei Date: Wed, 18 Jan 2023 18:12:30 +0100 Subject: [PATCH 35/76] WIP, Qbeast format compatibility with delta lake --- .../qbeast/core/model/RevisionClasses.scala | 26 +++++- .../spark/delta/DeltaMetadataWriter.scala | 2 +- .../spark/delta/DeltaQbeastSnapshot.scala | 24 ++++-- .../spark/delta/IndexStatusBuilder.scala | 65 ++++++++++----- .../io/qbeast/spark/delta/OTreeIndex.scala | 8 +- .../qbeast/spark/delta/writer/Compactor.scala | 18 +++-- .../scala/io/qbeast/spark/utils/Params.scala | 4 + .../spark/utils/QbeastDeltaStagingTest.scala | 81 +++++++++++++++++++ 8 files changed, 191 insertions(+), 37 deletions(-) create mode 100644 src/test/scala/io/qbeast/spark/utils/QbeastDeltaStagingTest.scala diff --git a/core/src/main/scala/io/qbeast/core/model/RevisionClasses.scala b/core/src/main/scala/io/qbeast/core/model/RevisionClasses.scala index 565ba6e2c..2f05018cc 100644 --- a/core/src/main/scala/io/qbeast/core/model/RevisionClasses.scala +++ b/core/src/main/scala/io/qbeast/core/model/RevisionClasses.scala @@ -4,7 +4,8 @@ import com.fasterxml.jackson.annotation.{JsonCreator, JsonValue} import com.fasterxml.jackson.databind.annotation.JsonSerialize import com.fasterxml.jackson.databind.annotation.JsonSerialize.Typing import io.qbeast.IISeq -import io.qbeast.core.transform.{Transformation, Transformer} +import io.qbeast.core.model.Revision.stagingRevisionID +import io.qbeast.core.transform.{HashTransformation, HashTransformer, Transformation, Transformer} import scala.collection.immutable.SortedMap @@ -41,6 +42,8 @@ final class QTableID(_id: String) extends Serializable { * Companion object for Revision */ object Revision { + val stagingRevisionID: RevisionID = -1 + val stagingIndexColumn: String = "" /** * Create a new first revision for a table @@ -60,7 +63,16 @@ object Revision { desiredCubeSize, columnTransformers, Vector.empty) + } + def stagingRevision(tableID: QTableID): Revision = { + Revision( + stagingRevisionID, + System.currentTimeMillis(), + tableID, + desiredCubeSize = -1, + Vector(HashTransformer(stagingIndexColumn, StringDataType)), + Vector(HashTransformation())) } } @@ -88,6 +100,8 @@ final case class Revision( extends Serializable { assert(columnTransformers != null || transformations != null) + val isStaging: Boolean = revisionID == stagingRevisionID + /** * * * Controls that the this revision indexes all and only the provided columns. @@ -117,7 +131,7 @@ final case class Revision( /** * returns the normalized values - * @param values + * @param values row values for the indexing columns * @return the normalized values */ def transform(values: IISeq[_]): IISeq[Double] = { @@ -193,8 +207,12 @@ case class IndexStatus( cubesStatuses: SortedMap[CubeId, CubeStatus] = SortedMap.empty) extends Serializable { - def addAnnouncements(newAnnouncedSet: Set[CubeId]): IndexStatus = - copy(announcedSet = announcedSet ++ newAnnouncedSet) + val isStaging: Boolean = revision.isStaging + + def addAnnouncements(newAnnouncedSet: Set[CubeId]): IndexStatus = { + if (isStaging) this + else copy(announcedSet = announcedSet ++ newAnnouncedSet) + } def cubesToOptimize: Set[CubeId] = announcedSet.diff(replicatedSet) diff --git a/src/main/scala/io/qbeast/spark/delta/DeltaMetadataWriter.scala b/src/main/scala/io/qbeast/spark/delta/DeltaMetadataWriter.scala index e442c2ce4..8bcf14ef0 100644 --- a/src/main/scala/io/qbeast/spark/delta/DeltaMetadataWriter.scala +++ b/src/main/scala/io/qbeast/spark/delta/DeltaMetadataWriter.scala @@ -137,7 +137,7 @@ private[delta] case class DeltaMetadataWriter( addFiles.map(_.copy(dataChange = !rearrangeOnly)) ++ deletedFiles.map(_.copy(dataChange = !rearrangeOnly)) } else { - newFiles ++ deletedFiles + addFiles ++ deletedFiles } if (isOptimizeOperation) { diff --git a/src/main/scala/io/qbeast/spark/delta/DeltaQbeastSnapshot.scala b/src/main/scala/io/qbeast/spark/delta/DeltaQbeastSnapshot.scala index abc4e31b4..7ab8ce4f6 100644 --- a/src/main/scala/io/qbeast/spark/delta/DeltaQbeastSnapshot.scala +++ b/src/main/scala/io/qbeast/spark/delta/DeltaQbeastSnapshot.scala @@ -4,7 +4,9 @@ package io.qbeast.spark.delta import io.qbeast.IISeq +import io.qbeast.core.model.Revision.stagingRevisionID import io.qbeast.core.model._ +import io.qbeast.spark.utils.Staging.isStagingFile import io.qbeast.spark.utils.{MetadataConfig, TagColumns} import org.apache.spark.sql.delta.Snapshot import org.apache.spark.sql.delta.actions.AddFile @@ -20,6 +22,17 @@ case class DeltaQbeastSnapshot(private val snapshot: Snapshot) extends QbeastSna def isInitial: Boolean = snapshot.version == -1 + private var stagingRevisionOption: Option[Revision] = None + + private def stagingRevision: Revision = { + if (stagingRevisionOption.isEmpty) { + val tableID = QTableID(snapshot.deltaLog.dataPath.toString) + val stagingRevision = Revision.stagingRevision(tableID) + stagingRevisionOption = Some(stagingRevision) + } + stagingRevisionOption.get + } + private val metadataMap: Map[String, String] = snapshot.metadata.configuration /** @@ -34,7 +47,7 @@ case class DeltaQbeastSnapshot(private val snapshot: Snapshot) extends QbeastSna val revision = mapper .readValue[Revision](json, classOf[Revision]) (revisionID, revision) - } + } + (stagingRevisionID -> stagingRevision) } /** @@ -61,7 +74,7 @@ case class DeltaQbeastSnapshot(private val snapshot: Snapshot) extends QbeastSna * @return revision identifier */ private val lastRevisionID: RevisionID = - metadataMap.getOrElse(MetadataConfig.lastRevisionID, "-1").toLong + metadataMap.getOrElse(MetadataConfig.lastRevisionID, stagingRevisionID.toString).toLong /** * Looks up for a revision with a certain identifier @@ -122,7 +135,8 @@ case class DeltaQbeastSnapshot(private val snapshot: Snapshot) extends QbeastSna * * @return an immutable Seq of Revision for qtable */ - override def loadAllRevisions: IISeq[Revision] = revisionsMap.values.toVector + override def loadAllRevisions: IISeq[Revision] = + (revisionsMap - stagingRevisionID).values.toVector /** * Obtain the last Revisions @@ -161,8 +175,8 @@ case class DeltaQbeastSnapshot(private val snapshot: Snapshot) extends QbeastSna * @return the Dataset of QbeastBlocks */ def loadRevisionBlocks(revisionID: RevisionID): Dataset[AddFile] = { - snapshot.allFiles - .where(TagColumns.revision === lit(revisionID.toString)) + if (getRevision(revisionID).isStaging) snapshot.allFiles.where(isStagingFile) + else snapshot.allFiles.where(TagColumns.revision === lit(revisionID.toString)) } } diff --git a/src/main/scala/io/qbeast/spark/delta/IndexStatusBuilder.scala b/src/main/scala/io/qbeast/spark/delta/IndexStatusBuilder.scala index bfde5b047..07ce22990 100644 --- a/src/main/scala/io/qbeast/spark/delta/IndexStatusBuilder.scala +++ b/src/main/scala/io/qbeast/spark/delta/IndexStatusBuilder.scala @@ -5,6 +5,7 @@ package io.qbeast.spark.delta import io.qbeast.core.model._ import io.qbeast.spark.delta.QbeastMetadataSQL._ +import io.qbeast.spark.utils.State.FLOODED import io.qbeast.spark.utils.TagColumns import org.apache.spark.sql.delta.actions.AddFile import org.apache.spark.sql.functions.{col, collect_list, lit, min, sum} @@ -42,34 +43,60 @@ private[delta] class IndexStatusBuilder( cubesStatuses = buildCubesStatuses) } + private def stagingCubeStatus(): CubeStatus = { + val root = revision.createCubeIdRoot() + val maxWeight = Weight.MaxValue + val blocks = revisionFiles + .collect() + .map(addFile => + QbeastBlock( + addFile.path, + revision.revisionID, + Weight.MinValue, + maxWeight, + FLOODED, + 0, + addFile.size, + addFile.modificationTime)) + .toIndexedSeq + CubeStatus(root, maxWeight, maxWeight.fraction, blocks) + } + /** * Returns the index state for the given space revision * @return Dataset containing cube information */ def buildCubesStatuses: SortedMap[CubeId, CubeStatus] = { - val spark = SparkSession.active val builder = SortedMap.newBuilder[CubeId, CubeStatus] - val rev = revision + if (revision.isStaging) { + val stagingCs = stagingCubeStatus() + builder += (stagingCs.cubeId -> stagingCs) - import spark.implicits._ - val ndims: Int = rev.transformations.size - revisionFiles - .groupBy(TagColumns.cube) - .agg( - min(weight(TagColumns.maxWeight)).as("maxWeight"), - sum(TagColumns.elementCount).as("elementCount"), - collect_list(qblock).as("files")) - .select( - createCube(col("cube"), lit(ndims)).as("cubeId"), - col("maxWeight"), - normalizeWeight(col("maxWeight"), col("elementCount"), lit(rev.desiredCubeSize)).as( - "normalizedWeight"), - col("files")) - .as[CubeStatus] - .collect() - .foreach(row => builder += row.cubeId -> row) + } else { + import spark.implicits._ + + val dimCount: Int = revision.transformations.size + val dcs = revision.desiredCubeSize + + revisionFiles + .groupBy(TagColumns.cube) + .agg( + min(weight(TagColumns.maxWeight)).as("maxWeight"), + sum(TagColumns.elementCount).as("elementCount"), + collect_list(qblock).as("files")) + .select( + createCube(col("cube"), lit(dimCount)).as("cubeId"), + col("maxWeight"), + normalizeWeight(col("maxWeight"), col("elementCount"), lit(dcs)) + .as("normalizedWeight"), + col("files")) + .as[CubeStatus] + .collect() + .foreach(row => builder += row.cubeId -> row) + + } builder.result() } diff --git a/src/main/scala/io/qbeast/spark/delta/OTreeIndex.scala b/src/main/scala/io/qbeast/spark/delta/OTreeIndex.scala index c2b8d6108..efc63b800 100644 --- a/src/main/scala/io/qbeast/spark/delta/OTreeIndex.scala +++ b/src/main/scala/io/qbeast/spark/delta/OTreeIndex.scala @@ -5,6 +5,7 @@ package io.qbeast.spark.delta import io.qbeast.core.model.QbeastBlock import io.qbeast.spark.index.query.{QueryExecutor, QuerySpecBuilder} +import io.qbeast.spark.utils.Staging.isStagingFile import org.apache.hadoop.fs.{FileStatus, Path} import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.expressions.{Expression, GenericInternalRow} @@ -49,8 +50,13 @@ case class OTreeIndex(index: TahoeLogFileIndex) extends FileIndex { queryExecutor.execute() } + /** + * Collect Staging AddFiles from _delta_log and convert them into FileStatuses. + * The output is merged with those built from QbeastBlocks. + * @return + */ private def stagingFiles: Seq[FileStatus] = { - snapshot.allFiles.where("tags IS NULL").collect().map { a: AddFile => + snapshot.allFiles.where(isStagingFile).collect().map { a: AddFile => new FileStatus( /* length */ a.size, /* isDir */ false, diff --git a/src/main/scala/io/qbeast/spark/delta/writer/Compactor.scala b/src/main/scala/io/qbeast/spark/delta/writer/Compactor.scala index 5fb3ba04a..3c8c34cac 100644 --- a/src/main/scala/io/qbeast/spark/delta/writer/Compactor.scala +++ b/src/main/scala/io/qbeast/spark/delta/writer/Compactor.scala @@ -59,13 +59,17 @@ case class Compactor( val revision = tableChanges.updatedRevision // Update the tags of the block with the information of the cubeBlocks - val tags: Map[String, String] = Map( - TagUtils.cube -> cubeId.string, - TagUtils.minWeight -> minWeight.value.toString, - TagUtils.maxWeight -> maxWeight.value.toString, - TagUtils.state -> state, - TagUtils.revision -> revision.revisionID.toString, - TagUtils.elementCount -> elementCount.toString) + val tags: Map[String, String] = + if (revision.isStaging) null + else { + Map( + TagUtils.cube -> cubeId.string, + TagUtils.minWeight -> minWeight.value.toString, + TagUtils.maxWeight -> maxWeight.value.toString, + TagUtils.state -> state, + TagUtils.revision -> revision.revisionID.toString, + TagUtils.elementCount -> elementCount.toString) + } val writtenPath = new Path(tableID.id, s"${UUID.randomUUID()}.parquet") val writer: OutputWriter = factory.newInstance( diff --git a/src/main/scala/io/qbeast/spark/utils/Params.scala b/src/main/scala/io/qbeast/spark/utils/Params.scala index c8feb4949..d3b1b0f91 100644 --- a/src/main/scala/io/qbeast/spark/utils/Params.scala +++ b/src/main/scala/io/qbeast/spark/utils/Params.scala @@ -40,3 +40,7 @@ object MetadataConfig { final val replicatedSet = "qbeast.replicatedSet" final val lastRevisionID = "qbeast.lastRevisionID" } + +object Staging { + final val isStagingFile = "tags IS NULL" +} diff --git a/src/test/scala/io/qbeast/spark/utils/QbeastDeltaStagingTest.scala b/src/test/scala/io/qbeast/spark/utils/QbeastDeltaStagingTest.scala new file mode 100644 index 000000000..5df8bdedd --- /dev/null +++ b/src/test/scala/io/qbeast/spark/utils/QbeastDeltaStagingTest.scala @@ -0,0 +1,81 @@ +package io.qbeast.spark.utils + +import io.qbeast.core.model.Revision.stagingRevisionID +import io.qbeast.spark.delta.DeltaQbeastSnapshot +import io.qbeast.spark.{QbeastIntegrationTestSpec, QbeastTable} +import org.apache.spark.sql.delta.DeltaLog + +class QbeastDeltaStagingTest extends QbeastIntegrationTestSpec { + it should "read a delta table as an entirely staged qbeast table" in + withSparkAndTmpDir((spark, tmpDir) => { + val df = loadTestData(spark).limit(10000) + df.write.format("delta").save(tmpDir) + + val numStagingRows = spark.read.format("qbeast").load(tmpDir).count + numStagingRows shouldBe 10000L + }) + + it should "not analyze or optimize the staging revision" in withSparkAndTmpDir( + (spark, tmpDir) => { + val df = loadTestData(spark).limit(10000) + df.write.format("delta").save(tmpDir) + + val table = QbeastTable.forPath(spark, tmpDir) + table.analyze(stagingRevisionID) + table.optimize(stagingRevisionID) + + val allElements = spark.read.parquet(tmpDir).count + allElements shouldBe 10000 + + val snapshot = DeltaLog.forTable(spark, tmpDir).snapshot + val qbeastSnapshot = DeltaQbeastSnapshot(snapshot) + val stagingIndexStatus = qbeastSnapshot.loadIndexStatus(stagingRevisionID) + + stagingIndexStatus.replicatedOrAnnouncedSet.isEmpty shouldBe true + }) + + it should "correctly compact the staging revision" in withExtendedSparkAndTmpDir( + sparkConfWithSqlAndCatalog.set("spark.qbeast.compact.minFileSize", "1")) { (spark, tmpDir) => + { + val df = loadTestData(spark) + + df.repartition(20).write.format("delta").save(tmpDir) + + val deltaLog = DeltaLog.forTable(spark, tmpDir) + val snapshot = deltaLog.snapshot + val numFilesBefore = snapshot.numOfFiles + + val table = QbeastTable.forPath(spark, tmpDir) + table.compact(stagingRevisionID) + + deltaLog.update() + val numFilesAfter = deltaLog.snapshot.numOfFiles + numFilesAfter shouldBe <(numFilesBefore) + + val deltaCount = spark.read.format("delta").load(tmpDir).count() + val qbeastCount = spark.read.format("qbeast").load(tmpDir).count() + + deltaCount shouldBe 99986 + qbeastCount shouldBe 99986 + } + } + + it should "sample a qbeast staging table correctly" in withSparkAndTmpDir((spark, tmpDir) => { +// val dataSize = 99986 +// val df = loadTestData(spark) // 99986 +// +// df.write.format("delta").save(tmpDir) +// val qdf = spark.read.format("qbeast").load(tmpDir) +// +// // We allow a 1% of tolerance in the sampling +// val tolerance = 0.01 +// List(0.1, 0.2, 0.5, 0.7, 0.99).foreach(f => { +// val result = qdf +// .sample(withReplacement = false, f) +// .count() +// .toDouble +// +// result shouldBe (dataSize * f) +- dataSize * f * tolerance + }) + +} From e48732a72d37b613083631bc6a285080c2957999 Mon Sep 17 00:00:00 2001 From: Jiawei Date: Thu, 19 Jan 2023 13:13:57 +0100 Subject: [PATCH 36/76] WIP, Use all columns for staging sampling --- .../spark/internal/rules/SampleRule.scala | 8 +++--- .../spark/utils/QbeastDeltaStagingTest.scala | 27 +++++++++---------- 2 files changed, 17 insertions(+), 18 deletions(-) diff --git a/src/main/scala/io/qbeast/spark/internal/rules/SampleRule.scala b/src/main/scala/io/qbeast/spark/internal/rules/SampleRule.scala index 37a7f638a..0c8474c54 100644 --- a/src/main/scala/io/qbeast/spark/internal/rules/SampleRule.scala +++ b/src/main/scala/io/qbeast/spark/internal/rules/SampleRule.scala @@ -3,7 +3,7 @@ */ package io.qbeast.spark.internal.rules -import io.qbeast.core.model.Weight +import io.qbeast.core.model.{Weight, WeightRange} import io.qbeast.spark.internal.expressions.QbeastMurmur3Hash import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.expressions.{And, GreaterThanOrEqual, LessThan, Literal} @@ -11,8 +11,8 @@ import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan, Project import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.datasources.LogicalRelation import org.apache.spark.sql.SparkSession -import io.qbeast.core.model.WeightRange import io.qbeast.IndexedColumns +import io.qbeast.core.model.Revision.stagingIndexColumn import org.apache.spark.sql.execution.datasources.HadoopFsRelation import io.qbeast.spark.delta.OTreeIndex @@ -48,8 +48,10 @@ class SampleRule(spark: SparkSession) extends Rule[LogicalPlan] with Logging { val weightRange = extractWeightRange(sample) + val isStaging = indexedColumns.size == 1 && indexedColumns.head == stagingIndexColumn val columns = - indexedColumns.map(c => logicalRelation.output.find(_.name == c).get) + if (isStaging) logicalRelation.output + else indexedColumns.map(c => logicalRelation.output.find(_.name == c).get) val qbeastHash = new QbeastMurmur3Hash(columns) Filter( diff --git a/src/test/scala/io/qbeast/spark/utils/QbeastDeltaStagingTest.scala b/src/test/scala/io/qbeast/spark/utils/QbeastDeltaStagingTest.scala index 5df8bdedd..9ef96b860 100644 --- a/src/test/scala/io/qbeast/spark/utils/QbeastDeltaStagingTest.scala +++ b/src/test/scala/io/qbeast/spark/utils/QbeastDeltaStagingTest.scala @@ -61,21 +61,18 @@ class QbeastDeltaStagingTest extends QbeastIntegrationTestSpec { } it should "sample a qbeast staging table correctly" in withSparkAndTmpDir((spark, tmpDir) => { -// val dataSize = 99986 -// val df = loadTestData(spark) // 99986 -// -// df.write.format("delta").save(tmpDir) -// val qdf = spark.read.format("qbeast").load(tmpDir) -// -// // We allow a 1% of tolerance in the sampling -// val tolerance = 0.01 -// List(0.1, 0.2, 0.5, 0.7, 0.99).foreach(f => { -// val result = qdf -// .sample(withReplacement = false, f) -// .count() -// .toDouble -// -// result shouldBe (dataSize * f) +- dataSize * f * tolerance + val dataSize = 99986 + val df = loadTestData(spark) // 99986 + + df.write.format("delta").save(tmpDir) + val qdf = spark.read.format("qbeast").load(tmpDir) + + // We allow a 1% of tolerance in the sampling + val tolerance = 0.01 + List(0.1, 0.2, 0.5, 0.7, 0.99).foreach(f => { + val result = qdf.sample(withReplacement = false, f).count().toDouble + result shouldBe (dataSize * f) +- dataSize * f * tolerance + }) }) } From fbb84c04d661164c623a260491fbb71de2b14f29 Mon Sep 17 00:00:00 2001 From: Jiawei Date: Fri, 20 Jan 2023 17:59:39 +0100 Subject: [PATCH 37/76] Rely on user input for source format, direct metadata update --- .../commands/ConvertToQbeastCommand.scala | 223 +++++------------- 1 file changed, 55 insertions(+), 168 deletions(-) diff --git a/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala b/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala index 2fca3b8ca..f36ac7b79 100644 --- a/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala +++ b/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala @@ -3,93 +3,54 @@ */ package io.qbeast.spark.internal.commands -import io.qbeast.IISeq import io.qbeast.core.model._ import io.qbeast.core.transform._ -import io.qbeast.spark.delta.{DeltaQbeastLog, SparkDeltaMetadataManager} +import io.qbeast.spark.delta.DeltaQbeastSnapshot import io.qbeast.spark.index.SparkRevisionFactory -import io.qbeast.spark.internal.commands.ConvertToQbeastCommand.{ - dataTypeMinMax, - extractQbeastTag, - sparkToSqlTypeNames -} -import io.qbeast.spark.utils.{State, TagUtils} -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.Path +import io.qbeast.spark.internal.commands.ConvertToQbeastCommand.dataTypeMinMax +import io.qbeast.spark.utils.MetadataConfig import org.apache.http.annotation.Experimental -import org.apache.parquet.hadoop.ParquetFileReader -import org.apache.parquet.hadoop.util.HadoopInputFile +import org.apache.spark.internal.Logging import org.apache.spark.qbeast.config.DEFAULT_CUBE_SIZE -import org.apache.spark.sql.delta.actions.{AddFile, FileAction} -import org.apache.spark.sql.{Row, SparkSession} -import org.apache.spark.sql.delta.{DeltaLog, Snapshot} +import org.apache.spark.sql.delta.DeltaLog +import org.apache.spark.sql.delta.metering.DeltaLogging import org.apache.spark.sql.execution.command.LeafRunnableCommand import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.{Row, SparkSession} -import scala.util.matching.Regex +import java.util.Locale @Experimental case class ConvertToQbeastCommand( - path: String, + identifier: String, columnsToIndex: Seq[String], cubeSize: Int = DEFAULT_CUBE_SIZE, - partitionColumns: Seq[String] = Seq.empty) - extends LeafRunnableCommand { + partitionColumns: Option[String]) + extends LeafRunnableCommand + with Logging + with DeltaLogging { - private val isPartitioned: Boolean = partitionColumns.nonEmpty + private val isPartitioned: Boolean = partitionColumns.isDefined /** * Format inference for the input table. If partition columns are provided, * the format is assumed to be parquet. Any unsupported format is considered * as parquet and is detected when trying to convert it into delta. - * @param sparkSession SparkSession to use - * @return */ - private def resolveTableFormat(sparkSession: SparkSession): (String, StructType) = { - val deltaLog = DeltaLog.forTable(sparkSession, path) - val qbeastSnapshot = DeltaQbeastLog(deltaLog).qbeastSnapshot - val schema = deltaLog.snapshot.schema - - val isDelta = deltaLog.tableExists - // The first revisionID for a converted table is 0, - // while for one that's written in the conventional fashion is 1. - val isQbeast = - isDelta && (qbeastSnapshot.existsRevision(0) || qbeastSnapshot.existsRevision(1)) - - if (isQbeast) { - ("qbeast", schema) - } else if (isDelta) { - ("delta", schema) - } else if (isPartitioned) { - // Partitioned parquet, table schema is required for its conversion into delta - ("parquet", sparkSession.read.parquet(path).schema) - } else { - // Parquet, or any other unsupported format, schema.isEmpty but we don't need it - ("parquet", schema) + private def resolveTableFormat(): (String, String) = + identifier.split("\\.") match { + case Array(f, p) => (f.toLowerCase(Locale.ROOT), p) + case _ => throw new RuntimeException(s"Table doesn't exists at $identifier") } - } - - // scalastyle:off println - def logConsole(line: String): Unit = println(line) - // scalastyle:on println /** * Convert the parquet table using ConvertToDeltaCommand from Delta Lake. * Any unsupported format will cause a SparkException error. - * @param spark SparkSession to use */ - private def convertParquetToDelta(spark: SparkSession, schema: StructType): Unit = { - if (isPartitioned) { - val colsAndTypes = - partitionColumns.map(colName => { - val typeName = schema(colName).dataType.typeName - val sqlTypeName = sparkToSqlTypeNames.getOrElse(typeName, typeName) - colName + " " + sqlTypeName - }) - spark.sql( - s"CONVERT TO DELTA parquet.`$path` PARTITIONED BY (${colsAndTypes.mkString(", ")})") - } else { - spark.sql(s"CONVERT TO DELTA parquet.`$path`") + private def convertParquetToDelta(spark: SparkSession, path: String): Unit = { + if (!isPartitioned) spark.sql(s"CONVERT TO DELTA parquet.`$path`") + else { + spark.sql(s"CONVERT TO DELTA parquet.`$path` PARTITIONED BY (${partitionColumns.get})") } } @@ -99,7 +60,7 @@ case class ConvertToQbeastCommand( * @param schema table schema * @return */ - private def initializeRevision(schema: StructType): Revision = { + private def initializeRevision(path: String, schema: StructType): Revision = { val revision = SparkRevisionFactory.createNewRevision( QTableID(path), @@ -116,69 +77,49 @@ case class ConvertToQbeastCommand( revision.copy(transformations = transformations) } - private def createQbeastActions(snapshot: Snapshot, revision: Revision): IISeq[FileAction] = { - val allFiles = snapshot.allFiles.collect() - - allFiles - .map(addFile => { - val metadataTag = extractQbeastTag(addFile, revision) - addFile.copy(tags = metadataTag) - }) - .toIndexedSeq - } + private def isQbeastFormat(deltaLog: DeltaLog): Boolean = { + val qbeastSnapshot = DeltaQbeastSnapshot(deltaLog.snapshot) + val isDelta = deltaLog.tableExists - private def getTableChanges(revision: Revision, sparkSession: SparkSession): TableChanges = { - val root = revision.createCubeIdRoot() - - BroadcastedTableChanges( - isNewRevision = true, - isOptimizeOperation = false, - revision, - Set.empty[CubeId], - Set.empty[CubeId], - sparkSession.sparkContext.broadcast(Map(root -> State.FLOODED)), - sparkSession.sparkContext.broadcast(Map(root -> Weight.MaxValue))) + isDelta && qbeastSnapshot.loadAllRevisions.nonEmpty } - override def run(sparkSession: SparkSession): Seq[Row] = { - val (fileFormat, sourceSchema) = resolveTableFormat(sparkSession) - - fileFormat match { - // Idempotent conversion - case "qbeast" => - logConsole("The table you are trying to convert is already a qbeast table") - return Seq.empty[Row] - // Convert parquet to delta - case "parquet" => convertParquetToDelta(sparkSession, sourceSchema) - // delta, do nothing - case _ => - } - - // Convert delta to qbeast - val snapshot = DeltaLog.forTable(sparkSession, path).snapshot - val revision = initializeRevision(snapshot.schema) + override def run(spark: SparkSession): Seq[Row] = { + val (fileFormat, path) = resolveTableFormat() - SparkDeltaMetadataManager.updateWithTransaction( - revision.tableID, - snapshot.schema, - append = true) { - val tableChanges = getTableChanges(revision, sparkSession) - val newFiles = createQbeastActions(snapshot, revision) - - (tableChanges, newFiles) + val deltaLog = DeltaLog.forTable(spark, path) + if (isQbeastFormat(deltaLog)) { + logInfo("The table you are trying to convert is already a qbeast table") + } else { + fileFormat match { + // Convert parquet to delta + case "parquet" => convertParquetToDelta(spark, path) + case _ => + } + + // Convert delta to qbeast + val txn = deltaLog.startTransaction() + val revision = initializeRevision(path, deltaLog.snapshot.schema) + val revisionID = revision.revisionID + val isOverwritingSchema = txn.metadata.partitionColumns.nonEmpty + val updatedConf = txn.metadata.configuration + .updated(MetadataConfig.lastRevisionID, revisionID.toString) + .updated( + s"${MetadataConfig.revision}.$revisionID" -> mapper.writeValueAsString(revisionID)) + val newMetadata = + txn.metadata.copy(configuration = updatedConf, partitionColumns = Seq.empty) + txn.updateMetadata(newMetadata) + if (isOverwritingSchema) recordDeltaEvent(txn.deltaLog, "delta.ddl.overwriteSchema") } - Seq.empty[Row] } } object ConvertToQbeastCommand { - private val numRecordsPattern: Regex = """"numRecords":(\d+),""".r - - private val intMinMax = ColumnMinMax(Int.MinValue, Int.MaxValue) - private val doubleMinMax = ColumnMinMax(-1e308, 1e307) - private val longMinMax = ColumnMinMax(Long.MinValue, Long.MaxValue) + private val intMinMax = ColumnMinMax(Int.MaxValue, Int.MinValue) + private val doubleMinMax = ColumnMinMax(Double.MaxValue, Double.MinValue) + private val longMinMax = ColumnMinMax(Long.MaxValue, Long.MinValue) private val dataTypeMinMax = Map( DoubleDataType -> doubleMinMax, @@ -189,60 +130,6 @@ object ConvertToQbeastCommand { TimestampDataType -> longMinMax, DateDataType -> longMinMax) - /** - * Map a Spark data type name to a Spark SQL type name. Currently only the following - * are supported for conversion. Any other data type are inferred as String after - * being used as Partition Columns. Timestamp but this is not supported as a partition - * column by delta conversion. - */ - private val sparkToSqlTypeNames = Map( - "integer" -> "INT", - "double" -> "DOUBLE", - "long" -> "BIGINT", - "date" -> "DATE", - "timestamp" -> "TIMESTAMP", - "string" -> "STRING") - - /** - * Extract record count from a parquet file metadata. - * @param parquetFilePath target parquet file path - * @return - */ - def extractParquetFileCount(parquetFilePath: Path): String = { - val file = HadoopInputFile.fromPath(parquetFilePath, new Configuration()) - val reader = ParquetFileReader.open(file) - reader.getRecordCount.toString - } - - /** - * Extract Qbeast metadata for an AddFile. - * @param addFile AddFile to be converted into a qbeast block for the root - * @param revision the conversion revision to use, revisionID = 0 - * @return - */ - def extractQbeastTag(addFile: AddFile, revision: Revision): Map[String, String] = { - val elementCount = addFile.stats match { - case stats: String => - numRecordsPattern.findFirstMatchIn(stats) match { - case Some(matching) => matching.group(1) - // stats does not contain record count, proceed extraction using parquet metadata - case _ => - extractParquetFileCount(new Path(revision.tableID.id, addFile.path)) - } - // AddFile entries with no 'stats' field, proceed extraction using parquet metadata - case _ => - extractParquetFileCount(new Path(revision.tableID.id, addFile.path)) - } - - Map( - TagUtils.cube -> "", - TagUtils.minWeight -> Weight.MinValue.value.toString, - TagUtils.maxWeight -> (Weight.MaxValue.value - 1).toString, - TagUtils.state -> State.FLOODED, - TagUtils.revision -> revision.revisionID.toString, - TagUtils.elementCount -> elementCount) - } - } case class ColumnMinMax(minValue: Any, maxValue: Any) From 1da9bff254f0a4ff8dc9cca7ba2b069ee0deaa75 Mon Sep 17 00:00:00 2001 From: Jiawei Date: Mon, 23 Jan 2023 11:02:49 +0100 Subject: [PATCH 38/76] Add comment --- .../commands/ConvertToQbeastCommand.scala | 39 ++++++++++--------- 1 file changed, 21 insertions(+), 18 deletions(-) diff --git a/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala b/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala index f36ac7b79..5eb2969a4 100644 --- a/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala +++ b/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala @@ -8,7 +8,7 @@ import io.qbeast.core.transform._ import io.qbeast.spark.delta.DeltaQbeastSnapshot import io.qbeast.spark.index.SparkRevisionFactory import io.qbeast.spark.internal.commands.ConvertToQbeastCommand.dataTypeMinMax -import io.qbeast.spark.utils.MetadataConfig +import io.qbeast.spark.utils.MetadataConfig.{lastRevisionID, revision} import org.apache.http.annotation.Experimental import org.apache.spark.internal.Logging import org.apache.spark.qbeast.config.DEFAULT_CUBE_SIZE @@ -32,11 +32,6 @@ case class ConvertToQbeastCommand( private val isPartitioned: Boolean = partitionColumns.isDefined - /** - * Format inference for the input table. If partition columns are provided, - * the format is assumed to be parquet. Any unsupported format is considered - * as parquet and is detected when trying to convert it into delta. - */ private def resolveTableFormat(): (String, String) = identifier.split("\\.") match { case Array(f, p) => (f.toLowerCase(Locale.ROOT), p) @@ -45,18 +40,20 @@ case class ConvertToQbeastCommand( /** * Convert the parquet table using ConvertToDeltaCommand from Delta Lake. - * Any unsupported format will cause a SparkException error. */ private def convertParquetToDelta(spark: SparkSession, path: String): Unit = { - if (!isPartitioned) spark.sql(s"CONVERT TO DELTA parquet.`$path`") - else { - spark.sql(s"CONVERT TO DELTA parquet.`$path` PARTITIONED BY (${partitionColumns.get})") - } + val conversionCommand = + if (!isPartitioned) s"CONVERT TO DELTA parquet.`$path`" + else s"CONVERT TO DELTA parquet.`$path` PARTITIONED BY (${partitionColumns.get})" + + spark.sql(conversionCommand) } /** - * Initialize Revision for table conversion. - * The smallest RevisionID for a converted table is 0. + * Initialize Revision for table conversion. The RevisionID for a converted table is 0. + * Invalid dimension ranges are used to make sure the Transformations are always superseded. + * + * e.g. LinearTransformation(minNumber = Int.MaxValue, maxNumber = Int.MinValue) * @param schema table schema * @return */ @@ -99,15 +96,21 @@ case class ConvertToQbeastCommand( // Convert delta to qbeast val txn = deltaLog.startTransaction() - val revision = initializeRevision(path, deltaLog.snapshot.schema) - val revisionID = revision.revisionID + + val convRevision = initializeRevision(path, deltaLog.snapshot.schema) + val revisionID = convRevision.revisionID + + // If the table has partition columns, its conversion to qbeast will + // remove them by overwriting the schema val isOverwritingSchema = txn.metadata.partitionColumns.nonEmpty + + // Add revision map as a metadata entry val updatedConf = txn.metadata.configuration - .updated(MetadataConfig.lastRevisionID, revisionID.toString) - .updated( - s"${MetadataConfig.revision}.$revisionID" -> mapper.writeValueAsString(revisionID)) + .updated(lastRevisionID, revisionID.toString) + .updated(s"$revision.$revisionID", mapper.writeValueAsString(convRevision)) val newMetadata = txn.metadata.copy(configuration = updatedConf, partitionColumns = Seq.empty) + txn.updateMetadata(newMetadata) if (isOverwritingSchema) recordDeltaEvent(txn.deltaLog, "delta.ddl.overwriteSchema") } From 6e00d91e0076d33e863781492ef6ea110014b1c0 Mon Sep 17 00:00:00 2001 From: Jiawei Date: Mon, 23 Jan 2023 15:06:03 +0100 Subject: [PATCH 39/76] Use the indexing columns from the latest revision for online sampling --- .../scala/io/qbeast/spark/internal/rules/SampleRule.scala | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/src/main/scala/io/qbeast/spark/internal/rules/SampleRule.scala b/src/main/scala/io/qbeast/spark/internal/rules/SampleRule.scala index 0c8474c54..659b423ae 100644 --- a/src/main/scala/io/qbeast/spark/internal/rules/SampleRule.scala +++ b/src/main/scala/io/qbeast/spark/internal/rules/SampleRule.scala @@ -12,7 +12,6 @@ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.datasources.LogicalRelation import org.apache.spark.sql.SparkSession import io.qbeast.IndexedColumns -import io.qbeast.core.model.Revision.stagingIndexColumn import org.apache.spark.sql.execution.datasources.HadoopFsRelation import io.qbeast.spark.delta.OTreeIndex @@ -48,10 +47,8 @@ class SampleRule(spark: SparkSession) extends Rule[LogicalPlan] with Logging { val weightRange = extractWeightRange(sample) - val isStaging = indexedColumns.size == 1 && indexedColumns.head == stagingIndexColumn val columns = - if (isStaging) logicalRelation.output - else indexedColumns.map(c => logicalRelation.output.find(_.name == c).get) + indexedColumns.map(c => logicalRelation.output.find(_.name == c).get) val qbeastHash = new QbeastMurmur3Hash(columns) Filter( From df27cd302f261f017eaada62497406f18895bbfc Mon Sep 17 00:00:00 2001 From: Jiawei Date: Mon, 23 Jan 2023 15:07:25 +0100 Subject: [PATCH 40/76] Use EmptyTransformation/ers for the conversion revision --- .../core/transform/EmptyTransformation.scala | 13 +++ .../core/transform/EmptyTransformer.scala | 22 +++++ .../commands/ConvertToQbeastCommand.scala | 82 ++++++++----------- 3 files changed, 68 insertions(+), 49 deletions(-) create mode 100644 core/src/main/scala/io/qbeast/core/transform/EmptyTransformation.scala create mode 100644 core/src/main/scala/io/qbeast/core/transform/EmptyTransformer.scala diff --git a/core/src/main/scala/io/qbeast/core/transform/EmptyTransformation.scala b/core/src/main/scala/io/qbeast/core/transform/EmptyTransformation.scala new file mode 100644 index 000000000..166f59c1e --- /dev/null +++ b/core/src/main/scala/io/qbeast/core/transform/EmptyTransformation.scala @@ -0,0 +1,13 @@ +package io.qbeast.core.transform + +/** + * An empty Transformation meant for empty revisions + */ +case class EmptyTransformation() extends Transformation { + + override def transform(value: Any): Double = 0d + + override def isSupersededBy(newTransformation: Transformation): Boolean = true + + override def merge(other: Transformation): Transformation = this +} diff --git a/core/src/main/scala/io/qbeast/core/transform/EmptyTransformer.scala b/core/src/main/scala/io/qbeast/core/transform/EmptyTransformer.scala new file mode 100644 index 000000000..a9b2631ac --- /dev/null +++ b/core/src/main/scala/io/qbeast/core/transform/EmptyTransformer.scala @@ -0,0 +1,22 @@ +package io.qbeast.core.transform + +import io.qbeast.core.model.QDataType + +object EmptyTransformer extends TransformerType { + override def transformerSimpleName: String = "empty" + + override def apply(columnName: String, dataType: QDataType): Transformer = + EmptyTransformer(columnName) + +} + +/** + * An empty Transformer meant for empty revisions + */ +case class EmptyTransformer(columnName: String) extends Transformer { + override protected def transformerType: TransformerType = EmptyTransformer + + override def stats: ColumnStats = NoColumnStats + + override def makeTransformation(row: String => Any): Transformation = EmptyTransformation() +} diff --git a/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala b/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala index 5eb2969a4..a387c69d8 100644 --- a/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala +++ b/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala @@ -6,17 +6,17 @@ package io.qbeast.spark.internal.commands import io.qbeast.core.model._ import io.qbeast.core.transform._ import io.qbeast.spark.delta.DeltaQbeastSnapshot -import io.qbeast.spark.index.SparkRevisionFactory -import io.qbeast.spark.internal.commands.ConvertToQbeastCommand.dataTypeMinMax import io.qbeast.spark.utils.MetadataConfig.{lastRevisionID, revision} import org.apache.http.annotation.Experimental import org.apache.spark.internal.Logging import org.apache.spark.qbeast.config.DEFAULT_CUBE_SIZE +import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.delta.DeltaLog +import org.apache.spark.sql.delta.DeltaOperations.Convert import org.apache.spark.sql.delta.metering.DeltaLogging import org.apache.spark.sql.execution.command.LeafRunnableCommand import org.apache.spark.sql.types.StructType -import org.apache.spark.sql.{Row, SparkSession} +import org.apache.spark.sql.{AnalysisExceptionFactory, Row, SparkSession} import java.util.Locale @@ -25,16 +25,17 @@ case class ConvertToQbeastCommand( identifier: String, columnsToIndex: Seq[String], cubeSize: Int = DEFAULT_CUBE_SIZE, - partitionColumns: Option[String]) + partitionColumns: Option[String] = None) extends LeafRunnableCommand with Logging with DeltaLogging { private val isPartitioned: Boolean = partitionColumns.isDefined - private def resolveTableFormat(): (String, String) = + private def resolveTableFormat(spark: SparkSession): (String, TableIdentifier) = identifier.split("\\.") match { - case Array(f, p) => (f.toLowerCase(Locale.ROOT), p) + case Array(f, p) => + (f.toLowerCase(Locale.ROOT), spark.sessionState.sqlParser.parseTableIdentifier(p)) case _ => throw new RuntimeException(s"Table doesn't exists at $identifier") } @@ -43,38 +44,34 @@ case class ConvertToQbeastCommand( */ private def convertParquetToDelta(spark: SparkSession, path: String): Unit = { val conversionCommand = - if (!isPartitioned) s"CONVERT TO DELTA parquet.`$path`" - else s"CONVERT TO DELTA parquet.`$path` PARTITIONED BY (${partitionColumns.get})" + if (!isPartitioned) s"CONVERT TO DELTA parquet.$path" + else s"CONVERT TO DELTA parquet.$path PARTITIONED BY (${partitionColumns.get})" spark.sql(conversionCommand) } /** * Initialize Revision for table conversion. The RevisionID for a converted table is 0. - * Invalid dimension ranges are used to make sure the Transformations are always superseded. - * - * e.g. LinearTransformation(minNumber = Int.MaxValue, maxNumber = Int.MinValue) + * EmptyTransformers and EmptyTransformations are used. This Revision should always be + * superseded. * @param schema table schema * @return */ - private def initializeRevision(path: String, schema: StructType): Revision = { - val revision = - SparkRevisionFactory.createNewRevision( - QTableID(path), - schema, - Map("columnsToIndex" -> columnsToIndex.mkString(","), "cubeSize" -> cubeSize.toString)) - - val transformations = revision.columnTransformers.map { - case LinearTransformer(_, dataType: OrderedDataType) => - val minMax = dataTypeMinMax(dataType) - LinearTransformation(minMax.minValue, minMax.maxValue, dataType) - case HashTransformer(_, _) => HashTransformation() - }.toIndexedSeq - - revision.copy(transformations = transformations) + private def emptyRevision(path: String, schema: StructType): Revision = { + val transformers = columnsToIndex.map(s => EmptyTransformer(s)).toIndexedSeq + val transformations = transformers.map(_.makeTransformation(r => r)) + + Revision( + 0, + System.currentTimeMillis(), + QTableID(path), + cubeSize, + transformers, + transformations) } - private def isQbeastFormat(deltaLog: DeltaLog): Boolean = { + private def isQbeastFormat(spark: SparkSession, path: String): Boolean = { + val deltaLog = DeltaLog.forTable(spark, path) val qbeastSnapshot = DeltaQbeastSnapshot(deltaLog.snapshot) val isDelta = deltaLog.tableExists @@ -82,22 +79,24 @@ case class ConvertToQbeastCommand( } override def run(spark: SparkSession): Seq[Row] = { - val (fileFormat, path) = resolveTableFormat() + val (fileFormat, tableId) = resolveTableFormat(spark) - val deltaLog = DeltaLog.forTable(spark, path) - if (isQbeastFormat(deltaLog)) { + if (isQbeastFormat(spark, tableId.table)) { logInfo("The table you are trying to convert is already a qbeast table") } else { fileFormat match { // Convert parquet to delta - case "parquet" => convertParquetToDelta(spark, path) - case _ => + case "parquet" => convertParquetToDelta(spark, tableId.quotedString) + case "delta" => + case _ => throw AnalysisExceptionFactory.create(s"Unsupported file format: $fileFormat") } // Convert delta to qbeast + val deltaLog = DeltaLog.forTable(spark, tableId.table) + val txn = deltaLog.startTransaction() - val convRevision = initializeRevision(path, deltaLog.snapshot.schema) + val convRevision = emptyRevision(tableId.table, deltaLog.snapshot.schema) val revisionID = convRevision.revisionID // If the table has partition columns, its conversion to qbeast will @@ -113,26 +112,11 @@ case class ConvertToQbeastCommand( txn.updateMetadata(newMetadata) if (isOverwritingSchema) recordDeltaEvent(txn.deltaLog, "delta.ddl.overwriteSchema") + txn.commit(Seq.empty, Convert(0, Seq.empty, collectStats = false, None)) } Seq.empty[Row] } } -object ConvertToQbeastCommand { - private val intMinMax = ColumnMinMax(Int.MaxValue, Int.MinValue) - private val doubleMinMax = ColumnMinMax(Double.MaxValue, Double.MinValue) - private val longMinMax = ColumnMinMax(Long.MaxValue, Long.MinValue) - - private val dataTypeMinMax = Map( - DoubleDataType -> doubleMinMax, - IntegerDataType -> intMinMax, - LongDataType -> longMinMax, - FloatDataType -> doubleMinMax, - DecimalDataType -> doubleMinMax, - TimestampDataType -> longMinMax, - DateDataType -> longMinMax) - -} - case class ColumnMinMax(minValue: Any, maxValue: Any) From a4c89195cad1d9bcb3e2e6b747ab0141d5cd0fdc Mon Sep 17 00:00:00 2001 From: Jiawei Date: Mon, 23 Jan 2023 15:08:30 +0100 Subject: [PATCH 41/76] Test Conversion command. Analyze command should not change ANNOUNCED set when index is empty --- .../scala/io/qbeast/spark/QbeastTable.scala | 2 +- .../spark/index/SparkOTreeManager.scala | 25 +- .../spark/utils/ConvertToQbeastTest.scala | 275 +++++------------- 3 files changed, 90 insertions(+), 212 deletions(-) diff --git a/src/main/scala/io/qbeast/spark/QbeastTable.scala b/src/main/scala/io/qbeast/spark/QbeastTable.scala index 2d7b22962..37871c5b5 100644 --- a/src/main/scala/io/qbeast/spark/QbeastTable.scala +++ b/src/main/scala/io/qbeast/spark/QbeastTable.scala @@ -103,7 +103,7 @@ class QbeastTable private ( val allCubeStatuses = qbeastSnapshot.loadLatestIndexStatus.cubesStatuses val cubeCount = allCubeStatuses.size - val depth = allCubeStatuses.map(_._1.depth).max + val depth = if (cubeCount == 0) 0 else allCubeStatuses.map(_._1.depth).max val rowCount = allCubeStatuses.flatMap(_._2.files.map(_.elementCount)).sum val dimensionCount = indexedColumns().size diff --git a/src/main/scala/io/qbeast/spark/index/SparkOTreeManager.scala b/src/main/scala/io/qbeast/spark/index/SparkOTreeManager.scala index a6acf9988..2474b7097 100644 --- a/src/main/scala/io/qbeast/spark/index/SparkOTreeManager.scala +++ b/src/main/scala/io/qbeast/spark/index/SparkOTreeManager.scala @@ -42,20 +42,23 @@ object SparkOTreeManager extends IndexManager[DataFrame] with Serializable { // PRIVATE METHODS // private def findCubesToOptimize(indexStatus: IndexStatus): IISeq[CubeId] = { - val overflowedSet = indexStatus.overflowedSet - val replicatedSet = indexStatus.replicatedSet + if (indexStatus.cubesStatuses.isEmpty) Vector.empty + else { + val overflowedSet = indexStatus.overflowedSet + val replicatedSet = indexStatus.replicatedSet - val cubesToOptimize = overflowedSet - .filter(cube => { - !replicatedSet.contains(cube) && (cube.parent match { - case None => true - case Some(p) => replicatedSet.contains(p) + val cubesToOptimize = overflowedSet + .filter(cube => { + !replicatedSet.contains(cube) && (cube.parent match { + case None => true + case Some(p) => replicatedSet.contains(p) + }) }) - }) - if (cubesToOptimize.isEmpty && replicatedSet.isEmpty) { - Seq(indexStatus.revision.createCubeIdRoot()).toIndexedSeq - } else cubesToOptimize.toIndexedSeq + if (cubesToOptimize.isEmpty && replicatedSet.isEmpty) { + Seq(indexStatus.revision.createCubeIdRoot()).toIndexedSeq + } else cubesToOptimize.toIndexedSeq + } } private def index( diff --git a/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala b/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala index ddc90e974..294fb5c09 100644 --- a/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala +++ b/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala @@ -1,152 +1,123 @@ package io.qbeast.spark.utils -import io.qbeast.core.model.{CubeId, QTableID} -import io.qbeast.spark.index.SparkRevisionFactory -import io.qbeast.spark.{QbeastIntegrationTestSpec, QbeastTable} import io.qbeast.spark.internal.commands.ConvertToQbeastCommand -import org.apache.spark.{SparkConf, SparkException} -import org.apache.spark.qbeast.config.DEFAULT_CUBE_SIZE -import org.apache.spark.sql.delta.DeltaLog -import org.apache.spark.sql.{DataFrame, SparkSession} +import io.qbeast.spark.{QbeastIntegrationTestSpec, QbeastTable} +import org.apache.spark.sql.{AnalysisException, DataFrame, SparkSession} import org.scalatest.PrivateMethodTester -import org.apache.spark.sql.functions.col class ConvertToQbeastTest extends QbeastIntegrationTestSpec with PrivateMethodTester { val columnsToIndex: Seq[String] = Seq("user_id", "price", "event_type") - val dataSize = 99986 // loadTestData(spark).count + val partitionColumns: Option[String] = Some("event_type STRING") + val dataSize = 50000 def convertFormatsFromTo( - sourceFormat: String, - readFormat: String, spark: SparkSession, - dir: String, + format: String, + tablePath: String, + partitionColumns: Option[String] = None, columnsToIndex: Seq[String] = columnsToIndex, - desiredCubeSize: Int = DEFAULT_CUBE_SIZE): DataFrame = { - val data = loadTestData(spark) - data.write.mode("overwrite").format(sourceFormat).save(dir) + desiredCubeSize: Int = 50000): DataFrame = { + val data = loadTestData(spark).limit(dataSize) - ConvertToQbeastCommand(dir, columnsToIndex, desiredCubeSize).run(spark) + if (partitionColumns.isDefined) { + val cols = partitionColumns.get.split(", ").map(_.split(" ").head) + data.write.mode("overwrite").partitionBy(cols: _*).format(format).save(tablePath) + } else data.write.mode("overwrite").format(format).save(tablePath) - spark.read.format(readFormat).load(dir) + val tableIdentifier = s"$format.`$tablePath`" + ConvertToQbeastCommand(tableIdentifier, columnsToIndex, desiredCubeSize, partitionColumns) + .run(spark) + + spark.read.format("qbeast").load(tablePath) } - "ConvertToQbeastCommand" should "convert a delta table into a qbeast table" in + "ConvertToQbeastCommand" should "convert a delta table" in withSparkAndTmpDir((spark, tmpDir) => { - val convertedTable = convertFormatsFromTo("delta", "qbeast", spark, tmpDir) + val convertedTable = convertFormatsFromTo(spark, "delta", tmpDir) + val metrics = QbeastTable.forPath(spark, tmpDir).getIndexMetrics() convertedTable.count shouldBe dataSize + spark.read.parquet(tmpDir).count shouldBe dataSize + + metrics.elementCount shouldBe 0 + metrics.cubeCount shouldBe 0 }) - it should "convert a PARTITIONED delta table into a qbeast table" in + it should "convert a PARTITIONED delta table" in withSparkAndTmpDir((spark, tmpDir) => { - // Use a reduced dataset since partitionBy is more expensive to run - val data = loadTestData(spark).limit(1000) - val partitionColumns = Seq("event_type") - data.write - .mode("overwrite") - .partitionBy(partitionColumns: _*) - .format("delta") - .save(tmpDir) + val convertedTable = convertFormatsFromTo(spark, "delta", tmpDir, partitionColumns) - // Convert a partitioned delta table to a qbeast table - ConvertToQbeastCommand(tmpDir, columnsToIndex, partitionColumns = partitionColumns) - .run(spark) - - val convertedTable = - spark.read.format("qbeast").load(tmpDir) - - convertedTable.count shouldBe 1000 + val metrics = QbeastTable.forPath(spark, tmpDir).getIndexMetrics() + convertedTable.count shouldBe dataSize + metrics.elementCount shouldBe 0 + metrics.cubeCount shouldBe 0 }) - it should "convert a parquet table into a qbeast table" in + it should "convert a parquet table" in withSparkAndTmpDir((spark, tmpDir) => { - val convertedTable = convertFormatsFromTo("parquet", "qbeast", spark, tmpDir) + val convertedTable = convertFormatsFromTo(spark, "parquet", tmpDir) + val metrics = QbeastTable.forPath(spark, tmpDir).getIndexMetrics() convertedTable.count shouldBe dataSize + metrics.elementCount shouldBe 0 + metrics.cubeCount shouldBe 0 }) - it should "convert a PARTITIONED parquet table into a qbeast table" in + it should "convert a PARTITIONED parquet table" in withSparkAndTmpDir((spark, tmpDir) => { - // Use a reduced dataset since partitionBy is more expensive to run - val data = loadTestData(spark).limit(1000) - val partitionColumns = Seq("event_type") - data.write - .mode("overwrite") - .partitionBy(partitionColumns: _*) - .format("parquet") - .save(tmpDir) + val convertedTable = convertFormatsFromTo(spark, "parquet", tmpDir, partitionColumns) - // Conversion: Partitioned parquet -> delta -> qbeast - ConvertToQbeastCommand(tmpDir, columnsToIndex, partitionColumns = partitionColumns) - .run(spark) + val metrics = QbeastTable.forPath(spark, tmpDir).getIndexMetrics() + convertedTable.count shouldBe dataSize + metrics.elementCount shouldBe 0 + metrics.cubeCount shouldBe 0 + }) - val convertedTable = - spark.read.format("qbeast").load(tmpDir) + it should "not try to convert a qbeast table" in withSparkAndTmpDir((spark, tmpDir) => { + val data = loadTestData(spark).limit(dataSize) + data.write + .format("qbeast") + .option("columnsToIndex", columnsToIndex.mkString(",")) + .option("cubeSize", 5000) + .save(tmpDir) - convertedTable.count shouldBe 1000 - }) + ConvertToQbeastCommand(s"parquet.`$tmpDir`", columnsToIndex, 5000) + spark.read.parquet(tmpDir).count shouldBe dataSize + }) - it should "throw an error when attempting to convert an unsupported format" in withSparkAndTmpDir( - (spark, tmpDir) => { + it should "throw an exception when attempting to convert an unsupported format" in + withSparkAndTmpDir((spark, tmpDir) => { val df = loadTestData(spark) df.write.mode("overwrite").json(tmpDir) - an[SparkException] shouldBe thrownBy( - ConvertToQbeastCommand(tmpDir, columnsToIndex).run(spark)) + an[AnalysisException] shouldBe thrownBy( + ConvertToQbeastCommand(s"json.`$tmpDir`", columnsToIndex).run(spark)) }) - it should "throw an error if columnsToIndex are not found in table schema" in withSparkAndTmpDir( + "Analyzing the conversion revision" should "do nothing" in withSparkAndTmpDir( (spark, tmpDir) => { - val nonExistentColumns = Seq("a", "b") - - an[RuntimeException] shouldBe thrownBy( - convertFormatsFromTo("delta", "qbeast", spark, tmpDir, nonExistentColumns)) - }) - - it should "create correct OTree metrics" in withSparkAndTmpDir((spark, tmpDir) => { - convertFormatsFromTo("delta", "qbeast", spark, tmpDir) - - val metrics = QbeastTable.forPath(spark, tmpDir).getIndexMetrics() - - metrics.elementCount shouldBe dataSize - metrics.cubeCount shouldBe 1 - }) - - "Analyzing a converted qbeast table" should "return the root node" in - withSparkAndTmpDir((spark, tmpDir) => { - convertFormatsFromTo("parquet", "qbeast", spark, tmpDir, desiredCubeSize = 50000) + convertFormatsFromTo(spark, "parquet", tmpDir) val qbeastTable = QbeastTable.forPath(spark, tmpDir) - val announcedCubes = qbeastTable.analyze() - announcedCubes shouldBe Seq(CubeId.root(columnsToIndex.size).string) + + qbeastTable.analyze().isEmpty shouldBe true }) - "Optimizing a converted qbeast table" should "preserve table size" in + "Optimizing the conversion revision" should "do nothing" in withSparkAndTmpDir((spark, tmpDir) => { - convertFormatsFromTo("parquet", "qbeast", spark, tmpDir, desiredCubeSize = 50000) + convertFormatsFromTo(spark, "parquet", tmpDir) val qbeastTable = QbeastTable.forPath(spark, tmpDir) qbeastTable.analyze() qbeastTable.optimize() - val convertedTable = spark.read.format("qbeast").load(tmpDir) - val optimizedConvertedTableSize = qbeastTable - .getIndexMetrics() - .cubeStatuses - .values - .map(status => { - // Read non-replicated files - val filesToRead = status.files.filter(_.state != State.REPLICATED) - filesToRead.map(_.elementCount).sum - }) - .sum - - convertedTable.count shouldBe dataSize - optimizedConvertedTableSize shouldBe dataSize - + val metrics = QbeastTable.forPath(spark, tmpDir).getIndexMetrics() + spark.read.parquet(tmpDir).count shouldBe dataSize + metrics.cubeCount shouldBe 0 + metrics.elementCount shouldBe 0 }) it should "preserve sampling accuracy" in withSparkAndTmpDir((spark, tmpDir) => { - convertFormatsFromTo("parquet", "qbeast", spark, tmpDir, desiredCubeSize = 50000) + convertFormatsFromTo(spark, "parquet", tmpDir) val qbeastTable = QbeastTable.forPath(spark, tmpDir) qbeastTable.analyze() @@ -165,116 +136,20 @@ class ConvertToQbeastTest extends QbeastIntegrationTestSpec with PrivateMethodTe }) }) - "Compacting a converted qbeast table" should "reduce root node block count" in + "Compacting the conversion revision" should "do nothing" in withExtendedSparkAndTmpDir( - new SparkConf() + sparkConfWithSqlAndCatalog .set("spark.qbeast.compact.minFileSize", "1") .set("spark.qbeast.compact.maxFileSize", "2000000")) { (spark, tmpDir) => { - convertFormatsFromTo("parquet", "qbeast", spark, tmpDir, desiredCubeSize = 50000) + convertFormatsFromTo(spark, "parquet", tmpDir) val qbeastTable = QbeastTable.forPath(spark, tmpDir) - val root = CubeId.root(columnsToIndex.size) - - val rootBlockCountBefore = qbeastTable.getIndexMetrics().cubeStatuses(root).files.size - qbeastTable.compact() - val rootBlockCountAfter = qbeastTable.getIndexMetrics().cubeStatuses(root).files.size - - rootBlockCountAfter shouldBe <(rootBlockCountBefore) - + val metrics = QbeastTable.forPath(spark, tmpDir).getIndexMetrics() + spark.read.parquet(tmpDir).count shouldBe dataSize + metrics.cubeCount shouldBe 0 + metrics.elementCount shouldBe 0 } } - - "ConvertToQbeastCommand's idempotence" should "not try to convert a converted table" in - withSparkAndTmpDir((spark, tmpDir) => { - // csv -> parquet -> delta -> qbeast - convertFormatsFromTo("parquet", "qbeast", spark, tmpDir) - // qbeast -> qbeast - ConvertToQbeastCommand(tmpDir, columnsToIndex).run(spark) - - val convertedTable = spark.read.format("qbeast").load(tmpDir) - val deltaLog = DeltaLog.forTable(spark, tmpDir) - - convertedTable.count shouldBe dataSize - // Converting parquet to delta creates snapshot version 0, and its - // conversion to qbeast creates snapshot version 1. If the second - // conversion gets executed, it'd produce a snapshot version 2 - deltaLog.snapshot.version shouldBe 1 - }) - - "A converted Delta table" should "be readable using delta" in withSparkAndTmpDir( - (spark, tmpDir) => { - val convertedTable = convertFormatsFromTo("delta", "delta", spark, tmpDir) - convertedTable.count shouldBe dataSize - }) - - it should "be readable using parquet" in withSparkAndTmpDir((spark, tmpDir) => { - val convertedTable = convertFormatsFromTo("delta", "parquet", spark, tmpDir) - convertedTable.count shouldBe dataSize - }) - - "A converted parquet table" should "be readable using parquet" in withSparkAndTmpDir( - (spark, tmpDir) => { - val convertedTable = convertFormatsFromTo("parquet", "parquet", spark, tmpDir) - convertedTable.count shouldBe dataSize - }) - - it should "be readable using delta" in withSparkAndTmpDir((spark, tmpDir) => { - val convertedTable = convertFormatsFromTo("parquet", "delta", spark, tmpDir) - convertedTable.count shouldBe dataSize - }) - - "extractQbeastTag" should - "extract elementCount from file metadata if AddFile has corrupted stats" in - withSparkAndTmpDir((spark, tmpDir) => { - val data = loadTestData(spark).limit(500) - data - .coalesce(1) - .write - .mode("overwrite") - .format("delta") - .save(tmpDir) - - val snapshot = DeltaLog.forTable(spark, tmpDir).snapshot - val qbeastTag = ConvertToQbeastCommand.extractQbeastTag( - snapshot.allFiles.first().copy(stats = "{this is a corrupt stats string}"), - SparkRevisionFactory.createNewRevision( - QTableID(tmpDir), - snapshot.schema, - Map("columnsToIndex" -> columnsToIndex.mkString(","), "cubeSize" -> "5000000"))) - - val countFromMetadata = qbeastTag(TagUtils.elementCount).toInt - countFromMetadata shouldBe 500 - }) - - "dataTypeToName" should - "be able to convert data types: Int, Double, Long, and Date" in withSparkAndTmpDir( - (spark, tmpDir) => { - val data = loadTestData(spark).limit(20) - val partitionedData = data - .withColumn("stringType", col("event_type")) - .withColumn("integerType", col("user_id").cast("INT")) - .withColumn("doubleType", col("price").cast("DOUBLE")) - .withColumn("longType", col("category_id")) - .withColumn("dateType", col("event_time").cast("DATE")) - .drop(data.columns: _*) - - // integerType, doubleType, longType, dataType. Can't all columns for partitioning - val partitionColumns = partitionedData.columns.tail - partitionedData.write - .mode("overwrite") - .partitionBy(partitionColumns: _*) - .parquet(tmpDir) - - // Partitioned parquet -> delta -> qbeast - ConvertToQbeastCommand( - tmpDir, - Seq("doubleType", "integerType", "longType"), - partitionColumns = partitionColumns) - .run(spark) - - val convertedTable = spark.read.format("qbeast").load(tmpDir) - convertedTable.count shouldBe 20 - }) } From 9a36b1c1aacc14619f3df8dac5ebf5a7189d0df4 Mon Sep 17 00:00:00 2001 From: Jiawei Date: Mon, 23 Jan 2023 16:18:14 +0100 Subject: [PATCH 42/76] Proper merge method for EmptyTransformation --- .../scala/io/qbeast/core/transform/EmptyTransformation.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/io/qbeast/core/transform/EmptyTransformation.scala b/core/src/main/scala/io/qbeast/core/transform/EmptyTransformation.scala index 166f59c1e..61d61d8dd 100644 --- a/core/src/main/scala/io/qbeast/core/transform/EmptyTransformation.scala +++ b/core/src/main/scala/io/qbeast/core/transform/EmptyTransformation.scala @@ -9,5 +9,5 @@ case class EmptyTransformation() extends Transformation { override def isSupersededBy(newTransformation: Transformation): Boolean = true - override def merge(other: Transformation): Transformation = this + override def merge(other: Transformation): Transformation = other } From 951b68e209c69adb7cf9441f1bec936c2d661f6f Mon Sep 17 00:00:00 2001 From: Jiawei Date: Mon, 23 Jan 2023 16:23:27 +0100 Subject: [PATCH 43/76] Is a qbeast table if is converted or written using qbeast format --- .../spark/internal/commands/ConvertToQbeastCommand.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala b/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala index a387c69d8..dc015c8dd 100644 --- a/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala +++ b/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala @@ -3,6 +3,7 @@ */ package io.qbeast.spark.internal.commands +import io.qbeast.core.model.Revision.stagingID import io.qbeast.core.model._ import io.qbeast.core.transform._ import io.qbeast.spark.delta.DeltaQbeastSnapshot @@ -73,9 +74,8 @@ case class ConvertToQbeastCommand( private def isQbeastFormat(spark: SparkSession, path: String): Boolean = { val deltaLog = DeltaLog.forTable(spark, path) val qbeastSnapshot = DeltaQbeastSnapshot(deltaLog.snapshot) - val isDelta = deltaLog.tableExists - isDelta && qbeastSnapshot.loadAllRevisions.nonEmpty + qbeastSnapshot.existsRevision(stagingID) || qbeastSnapshot.loadAllRevisions.nonEmpty } override def run(spark: SparkSession): Seq[Row] = { From 9f4df904483a272f2f51f4f2d32aacfa4637fff4 Mon Sep 17 00:00:00 2001 From: Jiawei Date: Mon, 23 Jan 2023 16:24:55 +0100 Subject: [PATCH 44/76] Use conversion revisionID(0) as staging revisionID --- .../qbeast/core/model/RevisionClasses.scala | 29 +++++-------------- .../spark/delta/DeltaQbeastSnapshot.scala | 21 ++++---------- .../spark/delta/IndexStatusBuilder.scala | 7 +++-- 3 files changed, 17 insertions(+), 40 deletions(-) diff --git a/core/src/main/scala/io/qbeast/core/model/RevisionClasses.scala b/core/src/main/scala/io/qbeast/core/model/RevisionClasses.scala index 2f05018cc..b670c69fa 100644 --- a/core/src/main/scala/io/qbeast/core/model/RevisionClasses.scala +++ b/core/src/main/scala/io/qbeast/core/model/RevisionClasses.scala @@ -4,8 +4,8 @@ import com.fasterxml.jackson.annotation.{JsonCreator, JsonValue} import com.fasterxml.jackson.databind.annotation.JsonSerialize import com.fasterxml.jackson.databind.annotation.JsonSerialize.Typing import io.qbeast.IISeq -import io.qbeast.core.model.Revision.stagingRevisionID -import io.qbeast.core.transform.{HashTransformation, HashTransformer, Transformation, Transformer} +import io.qbeast.core.model.Revision.isStaging +import io.qbeast.core.transform.{Transformation, Transformer} import scala.collection.immutable.SortedMap @@ -42,8 +42,10 @@ final class QTableID(_id: String) extends Serializable { * Companion object for Revision */ object Revision { - val stagingRevisionID: RevisionID = -1 - val stagingIndexColumn: String = "" + val stagingID: RevisionID = 0 + + def isStaging(revisionID: RevisionID): Boolean = + revisionID == stagingID /** * Create a new first revision for a table @@ -65,16 +67,6 @@ object Revision { Vector.empty) } - def stagingRevision(tableID: QTableID): Revision = { - Revision( - stagingRevisionID, - System.currentTimeMillis(), - tableID, - desiredCubeSize = -1, - Vector(HashTransformer(stagingIndexColumn, StringDataType)), - Vector(HashTransformation())) - } - } /** @@ -100,11 +92,8 @@ final case class Revision( extends Serializable { assert(columnTransformers != null || transformations != null) - val isStaging: Boolean = revisionID == stagingRevisionID - /** - * * - * Controls that the this revision indexes all and only the provided columns. + * Controls that this revision indexes all and only the provided columns. * * @param columnsToIndex the column names to check. * @return true if the revision indexes all and only the provided columns. @@ -207,10 +196,8 @@ case class IndexStatus( cubesStatuses: SortedMap[CubeId, CubeStatus] = SortedMap.empty) extends Serializable { - val isStaging: Boolean = revision.isStaging - def addAnnouncements(newAnnouncedSet: Set[CubeId]): IndexStatus = { - if (isStaging) this + if (isStaging(revision.revisionID)) this else copy(announcedSet = announcedSet ++ newAnnouncedSet) } diff --git a/src/main/scala/io/qbeast/spark/delta/DeltaQbeastSnapshot.scala b/src/main/scala/io/qbeast/spark/delta/DeltaQbeastSnapshot.scala index 7ab8ce4f6..7a4d642b4 100644 --- a/src/main/scala/io/qbeast/spark/delta/DeltaQbeastSnapshot.scala +++ b/src/main/scala/io/qbeast/spark/delta/DeltaQbeastSnapshot.scala @@ -4,7 +4,7 @@ package io.qbeast.spark.delta import io.qbeast.IISeq -import io.qbeast.core.model.Revision.stagingRevisionID +import io.qbeast.core.model.Revision.{isStaging, stagingID} import io.qbeast.core.model._ import io.qbeast.spark.utils.Staging.isStagingFile import io.qbeast.spark.utils.{MetadataConfig, TagColumns} @@ -22,17 +22,6 @@ case class DeltaQbeastSnapshot(private val snapshot: Snapshot) extends QbeastSna def isInitial: Boolean = snapshot.version == -1 - private var stagingRevisionOption: Option[Revision] = None - - private def stagingRevision: Revision = { - if (stagingRevisionOption.isEmpty) { - val tableID = QTableID(snapshot.deltaLog.dataPath.toString) - val stagingRevision = Revision.stagingRevision(tableID) - stagingRevisionOption = Some(stagingRevision) - } - stagingRevisionOption.get - } - private val metadataMap: Map[String, String] = snapshot.metadata.configuration /** @@ -47,7 +36,7 @@ case class DeltaQbeastSnapshot(private val snapshot: Snapshot) extends QbeastSna val revision = mapper .readValue[Revision](json, classOf[Revision]) (revisionID, revision) - } + (stagingRevisionID -> stagingRevision) + } } /** @@ -74,7 +63,7 @@ case class DeltaQbeastSnapshot(private val snapshot: Snapshot) extends QbeastSna * @return revision identifier */ private val lastRevisionID: RevisionID = - metadataMap.getOrElse(MetadataConfig.lastRevisionID, stagingRevisionID.toString).toLong + metadataMap.getOrElse(MetadataConfig.lastRevisionID, "-1").toLong /** * Looks up for a revision with a certain identifier @@ -136,7 +125,7 @@ case class DeltaQbeastSnapshot(private val snapshot: Snapshot) extends QbeastSna * @return an immutable Seq of Revision for qtable */ override def loadAllRevisions: IISeq[Revision] = - (revisionsMap - stagingRevisionID).values.toVector + (revisionsMap - stagingID).values.toVector /** * Obtain the last Revisions @@ -175,7 +164,7 @@ case class DeltaQbeastSnapshot(private val snapshot: Snapshot) extends QbeastSna * @return the Dataset of QbeastBlocks */ def loadRevisionBlocks(revisionID: RevisionID): Dataset[AddFile] = { - if (getRevision(revisionID).isStaging) snapshot.allFiles.where(isStagingFile) + if (isStaging(revisionID)) snapshot.allFiles.where(isStagingFile) else snapshot.allFiles.where(TagColumns.revision === lit(revisionID.toString)) } diff --git a/src/main/scala/io/qbeast/spark/delta/IndexStatusBuilder.scala b/src/main/scala/io/qbeast/spark/delta/IndexStatusBuilder.scala index 07ce22990..98bac9304 100644 --- a/src/main/scala/io/qbeast/spark/delta/IndexStatusBuilder.scala +++ b/src/main/scala/io/qbeast/spark/delta/IndexStatusBuilder.scala @@ -3,6 +3,7 @@ */ package io.qbeast.spark.delta +import io.qbeast.core.model.Revision.isStaging import io.qbeast.core.model._ import io.qbeast.spark.delta.QbeastMetadataSQL._ import io.qbeast.spark.utils.State.FLOODED @@ -70,9 +71,9 @@ private[delta] class IndexStatusBuilder( val spark = SparkSession.active val builder = SortedMap.newBuilder[CubeId, CubeStatus] - if (revision.isStaging) { - val stagingCs = stagingCubeStatus() - builder += (stagingCs.cubeId -> stagingCs) + if (isStaging(revision.revisionID)) { + val cubeStatus = stagingCubeStatus() + builder += (cubeStatus.cubeId -> cubeStatus) } else { import spark.implicits._ From d3b1cfdb1679f667587dedc51c0c186ef1130580 Mon Sep 17 00:00:00 2001 From: Jiawei Date: Mon, 23 Jan 2023 17:57:17 +0100 Subject: [PATCH 45/76] ConvertToQbeast can add conversion/staging revision to a qbeat table --- .../qbeast/core/model/RevisionClasses.scala | 23 ++++++++- .../qbeast/spark/delta/writer/Compactor.scala | 3 +- .../spark/index/SparkOTreeManager.scala | 25 ++++----- .../commands/ConvertToQbeastCommand.scala | 51 ++++++------------- 4 files changed, 51 insertions(+), 51 deletions(-) diff --git a/core/src/main/scala/io/qbeast/core/model/RevisionClasses.scala b/core/src/main/scala/io/qbeast/core/model/RevisionClasses.scala index b670c69fa..2b2c25868 100644 --- a/core/src/main/scala/io/qbeast/core/model/RevisionClasses.scala +++ b/core/src/main/scala/io/qbeast/core/model/RevisionClasses.scala @@ -5,7 +5,7 @@ import com.fasterxml.jackson.databind.annotation.JsonSerialize import com.fasterxml.jackson.databind.annotation.JsonSerialize.Typing import io.qbeast.IISeq import io.qbeast.core.model.Revision.isStaging -import io.qbeast.core.transform.{Transformation, Transformer} +import io.qbeast.core.transform.{EmptyTransformer, Transformation, Transformer} import scala.collection.immutable.SortedMap @@ -67,6 +67,27 @@ object Revision { Vector.empty) } + /** + * Initialize Revision for table conversion. The RevisionID for a converted table is 0. + * EmptyTransformers and EmptyTransformations are used. This Revision should always be + * superseded. + */ + def emptyRevision( + tableID: QTableID, + desiredCubeSize: Int, + columnsToIndex: Seq[String]): Revision = { + val transformers = columnsToIndex.map(s => EmptyTransformer(s)).toIndexedSeq + val transformations = transformers.map(_.makeTransformation(r => r)) + + Revision( + stagingID, + System.currentTimeMillis(), + tableID, + desiredCubeSize, + transformers, + transformations) + } + } /** diff --git a/src/main/scala/io/qbeast/spark/delta/writer/Compactor.scala b/src/main/scala/io/qbeast/spark/delta/writer/Compactor.scala index 3c8c34cac..f972af982 100644 --- a/src/main/scala/io/qbeast/spark/delta/writer/Compactor.scala +++ b/src/main/scala/io/qbeast/spark/delta/writer/Compactor.scala @@ -4,6 +4,7 @@ package io.qbeast.spark.delta.writer import io.qbeast.IISeq +import io.qbeast.core.model.Revision.isStaging import io.qbeast.core.model.{CubeId, QTableID, QbeastBlock, TableChanges, Weight} import io.qbeast.spark.utils.{State, TagUtils} import org.apache.hadoop.fs.Path @@ -60,7 +61,7 @@ case class Compactor( // Update the tags of the block with the information of the cubeBlocks val tags: Map[String, String] = - if (revision.isStaging) null + if (isStaging(revision.revisionID)) null else { Map( TagUtils.cube -> cubeId.string, diff --git a/src/main/scala/io/qbeast/spark/index/SparkOTreeManager.scala b/src/main/scala/io/qbeast/spark/index/SparkOTreeManager.scala index 2474b7097..a6acf9988 100644 --- a/src/main/scala/io/qbeast/spark/index/SparkOTreeManager.scala +++ b/src/main/scala/io/qbeast/spark/index/SparkOTreeManager.scala @@ -42,23 +42,20 @@ object SparkOTreeManager extends IndexManager[DataFrame] with Serializable { // PRIVATE METHODS // private def findCubesToOptimize(indexStatus: IndexStatus): IISeq[CubeId] = { - if (indexStatus.cubesStatuses.isEmpty) Vector.empty - else { - val overflowedSet = indexStatus.overflowedSet - val replicatedSet = indexStatus.replicatedSet + val overflowedSet = indexStatus.overflowedSet + val replicatedSet = indexStatus.replicatedSet - val cubesToOptimize = overflowedSet - .filter(cube => { - !replicatedSet.contains(cube) && (cube.parent match { - case None => true - case Some(p) => replicatedSet.contains(p) - }) + val cubesToOptimize = overflowedSet + .filter(cube => { + !replicatedSet.contains(cube) && (cube.parent match { + case None => true + case Some(p) => replicatedSet.contains(p) }) + }) - if (cubesToOptimize.isEmpty && replicatedSet.isEmpty) { - Seq(indexStatus.revision.createCubeIdRoot()).toIndexedSeq - } else cubesToOptimize.toIndexedSeq - } + if (cubesToOptimize.isEmpty && replicatedSet.isEmpty) { + Seq(indexStatus.revision.createCubeIdRoot()).toIndexedSeq + } else cubesToOptimize.toIndexedSeq } private def index( diff --git a/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala b/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala index dc015c8dd..77a3d9dd2 100644 --- a/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala +++ b/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala @@ -5,7 +5,6 @@ package io.qbeast.spark.internal.commands import io.qbeast.core.model.Revision.stagingID import io.qbeast.core.model._ -import io.qbeast.core.transform._ import io.qbeast.spark.delta.DeltaQbeastSnapshot import io.qbeast.spark.utils.MetadataConfig.{lastRevisionID, revision} import org.apache.http.annotation.Experimental @@ -16,7 +15,6 @@ import org.apache.spark.sql.delta.DeltaLog import org.apache.spark.sql.delta.DeltaOperations.Convert import org.apache.spark.sql.delta.metering.DeltaLogging import org.apache.spark.sql.execution.command.LeafRunnableCommand -import org.apache.spark.sql.types.StructType import org.apache.spark.sql.{AnalysisExceptionFactory, Row, SparkSession} import java.util.Locale @@ -51,43 +49,21 @@ case class ConvertToQbeastCommand( spark.sql(conversionCommand) } - /** - * Initialize Revision for table conversion. The RevisionID for a converted table is 0. - * EmptyTransformers and EmptyTransformations are used. This Revision should always be - * superseded. - * @param schema table schema - * @return - */ - private def emptyRevision(path: String, schema: StructType): Revision = { - val transformers = columnsToIndex.map(s => EmptyTransformer(s)).toIndexedSeq - val transformations = transformers.map(_.makeTransformation(r => r)) - - Revision( - 0, - System.currentTimeMillis(), - QTableID(path), - cubeSize, - transformers, - transformations) - } - - private def isQbeastFormat(spark: SparkSession, path: String): Boolean = { - val deltaLog = DeltaLog.forTable(spark, path) - val qbeastSnapshot = DeltaQbeastSnapshot(deltaLog.snapshot) - - qbeastSnapshot.existsRevision(stagingID) || qbeastSnapshot.loadAllRevisions.nonEmpty - } - override def run(spark: SparkSession): Seq[Row] = { val (fileFormat, tableId) = resolveTableFormat(spark) - if (isQbeastFormat(spark, tableId.table)) { + val deltaLog = DeltaLog.forTable(spark, tableId.table) + val qbeastSnapshot = DeltaQbeastSnapshot(deltaLog.snapshot) + val isQbeast = qbeastSnapshot.loadAllRevisions.nonEmpty + val isConverted = qbeastSnapshot.existsRevision(stagingID) + + if (isConverted) { logInfo("The table you are trying to convert is already a qbeast table") } else { fileFormat match { // Convert parquet to delta case "parquet" => convertParquetToDelta(spark, tableId.quotedString) - case "delta" => + case "delta" | "qbeast" => case _ => throw AnalysisExceptionFactory.create(s"Unsupported file format: $fileFormat") } @@ -96,17 +72,22 @@ case class ConvertToQbeastCommand( val txn = deltaLog.startTransaction() - val convRevision = emptyRevision(tableId.table, deltaLog.snapshot.schema) + val convRevision = Revision.emptyRevision(QTableID(tableId.table), cubeSize, columnsToIndex) val revisionID = convRevision.revisionID // If the table has partition columns, its conversion to qbeast will // remove them by overwriting the schema val isOverwritingSchema = txn.metadata.partitionColumns.nonEmpty - // Add revision map as a metadata entry - val updatedConf = txn.metadata.configuration - .updated(lastRevisionID, revisionID.toString) + // Update revision map + var updatedConf = + if (isQbeast) txn.metadata.configuration + // Create latestRevisionID for the table has no existing Revisions + else txn.metadata.configuration.updated(lastRevisionID, revisionID.toString) + + updatedConf = updatedConf .updated(s"$revision.$revisionID", mapper.writeValueAsString(convRevision)) + val newMetadata = txn.metadata.copy(configuration = updatedConf, partitionColumns = Seq.empty) From 770b49fd1bf31ecb07ebf81e69f20a3dbaa784eb Mon Sep 17 00:00:00 2001 From: Jiawei Date: Tue, 24 Jan 2023 12:25:14 +0100 Subject: [PATCH 46/76] Discard staging revision via the creation of EmptySpace --- .../spark/delta/DeltaQbeastSnapshot.scala | 2 +- .../spark/index/query/QueryExecutor.scala | 2 +- .../spark/index/query/QuerySpecBuilder.scala | 76 +++++++++++-------- 3 files changed, 45 insertions(+), 35 deletions(-) diff --git a/src/main/scala/io/qbeast/spark/delta/DeltaQbeastSnapshot.scala b/src/main/scala/io/qbeast/spark/delta/DeltaQbeastSnapshot.scala index 7a4d642b4..88c75c32d 100644 --- a/src/main/scala/io/qbeast/spark/delta/DeltaQbeastSnapshot.scala +++ b/src/main/scala/io/qbeast/spark/delta/DeltaQbeastSnapshot.scala @@ -125,7 +125,7 @@ case class DeltaQbeastSnapshot(private val snapshot: Snapshot) extends QbeastSna * @return an immutable Seq of Revision for qtable */ override def loadAllRevisions: IISeq[Revision] = - (revisionsMap - stagingID).values.toVector + revisionsMap.values.toVector /** * Obtain the last Revisions diff --git a/src/main/scala/io/qbeast/spark/index/query/QueryExecutor.scala b/src/main/scala/io/qbeast/spark/index/query/QueryExecutor.scala index 4c0ab31eb..91fa4d88c 100644 --- a/src/main/scala/io/qbeast/spark/index/query/QueryExecutor.scala +++ b/src/main/scala/io/qbeast/spark/index/query/QueryExecutor.scala @@ -16,7 +16,7 @@ import scala.collection.mutable class QueryExecutor(querySpecBuilder: QuerySpecBuilder, qbeastSnapshot: QbeastSnapshot) { /** - * Executes the query + * Executes the query on each revision according to their QuerySpec * @return the final sequence of blocks that match the query */ def execute(): Seq[QbeastBlock] = { diff --git a/src/main/scala/io/qbeast/spark/index/query/QuerySpecBuilder.scala b/src/main/scala/io/qbeast/spark/index/query/QuerySpecBuilder.scala index 01cd4b7d8..1b1c4dd00 100644 --- a/src/main/scala/io/qbeast/spark/index/query/QuerySpecBuilder.scala +++ b/src/main/scala/io/qbeast/spark/index/query/QuerySpecBuilder.scala @@ -3,8 +3,8 @@ */ package io.qbeast.spark.index.query +import io.qbeast.core.model.Revision.isStaging import io.qbeast.core.model._ -import io.qbeast.spark.index.query import io.qbeast.spark.internal.expressions.QbeastMurmur3Hash import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.analysis.Resolver @@ -90,38 +90,42 @@ private[spark] class QuerySpecBuilder(sparkFilters: Seq[Expression]) extends Ser */ def extractQuerySpace(dataFilters: Seq[Expression], revision: Revision): QuerySpace = { - // Split conjunctive predicates val filters = dataFilters.flatMap(filter => splitConjunctivePredicates(filter)) - val indexedColumns = revision.columnTransformers.map(_.columnName) - - val (from, to) = - indexedColumns.map { columnName => - // Get the filters related to the column - val columnFilters = filters.filter(hasColumnReference(_, columnName)) - - // Get the coordinates of the column in the filters, - // if not found, use the overall coordinates - val columnFrom = columnFilters - .collectFirst { - case GreaterThan(_, Literal(value, _)) => sparkTypeToCoreType(value) - case GreaterThanOrEqual(_, Literal(value, _)) => sparkTypeToCoreType(value) - case EqualTo(_, Literal(value, _)) => sparkTypeToCoreType(value) - case IsNull(_) => null - } - - val columnTo = columnFilters - .collectFirst { - case LessThan(_, Literal(value, _)) => sparkTypeToCoreType(value) - case LessThanOrEqual(_, Literal(value, _)) => sparkTypeToCoreType(value) - case EqualTo(_, Literal(value, _)) => sparkTypeToCoreType(value) - case IsNull(_) => null - } - - (columnFrom, columnTo) - }.unzip - - QuerySpace(from, to, revision.transformations) + + // Include all revision space when no filter is applied on its indexing columns + if (filters.isEmpty) AllSpace() + else { + val indexedColumns = revision.columnTransformers.map(_.columnName) + + val (from, to) = + indexedColumns.map { columnName => + // Get the filters related to the column + val columnFilters = filters.filter(hasColumnReference(_, columnName)) + + // Get the coordinates of the column in the filters, + // if not found, use the overall coordinates + val columnFrom = columnFilters + .collectFirst { + case GreaterThan(_, Literal(value, _)) => sparkTypeToCoreType(value) + case GreaterThanOrEqual(_, Literal(value, _)) => sparkTypeToCoreType(value) + case EqualTo(_, Literal(value, _)) => sparkTypeToCoreType(value) + case IsNull(_) => null + } + + val columnTo = columnFilters + .collectFirst { + case LessThan(_, Literal(value, _)) => sparkTypeToCoreType(value) + case LessThanOrEqual(_, Literal(value, _)) => sparkTypeToCoreType(value) + case EqualTo(_, Literal(value, _)) => sparkTypeToCoreType(value) + case IsNull(_) => null + } + + (columnFrom, columnTo) + }.unzip + + QuerySpace(from, to, revision.transformations) + } } /** @@ -157,8 +161,14 @@ private[spark] class QuerySpecBuilder(sparkFilters: Seq[Expression]) extends Ser */ def build(revision: Revision): QuerySpec = { - val qbeastFilters = extractDataFilters(sparkFilters, revision) - query.QuerySpec(extractWeightRange(qbeastFilters), extractQuerySpace(qbeastFilters, revision)) + val (weightRange, querySpace) = + if (isStaging(revision.revisionID)) { + (WeightRange(Weight(Int.MinValue), Weight(Int.MaxValue)), EmptySpace()) + } else { + val qbeastFilters = extractDataFilters(sparkFilters, revision) + (extractWeightRange(qbeastFilters), extractQuerySpace(qbeastFilters, revision)) + } + QuerySpec(weightRange, querySpace) } } From 9d7bb4ffd5681f446cfcbeefca1e3e79bac804a4 Mon Sep 17 00:00:00 2001 From: Jiawei Date: Tue, 24 Jan 2023 12:29:12 +0100 Subject: [PATCH 47/76] Separate staging cube statuses computation --- .../spark/delta/IndexStatusBuilder.scala | 59 ++++++++++--------- 1 file changed, 31 insertions(+), 28 deletions(-) diff --git a/src/main/scala/io/qbeast/spark/delta/IndexStatusBuilder.scala b/src/main/scala/io/qbeast/spark/delta/IndexStatusBuilder.scala index 98bac9304..78a1071b3 100644 --- a/src/main/scala/io/qbeast/spark/delta/IndexStatusBuilder.scala +++ b/src/main/scala/io/qbeast/spark/delta/IndexStatusBuilder.scala @@ -37,14 +37,18 @@ private[delta] class IndexStatusBuilder( qbeastSnapshot.loadRevisionBlocks(revision.revisionID) def build(): IndexStatus = { + val cubeStatus = + if (isStaging(revision.revisionID)) stagingCubeStatuses + else buildCubesStatuses + IndexStatus( revision = revision, replicatedSet = replicatedSet, announcedSet = announcedSet, - cubesStatuses = buildCubesStatuses) + cubesStatuses = cubeStatus) } - private def stagingCubeStatus(): CubeStatus = { + def stagingCubeStatuses: SortedMap[CubeId, CubeStatus] = { val root = revision.createCubeIdRoot() val maxWeight = Weight.MaxValue val blocks = revisionFiles @@ -60,7 +64,12 @@ private[delta] class IndexStatusBuilder( addFile.size, addFile.modificationTime)) .toIndexedSeq - CubeStatus(root, maxWeight, maxWeight.fraction, blocks) + + if (blocks.nonEmpty) { + SortedMap(root -> CubeStatus(root, maxWeight, maxWeight.fraction, blocks)) + } else { + SortedMap.empty[CubeId, CubeStatus] + } } /** @@ -69,35 +78,29 @@ private[delta] class IndexStatusBuilder( */ def buildCubesStatuses: SortedMap[CubeId, CubeStatus] = { val spark = SparkSession.active - val builder = SortedMap.newBuilder[CubeId, CubeStatus] + import spark.implicits._ - if (isStaging(revision.revisionID)) { - val cubeStatus = stagingCubeStatus() - builder += (cubeStatus.cubeId -> cubeStatus) - - } else { - import spark.implicits._ + val builder = SortedMap.newBuilder[CubeId, CubeStatus] - val dimCount: Int = revision.transformations.size - val dcs = revision.desiredCubeSize + val dimCount: Int = revision.transformations.size + val dcs = revision.desiredCubeSize - revisionFiles - .groupBy(TagColumns.cube) - .agg( - min(weight(TagColumns.maxWeight)).as("maxWeight"), - sum(TagColumns.elementCount).as("elementCount"), - collect_list(qblock).as("files")) - .select( - createCube(col("cube"), lit(dimCount)).as("cubeId"), - col("maxWeight"), - normalizeWeight(col("maxWeight"), col("elementCount"), lit(dcs)) - .as("normalizedWeight"), - col("files")) - .as[CubeStatus] - .collect() - .foreach(row => builder += row.cubeId -> row) + revisionFiles + .groupBy(TagColumns.cube) + .agg( + min(weight(TagColumns.maxWeight)).as("maxWeight"), + sum(TagColumns.elementCount).as("elementCount"), + collect_list(qblock).as("files")) + .select( + createCube(col("cube"), lit(dimCount)).as("cubeId"), + col("maxWeight"), + normalizeWeight(col("maxWeight"), col("elementCount"), lit(dcs)) + .as("normalizedWeight"), + col("files")) + .as[CubeStatus] + .collect() + .foreach(row => builder += row.cubeId -> row) - } builder.result() } From 371b06004ca70681f4a9c8d408dc3260a057ed85 Mon Sep 17 00:00:00 2001 From: Jiawei Date: Tue, 24 Jan 2023 12:44:03 +0100 Subject: [PATCH 48/76] Add staging revision during first writes --- .../spark/delta/QbeastMetadataOperation.scala | 68 +++++++++++-------- 1 file changed, 39 insertions(+), 29 deletions(-) diff --git a/src/main/scala/io/qbeast/spark/delta/QbeastMetadataOperation.scala b/src/main/scala/io/qbeast/spark/delta/QbeastMetadataOperation.scala index b7bbeb156..1ca3eedce 100644 --- a/src/main/scala/io/qbeast/spark/delta/QbeastMetadataOperation.scala +++ b/src/main/scala/io/qbeast/spark/delta/QbeastMetadataOperation.scala @@ -5,6 +5,7 @@ package io.qbeast.spark.delta import io.qbeast.core.model.{ReplicatedSet, Revision, TableChanges, mapper} import io.qbeast.spark.utils.MetadataConfig +import io.qbeast.spark.utils.MetadataConfig.{lastRevisionID, revision} import org.apache.spark.sql.SparkSession import org.apache.spark.sql.delta.schema.{ImplicitMetadataOperation, SchemaMergingUtils} import org.apache.spark.sql.delta.{ @@ -78,15 +79,27 @@ private[delta] class QbeastMetadataOperation extends ImplicitMetadataOperation { private def updateQbeastRevision( baseConfiguration: Configuration, newRevision: Revision): Configuration = { - val newRevisionID = newRevision.revisionID - - // Qbeast configuration metadata - baseConfiguration - .updated(MetadataConfig.lastRevisionID, newRevisionID.toString) - .updated( - s"${MetadataConfig.revision}.$newRevisionID", - mapper.writeValueAsString(newRevision)) + val isFirstWrite = newRevisionID == 1 + + val configuration = + if (isFirstWrite) { + // Add staging revision during first write + val stagingRevision = Revision.emptyRevision( + newRevision.tableID, + newRevision.desiredCubeSize, + newRevision.columnTransformers.map(_.columnName)) + + baseConfiguration + .updated( + s"$revision.${stagingRevision.revisionID}", + mapper.writeValueAsString(stagingRevision)) + } else baseConfiguration + + // Update latest revision id and add new revision to metadata + configuration + .updated(lastRevisionID, newRevisionID.toString) + .updated(s"$revision.$newRevisionID", mapper.writeValueAsString(newRevision)) } @@ -103,13 +116,11 @@ private[delta] class QbeastMetadataOperation extends ImplicitMetadataOperation { val dataSchema = StructType(schema.fields.map(field => field.copy(nullable = true, dataType = asNullable(field.dataType)))) - val mergedSchema = if (isOverwriteMode && canOverwriteSchema) { - dataSchema - } else { - SchemaMergingUtils.mergeSchemas(txn.metadata.schema, dataSchema) - } - val normalizedPartitionCols = - Seq.empty + val mergedSchema = + if (isOverwriteMode && canOverwriteSchema) dataSchema + else SchemaMergingUtils.mergeSchemas(txn.metadata.schema, dataSchema) + + val normalizedPartitionCols = Seq.empty // Merged schema will contain additional columns at the end val isNewSchema: Boolean = txn.metadata.schema != mergedSchema @@ -120,15 +131,17 @@ private[delta] class QbeastMetadataOperation extends ImplicitMetadataOperation { if (txn.readVersion == -1) Map.empty else if (isOverwriteMode) overwriteQbeastConfiguration(txn.metadata.configuration) else txn.metadata.configuration + // Qbeast configuration metadata - val configuration = if (isNewRevision || isOverwriteMode) { - updateQbeastRevision(baseConfiguration, latestRevision) - } else if (isOptimizeOperation) { - updateQbeastReplicatedSet( - baseConfiguration, - latestRevision, - tableChanges.announcedOrReplicatedSet) - } else { baseConfiguration } + val configuration = + if (isNewRevision || isOverwriteMode) { + updateQbeastRevision(baseConfiguration, latestRevision) + } else if (isOptimizeOperation) { + updateQbeastReplicatedSet( + baseConfiguration, + latestRevision, + tableChanges.announcedOrReplicatedSet) + } else baseConfiguration if (txn.readVersion == -1) { super.updateMetadata( @@ -163,15 +176,12 @@ private[delta] class QbeastMetadataOperation extends ImplicitMetadataOperation { } else if (isNewSchema) { recordDeltaEvent(txn.deltaLog, "delta.schemaValidation.failure") val errorBuilder = new MetadataMismatchErrorBuilder - if (isOverwriteMode) { - errorBuilder.addOverwriteBit() - } else { + if (isOverwriteMode) errorBuilder.addOverwriteBit() + else { errorBuilder.addSchemaMismatch(txn.metadata.schema, dataSchema, txn.metadata.id) } errorBuilder.finalizeAndThrow(spark.sessionState.conf) - } else { - txn.updateMetadata(txn.metadata.copy(configuration = configuration)) - } + } else txn.updateMetadata(txn.metadata.copy(configuration = configuration)) } override protected val canMergeSchema: Boolean = true From a2218fffe9fee14c403801a6af1177cf72f3f34f Mon Sep 17 00:00:00 2001 From: Jiawei Date: Tue, 24 Jan 2023 12:55:18 +0100 Subject: [PATCH 49/76] Add staging revision during first write, idempotent conversion --- .../spark/delta/QbeastMetadataOperation.scala | 17 +++++++++-------- .../commands/ConvertToQbeastCommand.scala | 19 +++++++------------ 2 files changed, 16 insertions(+), 20 deletions(-) diff --git a/src/main/scala/io/qbeast/spark/delta/QbeastMetadataOperation.scala b/src/main/scala/io/qbeast/spark/delta/QbeastMetadataOperation.scala index 1ca3eedce..1c7136472 100644 --- a/src/main/scala/io/qbeast/spark/delta/QbeastMetadataOperation.scala +++ b/src/main/scala/io/qbeast/spark/delta/QbeastMetadataOperation.scala @@ -3,6 +3,7 @@ */ package io.qbeast.spark.delta +import io.qbeast.core.model.Revision.stagingID import io.qbeast.core.model.{ReplicatedSet, Revision, TableChanges, mapper} import io.qbeast.spark.utils.MetadataConfig import io.qbeast.spark.utils.MetadataConfig.{lastRevisionID, revision} @@ -80,27 +81,27 @@ private[delta] class QbeastMetadataOperation extends ImplicitMetadataOperation { baseConfiguration: Configuration, newRevision: Revision): Configuration = { val newRevisionID = newRevision.revisionID - val isFirstWrite = newRevisionID == 1 + // Add staging revision if necessary + val stagingRevisionKey = s"$revision.$stagingID" + val addStagingRevision = + newRevisionID == 1 && !baseConfiguration.contains(stagingRevisionKey) val configuration = - if (isFirstWrite) { - // Add staging revision during first write + if (!addStagingRevision) baseConfiguration + else { val stagingRevision = Revision.emptyRevision( newRevision.tableID, newRevision.desiredCubeSize, newRevision.columnTransformers.map(_.columnName)) baseConfiguration - .updated( - s"$revision.${stagingRevision.revisionID}", - mapper.writeValueAsString(stagingRevision)) - } else baseConfiguration + .updated(stagingRevisionKey, mapper.writeValueAsString(stagingRevision)) + } // Update latest revision id and add new revision to metadata configuration .updated(lastRevisionID, newRevisionID.toString) .updated(s"$revision.$newRevisionID", mapper.writeValueAsString(newRevision)) - } def updateQbeastMetadata( diff --git a/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala b/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala index 77a3d9dd2..b79279784 100644 --- a/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala +++ b/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala @@ -3,7 +3,6 @@ */ package io.qbeast.spark.internal.commands -import io.qbeast.core.model.Revision.stagingID import io.qbeast.core.model._ import io.qbeast.spark.delta.DeltaQbeastSnapshot import io.qbeast.spark.utils.MetadataConfig.{lastRevisionID, revision} @@ -55,20 +54,19 @@ case class ConvertToQbeastCommand( val deltaLog = DeltaLog.forTable(spark, tableId.table) val qbeastSnapshot = DeltaQbeastSnapshot(deltaLog.snapshot) val isQbeast = qbeastSnapshot.loadAllRevisions.nonEmpty - val isConverted = qbeastSnapshot.existsRevision(stagingID) - if (isConverted) { + if (isQbeast) { logInfo("The table you are trying to convert is already a qbeast table") } else { fileFormat match { // Convert parquet to delta case "parquet" => convertParquetToDelta(spark, tableId.quotedString) - case "delta" | "qbeast" => + case "delta" => case _ => throw AnalysisExceptionFactory.create(s"Unsupported file format: $fileFormat") } // Convert delta to qbeast - val deltaLog = DeltaLog.forTable(spark, tableId.table) + deltaLog.update() val txn = deltaLog.startTransaction() @@ -80,13 +78,10 @@ case class ConvertToQbeastCommand( val isOverwritingSchema = txn.metadata.partitionColumns.nonEmpty // Update revision map - var updatedConf = - if (isQbeast) txn.metadata.configuration - // Create latestRevisionID for the table has no existing Revisions - else txn.metadata.configuration.updated(lastRevisionID, revisionID.toString) - - updatedConf = updatedConf - .updated(s"$revision.$revisionID", mapper.writeValueAsString(convRevision)) + val updatedConf = + txn.metadata.configuration + .updated(lastRevisionID, revisionID.toString) + .updated(s"$revision.$revisionID", mapper.writeValueAsString(convRevision)) val newMetadata = txn.metadata.copy(configuration = updatedConf, partitionColumns = Seq.empty) From 08c4aaf5ac63ae7057d9639fcc8a900386d8c2c6 Mon Sep 17 00:00:00 2001 From: Jiawei Date: Tue, 24 Jan 2023 14:04:01 +0100 Subject: [PATCH 50/76] Remove staging revision from loadRevisionAt --- src/main/scala/io/qbeast/spark/delta/DeltaQbeastSnapshot.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/scala/io/qbeast/spark/delta/DeltaQbeastSnapshot.scala b/src/main/scala/io/qbeast/spark/delta/DeltaQbeastSnapshot.scala index 88c75c32d..3987c6b11 100644 --- a/src/main/scala/io/qbeast/spark/delta/DeltaQbeastSnapshot.scala +++ b/src/main/scala/io/qbeast/spark/delta/DeltaQbeastSnapshot.scala @@ -153,7 +153,7 @@ case class DeltaQbeastSnapshot(private val snapshot: Snapshot) extends QbeastSna * @return the latest Revision at a concrete timestamp */ override def loadRevisionAt(timestamp: Long): Revision = { - revisionsMap.values.find(_.timestamp <= timestamp).getOrElse { + (revisionsMap - stagingID).values.find(_.timestamp <= timestamp).getOrElse { throw AnalysisExceptionFactory.create(s"No space revision available before $timestamp") } } From 1ff0dd237e633a91eda11b7a5c48ffb6761e7691 Mon Sep 17 00:00:00 2001 From: Jiawei Date: Tue, 24 Jan 2023 16:08:54 +0100 Subject: [PATCH 51/76] Allow the creation of empty cubes for the staging revision --- .../scala/io/qbeast/spark/delta/IndexStatusBuilder.scala | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/src/main/scala/io/qbeast/spark/delta/IndexStatusBuilder.scala b/src/main/scala/io/qbeast/spark/delta/IndexStatusBuilder.scala index 78a1071b3..b0e4d7de3 100644 --- a/src/main/scala/io/qbeast/spark/delta/IndexStatusBuilder.scala +++ b/src/main/scala/io/qbeast/spark/delta/IndexStatusBuilder.scala @@ -65,11 +65,7 @@ private[delta] class IndexStatusBuilder( addFile.modificationTime)) .toIndexedSeq - if (blocks.nonEmpty) { - SortedMap(root -> CubeStatus(root, maxWeight, maxWeight.fraction, blocks)) - } else { - SortedMap.empty[CubeId, CubeStatus] - } + SortedMap(root -> CubeStatus(root, maxWeight, maxWeight.fraction, blocks)) } /** From 08395d19337fe96ae3f42d316fc11865639f3cbf Mon Sep 17 00:00:00 2001 From: Jiawei Date: Tue, 24 Jan 2023 16:10:16 +0100 Subject: [PATCH 52/76] Compaction with no dataChange --- src/main/scala/io/qbeast/spark/delta/writer/Compactor.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/scala/io/qbeast/spark/delta/writer/Compactor.scala b/src/main/scala/io/qbeast/spark/delta/writer/Compactor.scala index f972af982..241af2671 100644 --- a/src/main/scala/io/qbeast/spark/delta/writer/Compactor.scala +++ b/src/main/scala/io/qbeast/spark/delta/writer/Compactor.scala @@ -98,7 +98,7 @@ case class Compactor( partitionValues = Map(), size = fileStatus.getLen, modificationTime = fileStatus.getModificationTime, - dataChange = true, + dataChange = false, stats = "", tags = tags) From b009c37abe9a162d9de725463f2ac1afb1d6863c Mon Sep 17 00:00:00 2001 From: Jiawei Date: Tue, 24 Jan 2023 18:10:40 +0100 Subject: [PATCH 53/76] Include staging revision and use initial revisionID = 1 for test --- .../scala/io/qbeast/spark/index/NewRevisionTest.scala | 7 +++++-- .../qbeast/spark/index/query/QueryExecutorTest.scala | 6 ++++-- .../spark/index/query/QuerySpecBuilderTest.scala | 10 ++++++---- .../spark/utils/QbeastCompactionIntegrationTest.scala | 6 ++++-- .../spark/utils/QbeastSparkCorrectnessTest.scala | 3 ++- .../scala/io/qbeast/spark/utils/QbeastTableTest.scala | 7 ++++--- 6 files changed, 25 insertions(+), 14 deletions(-) diff --git a/src/test/scala/io/qbeast/spark/index/NewRevisionTest.scala b/src/test/scala/io/qbeast/spark/index/NewRevisionTest.scala index 844aed9c4..261055d8c 100644 --- a/src/test/scala/io/qbeast/spark/index/NewRevisionTest.scala +++ b/src/test/scala/io/qbeast/spark/index/NewRevisionTest.scala @@ -10,6 +10,7 @@ import org.scalatest.PrivateMethodTester import org.scalatest.flatspec.AnyFlatSpec import org.scalatest.matchers.should.Matchers import io.qbeast.TestClasses._ +import io.qbeast.core.model.Revision.isStaging class NewRevisionTest extends AnyFlatSpec @@ -42,7 +43,8 @@ class NewRevisionTest val qbeastSnapshot = delta.DeltaQbeastSnapshot(deltaLog.snapshot) val spaceRevisions = qbeastSnapshot.loadAllRevisions - spaceRevisions.size shouldBe spaceMultipliers.length + // Including the staging revision + spaceRevisions.size shouldBe spaceMultipliers.length + 1 } @@ -121,7 +123,8 @@ class NewRevisionTest val deltaLog = DeltaLog.forTable(spark, tmpDir) val qbeastSnapshot = delta.DeltaQbeastSnapshot(deltaLog.snapshot) - qbeastSnapshot.loadAllRevisions.length shouldBe 2 + // Including the staging revision + qbeastSnapshot.loadAllRevisions.size shouldBe 3 qbeastSnapshot.loadLatestRevision.desiredCubeSize shouldBe cubeSize2 }) diff --git a/src/test/scala/io/qbeast/spark/index/query/QueryExecutorTest.scala b/src/test/scala/io/qbeast/spark/index/query/QueryExecutorTest.scala index 8b3ad317a..ac6d11b7e 100644 --- a/src/test/scala/io/qbeast/spark/index/query/QueryExecutorTest.scala +++ b/src/test/scala/io/qbeast/spark/index/query/QueryExecutorTest.scala @@ -1,6 +1,7 @@ package io.qbeast.spark.index.query import io.qbeast.TestClasses.T2 +import io.qbeast.core.model.Revision.isStaging import io.qbeast.core.model.{CubeId, QbeastBlock, Weight, WeightRange} import io.qbeast.spark.QbeastIntegrationTestSpec import io.qbeast.spark.delta.DeltaQbeastSnapshot @@ -121,7 +122,8 @@ class QueryExecutorTest extends QbeastIntegrationTestSpec { val deltaLog = DeltaLog.forTable(spark, tmpdir) val qbeastSnapshot = DeltaQbeastSnapshot(deltaLog.snapshot) - qbeastSnapshot.loadAllRevisions.size shouldBe 2 + // Including the staging revision + qbeastSnapshot.loadAllRevisions.size shouldBe 3 val filters = Seq.empty @@ -180,7 +182,7 @@ class QueryExecutorTest extends QbeastIntegrationTestSpec { val deltaLog = DeltaLog.forTable(spark, tmpdir) val qbeastSnapshot = DeltaQbeastSnapshot(deltaLog.snapshot) - val revision = qbeastSnapshot.loadAllRevisions.head + val revision = qbeastSnapshot.loadLatestRevision val indexStatus = qbeastSnapshot.loadIndexStatus(revision.revisionID) val innerCubesLevel1 = diff --git a/src/test/scala/io/qbeast/spark/index/query/QuerySpecBuilderTest.scala b/src/test/scala/io/qbeast/spark/index/query/QuerySpecBuilderTest.scala index 863035a64..6f40af862 100644 --- a/src/test/scala/io/qbeast/spark/index/query/QuerySpecBuilderTest.scala +++ b/src/test/scala/io/qbeast/spark/index/query/QuerySpecBuilderTest.scala @@ -30,7 +30,7 @@ class QuerySpecBuilderTest val columnTransformers = Seq(Transformer("linear", "id", IntegerDataType)).toIndexedSeq Revision( - 0, + 1, System.currentTimeMillis(), QTableID("test"), 100, @@ -96,7 +96,7 @@ class QuerySpecBuilderTest val columnTransformers = Seq(Transformer("hashing", "name", StringDataType)).toIndexedSeq val revision = Revision( - 0, + 1, System.currentTimeMillis(), QTableID("test"), 100, @@ -155,7 +155,8 @@ class QuerySpecBuilderTest "extractQuerySpace" should "filter Revision properly" in withSpark(_ => { // Revision space ranges: [0, 10], [10, 20], [20, 30], [30, 40] - val revisions = (0 to 3).map(i => createRevision(10 * i, 10 * (i + 1)).copy(revisionID = i)) + val revisions = + (0 to 3).map(i => createRevision(10 * i, 10 * (i + 1)).copy(revisionID = i + 1)) val expressions = Seq( ("id < -1", 0), @@ -179,7 +180,8 @@ class QuerySpecBuilderTest it should "retrieve all revisions with no filter expressions" in withSpark(_ => { // Revision space ranges: [0, 10], [10, 20], [20, 30], [30, 40] - val revisions = (0 to 3).map(i => createRevision(10 * i, 10 * (i + 1)).copy(revisionID = i)) + val revisions = + (0 to 3).map(i => createRevision(10 * i, 10 * (i + 1)).copy(revisionID = i + 1)) revisions.count { revision => val querySpec = new QuerySpecBuilder(Seq.empty).build(revision) diff --git a/src/test/scala/io/qbeast/spark/utils/QbeastCompactionIntegrationTest.scala b/src/test/scala/io/qbeast/spark/utils/QbeastCompactionIntegrationTest.scala index 23f851c76..b48a5edf2 100644 --- a/src/test/scala/io/qbeast/spark/utils/QbeastCompactionIntegrationTest.scala +++ b/src/test/scala/io/qbeast/spark/utils/QbeastCompactionIntegrationTest.scala @@ -153,7 +153,8 @@ class QbeastCompactionIntegrationTest extends QbeastIntegrationTestSpec { writeTestDataInBatches(newData, tmpDir, 4) val tableId = QTableID(tmpDir) - SparkDeltaMetadataManager.loadSnapshot(tableId).loadAllRevisions.size shouldBe 2 + // Including the staging revision + SparkDeltaMetadataManager.loadSnapshot(tableId).loadAllRevisions.size shouldBe 3 // Count files written for each revision val allFiles = DeltaLog.forTable(spark, tmpDir).snapshot.allFiles @@ -192,7 +193,8 @@ class QbeastCompactionIntegrationTest extends QbeastIntegrationTestSpec { writeTestDataInBatches(newData, tmpDir, 4) val tableId = QTableID(tmpDir) - SparkDeltaMetadataManager.loadSnapshot(tableId).loadAllRevisions.size shouldBe 2 + // Including the staging revision + SparkDeltaMetadataManager.loadSnapshot(tableId).loadAllRevisions.size shouldBe 3 // Count files written for each revision val allFiles = DeltaLog.forTable(spark, tmpDir).snapshot.allFiles diff --git a/src/test/scala/io/qbeast/spark/utils/QbeastSparkCorrectnessTest.scala b/src/test/scala/io/qbeast/spark/utils/QbeastSparkCorrectnessTest.scala index acce7fb03..87d5cd69a 100644 --- a/src/test/scala/io/qbeast/spark/utils/QbeastSparkCorrectnessTest.scala +++ b/src/test/scala/io/qbeast/spark/utils/QbeastSparkCorrectnessTest.scala @@ -84,7 +84,8 @@ class QbeastSparkCorrectnessTest extends QbeastIntegrationTestSpec { val deltaLog = DeltaLog.forTable(spark, tmpDir) val qbeastSnapshot = DeltaQbeastSnapshot(deltaLog.snapshot) - qbeastSnapshot.loadAllRevisions.size shouldBe 1 + // Include the staging revision + qbeastSnapshot.loadAllRevisions.size shouldBe 2 qbeastSnapshot.loadLatestRevision.revisionID shouldBe 1L } diff --git a/src/test/scala/io/qbeast/spark/utils/QbeastTableTest.scala b/src/test/scala/io/qbeast/spark/utils/QbeastTableTest.scala index 90af3270f..fe8989ab3 100644 --- a/src/test/scala/io/qbeast/spark/utils/QbeastTableTest.scala +++ b/src/test/scala/io/qbeast/spark/utils/QbeastTableTest.scala @@ -82,7 +82,7 @@ class QbeastTableTest extends QbeastIntegrationTestSpec { val revision1 = createDF(spark) val columnsToIndex = Seq("age", "val2") val cubeSize = 100 - // WRITE SOME DATA + // WRITE SOME DATA, adds revisionIDs 0 and 1 writeTestData(revision1, columnsToIndex, cubeSize, tmpDir) val revision2 = revision1.withColumn("age", col("age") * 2) @@ -92,8 +92,9 @@ class QbeastTableTest extends QbeastIntegrationTestSpec { writeTestData(revision3, columnsToIndex, cubeSize, tmpDir, "append") val qbeastTable = QbeastTable.forPath(spark, tmpDir) - qbeastTable.revisionsIDs().size shouldBe 3 - qbeastTable.revisionsIDs() shouldBe Seq(1L, 2L, 3L) + // Including the staging revision + qbeastTable.revisionsIDs().size shouldBe 4 + qbeastTable.revisionsIDs() == Seq(0L, 1L, 2L, 3L) } } From 7b976a2a58e0fb3b458e6c41796d2138f8d8b090 Mon Sep 17 00:00:00 2001 From: Jiawei Date: Tue, 24 Jan 2023 18:19:53 +0100 Subject: [PATCH 54/76] Check for staging revision via Transformer type --- .../scala/io/qbeast/core/model/RevisionClasses.scala | 9 ++++++++- .../scala/io/qbeast/spark/delta/IndexStatusBuilder.scala | 2 +- .../scala/io/qbeast/spark/delta/writer/Compactor.scala | 2 +- .../io/qbeast/spark/index/query/QuerySpecBuilder.scala | 2 +- 4 files changed, 11 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/io/qbeast/core/model/RevisionClasses.scala b/core/src/main/scala/io/qbeast/core/model/RevisionClasses.scala index 2b2c25868..3ac5e470d 100644 --- a/core/src/main/scala/io/qbeast/core/model/RevisionClasses.scala +++ b/core/src/main/scala/io/qbeast/core/model/RevisionClasses.scala @@ -47,6 +47,13 @@ object Revision { def isStaging(revisionID: RevisionID): Boolean = revisionID == stagingID + def isStaging(revision: Revision): Boolean = + isStaging(revision.revisionID) && + revision.columnTransformers.forall { + case _: EmptyTransformer => true + case _ => false + } + /** * Create a new first revision for a table * @param tableID the table identifier @@ -218,7 +225,7 @@ case class IndexStatus( extends Serializable { def addAnnouncements(newAnnouncedSet: Set[CubeId]): IndexStatus = { - if (isStaging(revision.revisionID)) this + if (isStaging(revision)) this else copy(announcedSet = announcedSet ++ newAnnouncedSet) } diff --git a/src/main/scala/io/qbeast/spark/delta/IndexStatusBuilder.scala b/src/main/scala/io/qbeast/spark/delta/IndexStatusBuilder.scala index b0e4d7de3..321dea183 100644 --- a/src/main/scala/io/qbeast/spark/delta/IndexStatusBuilder.scala +++ b/src/main/scala/io/qbeast/spark/delta/IndexStatusBuilder.scala @@ -38,7 +38,7 @@ private[delta] class IndexStatusBuilder( def build(): IndexStatus = { val cubeStatus = - if (isStaging(revision.revisionID)) stagingCubeStatuses + if (isStaging(revision)) stagingCubeStatuses else buildCubesStatuses IndexStatus( diff --git a/src/main/scala/io/qbeast/spark/delta/writer/Compactor.scala b/src/main/scala/io/qbeast/spark/delta/writer/Compactor.scala index 241af2671..9ea07756f 100644 --- a/src/main/scala/io/qbeast/spark/delta/writer/Compactor.scala +++ b/src/main/scala/io/qbeast/spark/delta/writer/Compactor.scala @@ -61,7 +61,7 @@ case class Compactor( // Update the tags of the block with the information of the cubeBlocks val tags: Map[String, String] = - if (isStaging(revision.revisionID)) null + if (isStaging(revision)) null else { Map( TagUtils.cube -> cubeId.string, diff --git a/src/main/scala/io/qbeast/spark/index/query/QuerySpecBuilder.scala b/src/main/scala/io/qbeast/spark/index/query/QuerySpecBuilder.scala index 1b1c4dd00..fbe09d1a1 100644 --- a/src/main/scala/io/qbeast/spark/index/query/QuerySpecBuilder.scala +++ b/src/main/scala/io/qbeast/spark/index/query/QuerySpecBuilder.scala @@ -162,7 +162,7 @@ private[spark] class QuerySpecBuilder(sparkFilters: Seq[Expression]) extends Ser def build(revision: Revision): QuerySpec = { val (weightRange, querySpace) = - if (isStaging(revision.revisionID)) { + if (isStaging(revision)) { (WeightRange(Weight(Int.MinValue), Weight(Int.MaxValue)), EmptySpace()) } else { val qbeastFilters = extractDataFilters(sparkFilters, revision) From 85a6fcd900726095e35ff9ab9f26d3432e16f470 Mon Sep 17 00:00:00 2001 From: Jiawei Date: Tue, 24 Jan 2023 18:20:50 +0100 Subject: [PATCH 55/76] Avoid appending on the staging revision --- .../io/qbeast/spark/table/IndexedTable.scala | 27 +++++++++++-------- 1 file changed, 16 insertions(+), 11 deletions(-) diff --git a/src/main/scala/io/qbeast/spark/table/IndexedTable.scala b/src/main/scala/io/qbeast/spark/table/IndexedTable.scala index 8e21e9c6d..370f91652 100644 --- a/src/main/scala/io/qbeast/spark/table/IndexedTable.scala +++ b/src/main/scala/io/qbeast/spark/table/IndexedTable.scala @@ -4,6 +4,7 @@ package io.qbeast.spark.table import io.qbeast.core.keeper.Keeper +import io.qbeast.core.model.Revision.isStaging import io.qbeast.core.model._ import io.qbeast.spark.delta.CubeDataLoader import io.qbeast.spark.index.QbeastColumns @@ -153,7 +154,7 @@ private[table] class IndexedTableImpl( * Add the required indexing parameters when the SaveMode is Append. * The user-provided parameters are respected. * @param latestRevision the latest revision - * @param params the parameters required for indexing + * @param parameters the parameters required for indexing */ private def addRequiredParams( latestRevision: Revision, @@ -175,15 +176,20 @@ private[table] class IndexedTableImpl( append: Boolean): BaseRelation = { val indexStatus = if (exists && append) { - val latestIndexStatus = snapshot.loadLatestIndexStatus - val updatedParameters = addRequiredParams(latestIndexStatus.revision, parameters) - if (checkRevisionParameters(QbeastOptions(updatedParameters), latestIndexStatus)) { - latestIndexStatus + val latestRevision = snapshot.loadLatestRevision + val updatedParameters = addRequiredParams(latestRevision, parameters) + if (isStaging(latestRevision)) { + IndexStatus(revisionBuilder.createNewRevision(tableID, data.schema, updatedParameters)) } else { - val oldRevisionID = latestIndexStatus.revision.revisionID - val newRevision = revisionBuilder - .createNextRevision(tableID, data.schema, updatedParameters, oldRevisionID) - IndexStatus(newRevision) + val latestIndexStatus = snapshot.loadIndexStatus(latestRevision.revisionID) + if (checkRevisionParameters(QbeastOptions(updatedParameters), latestIndexStatus)) { + latestIndexStatus + } else { + val oldRevisionID = latestIndexStatus.revision.revisionID + val newRevision = revisionBuilder + .createNextRevision(tableID, data.schema, updatedParameters, oldRevisionID) + IndexStatus(newRevision) + } } } else { IndexStatus(revisionBuilder.createNewRevision(tableID, data.schema, parameters)) @@ -219,7 +225,6 @@ private[table] class IndexedTableImpl( /** * Creates a QbeastBaseRelation for the given table. - * @param tableID the table identifier * @return the QbeastBaseRelation */ private def createQbeastBaseRelation(): BaseRelation = { @@ -332,7 +337,7 @@ private[table] class IndexedTableImpl( val schema = metadataManager.loadCurrentSchema(tableID) val currentIndexStatus = snapshot.loadIndexStatus(revisionID) - metadataManager.updateWithTransaction(tableID, schema, true) { + metadataManager.updateWithTransaction(tableID, schema, append = true) { // There's no affected table changes on compaction, so we send an empty object val tableChanges = BroadcastedTableChanges(None, currentIndexStatus, Map.empty) val fileActions = From abc6ea3ccf7b243f82d81d27b7aeb6011c094bfc Mon Sep 17 00:00:00 2001 From: Jiawei Date: Tue, 24 Jan 2023 18:22:08 +0100 Subject: [PATCH 56/76] Complete conversion test --- .../spark/utils/ConvertToQbeastTest.scala | 144 +++++++++++------- 1 file changed, 93 insertions(+), 51 deletions(-) diff --git a/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala b/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala index 294fb5c09..4ff8905d5 100644 --- a/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala +++ b/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala @@ -1,23 +1,28 @@ package io.qbeast.spark.utils +import io.qbeast.core.model.Revision.{isStaging, stagingID} +import io.qbeast.spark.delta.DeltaQbeastSnapshot import io.qbeast.spark.internal.commands.ConvertToQbeastCommand import io.qbeast.spark.{QbeastIntegrationTestSpec, QbeastTable} +import org.apache.spark.sql.delta.DeltaLog import org.apache.spark.sql.{AnalysisException, DataFrame, SparkSession} import org.scalatest.PrivateMethodTester class ConvertToQbeastTest extends QbeastIntegrationTestSpec with PrivateMethodTester { val columnsToIndex: Seq[String] = Seq("user_id", "price", "event_type") val partitionColumns: Option[String] = Some("event_type STRING") + val numSparkPartitions = 20 val dataSize = 50000 + val dcs = 5000 - def convertFormatsFromTo( + def convertFromFormat( spark: SparkSession, format: String, tablePath: String, partitionColumns: Option[String] = None, columnsToIndex: Seq[String] = columnsToIndex, - desiredCubeSize: Int = 50000): DataFrame = { - val data = loadTestData(spark).limit(dataSize) + desiredCubeSize: Int = dcs): DataFrame = { + val data = loadTestData(spark).limit(dataSize).repartition(numSparkPartitions) if (partitionColumns.isDefined) { val cols = partitionColumns.get.split(", ").map(_.split(" ").head) @@ -31,58 +36,77 @@ class ConvertToQbeastTest extends QbeastIntegrationTestSpec with PrivateMethodTe spark.read.format("qbeast").load(tablePath) } + def getQbeastSnapshot(spark: SparkSession, dir: String): DeltaQbeastSnapshot = { + val deltaLog = DeltaLog.forTable(spark, dir) + DeltaQbeastSnapshot(deltaLog.snapshot) + } + "ConvertToQbeastCommand" should "convert a delta table" in withSparkAndTmpDir((spark, tmpDir) => { - val convertedTable = convertFormatsFromTo(spark, "delta", tmpDir) + val convertedTable = convertFromFormat(spark, "delta", tmpDir) - val metrics = QbeastTable.forPath(spark, tmpDir).getIndexMetrics() convertedTable.count shouldBe dataSize spark.read.parquet(tmpDir).count shouldBe dataSize - metrics.elementCount shouldBe 0 - metrics.cubeCount shouldBe 0 + val indexStatus = getQbeastSnapshot(spark, tmpDir).loadIndexStatus(stagingID) + // All non-qbeast files are considered staging files and are placed + // directly into the staging revision(RevisionID = 0) + indexStatus.cubesStatuses.size shouldBe 1 + indexStatus.cubesStatuses.head._2.files.size shouldBe numSparkPartitions }) it should "convert a PARTITIONED delta table" in withSparkAndTmpDir((spark, tmpDir) => { - val convertedTable = convertFormatsFromTo(spark, "delta", tmpDir, partitionColumns) + val convertedTable = convertFromFormat(spark, "delta", tmpDir, partitionColumns) - val metrics = QbeastTable.forPath(spark, tmpDir).getIndexMetrics() convertedTable.count shouldBe dataSize - metrics.elementCount shouldBe 0 - metrics.cubeCount shouldBe 0 + spark.read.parquet(tmpDir).count shouldBe dataSize + + val indexStatus = getQbeastSnapshot(spark, tmpDir).loadIndexStatus(stagingID) + indexStatus.cubesStatuses.size shouldBe 1 }) it should "convert a parquet table" in withSparkAndTmpDir((spark, tmpDir) => { - val convertedTable = convertFormatsFromTo(spark, "parquet", tmpDir) + val convertedTable = convertFromFormat(spark, "parquet", tmpDir) - val metrics = QbeastTable.forPath(spark, tmpDir).getIndexMetrics() convertedTable.count shouldBe dataSize - metrics.elementCount shouldBe 0 - metrics.cubeCount shouldBe 0 + spark.read.parquet(tmpDir).count shouldBe dataSize + + val indexStatus = getQbeastSnapshot(spark, tmpDir).loadIndexStatus(stagingID) + indexStatus.cubesStatuses.size shouldBe 1 + indexStatus.cubesStatuses.head._2.files.size shouldBe numSparkPartitions }) it should "convert a PARTITIONED parquet table" in withSparkAndTmpDir((spark, tmpDir) => { - val convertedTable = convertFormatsFromTo(spark, "parquet", tmpDir, partitionColumns) + val convertedTable = convertFromFormat(spark, "parquet", tmpDir, partitionColumns) - val metrics = QbeastTable.forPath(spark, tmpDir).getIndexMetrics() convertedTable.count shouldBe dataSize - metrics.elementCount shouldBe 0 - metrics.cubeCount shouldBe 0 + spark.read.parquet(tmpDir).count shouldBe dataSize + + val indexStatus = getQbeastSnapshot(spark, tmpDir).loadIndexStatus(stagingID) + indexStatus.cubesStatuses.size shouldBe 1 }) - it should "not try to convert a qbeast table" in withSparkAndTmpDir((spark, tmpDir) => { + it should "add a staging revision for a qbeast table" in withSparkAndTmpDir((spark, tmpDir) => { val data = loadTestData(spark).limit(dataSize) data.write .format("qbeast") .option("columnsToIndex", columnsToIndex.mkString(",")) - .option("cubeSize", 5000) + .option("cubeSize", dcs) .save(tmpDir) - ConvertToQbeastCommand(s"parquet.`$tmpDir`", columnsToIndex, 5000) + val qsBefore = getQbeastSnapshot(spark, tmpDir) + val revisionsBefore = qsBefore.loadAllRevisions + + ConvertToQbeastCommand(s"parquet.`$tmpDir`", columnsToIndex, dcs) + + val qsAfter = getQbeastSnapshot(spark, tmpDir) + val revisionsAfter = qsAfter.loadAllRevisions + spark.read.parquet(tmpDir).count shouldBe dataSize + revisionsAfter shouldBe revisionsBefore }) it should "throw an exception when attempting to convert an unsupported format" in @@ -94,30 +118,8 @@ class ConvertToQbeastTest extends QbeastIntegrationTestSpec with PrivateMethodTe ConvertToQbeastCommand(s"json.`$tmpDir`", columnsToIndex).run(spark)) }) - "Analyzing the conversion revision" should "do nothing" in withSparkAndTmpDir( - (spark, tmpDir) => { - convertFormatsFromTo(spark, "parquet", tmpDir) - val qbeastTable = QbeastTable.forPath(spark, tmpDir) - - qbeastTable.analyze().isEmpty shouldBe true - }) - - "Optimizing the conversion revision" should "do nothing" in - withSparkAndTmpDir((spark, tmpDir) => { - convertFormatsFromTo(spark, "parquet", tmpDir) - val qbeastTable = QbeastTable.forPath(spark, tmpDir) - - qbeastTable.analyze() - qbeastTable.optimize() - - val metrics = QbeastTable.forPath(spark, tmpDir).getIndexMetrics() - spark.read.parquet(tmpDir).count shouldBe dataSize - metrics.cubeCount shouldBe 0 - metrics.elementCount shouldBe 0 - }) - it should "preserve sampling accuracy" in withSparkAndTmpDir((spark, tmpDir) => { - convertFormatsFromTo(spark, "parquet", tmpDir) + convertFromFormat(spark, "parquet", tmpDir) val qbeastTable = QbeastTable.forPath(spark, tmpDir) qbeastTable.analyze() @@ -136,20 +138,60 @@ class ConvertToQbeastTest extends QbeastIntegrationTestSpec with PrivateMethodTe }) }) - "Compacting the conversion revision" should "do nothing" in + "Appending to a converted table" should "create a new non-staging revision" in withSparkAndTmpDir( + (spark, tmpDir) => { + convertFromFormat(spark, "parquet", tmpDir) + val df = loadTestData(spark).limit(dataSize) + df.write + .mode("append") + .format("qbeast") + .option("columnsToIndex", columnsToIndex.mkString(",")) + .option("cubeSize", dcs) + .save(tmpDir) + + val qs = getQbeastSnapshot(spark, tmpDir) + val rev = qs.loadLatestRevision + isStaging(rev) shouldBe false + }) + + "Analyzing the staging revision" should "not change the ANNOUNCED set" in + withSparkAndTmpDir((spark, tmpDir) => { + convertFromFormat(spark, "parquet", tmpDir) + val qbeastTable = QbeastTable.forPath(spark, tmpDir) + qbeastTable.analyze() + + val qs = getQbeastSnapshot(spark, tmpDir) + qs.loadLatestIndexStatus.announcedSet.isEmpty shouldBe true + }) + + "Optimizing the staging revision" should "not replicate any data" in + withSparkAndTmpDir((spark, tmpDir) => { + convertFromFormat(spark, "parquet", tmpDir) + val qbeastTable = QbeastTable.forPath(spark, tmpDir) + + qbeastTable.analyze() + qbeastTable.optimize() + + spark.read.parquet(tmpDir).count shouldBe dataSize + }) + + "Compacting the staging revision" should "reduce the number of delta AddFiles" in withExtendedSparkAndTmpDir( sparkConfWithSqlAndCatalog .set("spark.qbeast.compact.minFileSize", "1") .set("spark.qbeast.compact.maxFileSize", "2000000")) { (spark, tmpDir) => { - convertFormatsFromTo(spark, "parquet", tmpDir) + convertFromFormat(spark, "parquet", tmpDir) val qbeastTable = QbeastTable.forPath(spark, tmpDir) qbeastTable.compact() - val metrics = QbeastTable.forPath(spark, tmpDir).getIndexMetrics() - spark.read.parquet(tmpDir).count shouldBe dataSize - metrics.cubeCount shouldBe 0 - metrics.elementCount shouldBe 0 + spark.read.parquet(tmpDir).count shouldBe >(dataSize.toLong) + + val qs = getQbeastSnapshot(spark, tmpDir) + val stagingCs = qs.loadLatestIndexStatus.cubesStatuses + + stagingCs.size shouldBe 1 + stagingCs.head._2.files.size shouldBe <(numSparkPartitions) } } } From a353645a5b27831037e22c567f0fa8cfecbb55a9 Mon Sep 17 00:00:00 2001 From: Jiawei Date: Tue, 24 Jan 2023 18:22:27 +0100 Subject: [PATCH 57/76] Complete hybrid table test --- .../spark/utils/QbeastDeltaStagingTest.scala | 85 ++++++++++++------- 1 file changed, 53 insertions(+), 32 deletions(-) diff --git a/src/test/scala/io/qbeast/spark/utils/QbeastDeltaStagingTest.scala b/src/test/scala/io/qbeast/spark/utils/QbeastDeltaStagingTest.scala index 9ef96b860..4a57ff08f 100644 --- a/src/test/scala/io/qbeast/spark/utils/QbeastDeltaStagingTest.scala +++ b/src/test/scala/io/qbeast/spark/utils/QbeastDeltaStagingTest.scala @@ -1,77 +1,98 @@ package io.qbeast.spark.utils -import io.qbeast.core.model.Revision.stagingRevisionID +import io.qbeast.TestClasses.T2 +import io.qbeast.core.model.Revision.stagingID import io.qbeast.spark.delta.DeltaQbeastSnapshot import io.qbeast.spark.{QbeastIntegrationTestSpec, QbeastTable} +import org.apache.spark.sql.SparkSession import org.apache.spark.sql.delta.DeltaLog class QbeastDeltaStagingTest extends QbeastIntegrationTestSpec { - it should "read a delta table as an entirely staged qbeast table" in - withSparkAndTmpDir((spark, tmpDir) => { - val df = loadTestData(spark).limit(10000) - df.write.format("delta").save(tmpDir) + val columnsToIndex: Seq[String] = Seq("a", "b") + val qDataSize = 10000 + val dDataSize = 10000 + val totalSize: Long = qDataSize + dDataSize + val numSparkPartitions = 20 + + def writeHybridTable(spark: SparkSession, dir: String): Unit = { + import spark.implicits._ + val qdf = (0 until qDataSize).map(i => T2(i, i)).toDF("a", "b") + qdf.write + .format("qbeast") + .option("columnsToIndex", columnsToIndex.mkString(",")) + .option("cubeSize", "5000") + .save(dir) + + val ddf = (qDataSize until qDataSize + dDataSize).map(i => T2(i, i)).toDF("a", "b") + ddf.repartition(numSparkPartitions).write.mode("append").format("delta").save(dir) + } + + "A qbeast + delta hybrid table" should "be read correctly" in withSparkAndTmpDir( + (spark, tmpDir) => { + writeHybridTable(spark, tmpDir) val numStagingRows = spark.read.format("qbeast").load(tmpDir).count - numStagingRows shouldBe 10000L + numStagingRows shouldBe totalSize + + val snapshot = DeltaLog.forTable(spark, tmpDir).snapshot + val qs = DeltaQbeastSnapshot(snapshot) + qs.loadAllRevisions.size shouldBe 2 + qs.existsRevision(stagingID) }) - it should "not analyze or optimize the staging revision" in withSparkAndTmpDir( + it should "not be altered by Optimizing the staging revision" in withSparkAndTmpDir( (spark, tmpDir) => { - val df = loadTestData(spark).limit(10000) - df.write.format("delta").save(tmpDir) + writeHybridTable(spark, tmpDir) val table = QbeastTable.forPath(spark, tmpDir) - table.analyze(stagingRevisionID) - table.optimize(stagingRevisionID) + table.analyze(stagingID) + table.optimize(stagingID) val allElements = spark.read.parquet(tmpDir).count - allElements shouldBe 10000 + allElements shouldBe totalSize val snapshot = DeltaLog.forTable(spark, tmpDir).snapshot val qbeastSnapshot = DeltaQbeastSnapshot(snapshot) - val stagingIndexStatus = qbeastSnapshot.loadIndexStatus(stagingRevisionID) + val stagingIndexStatus = qbeastSnapshot.loadIndexStatus(stagingID) + stagingIndexStatus.cubesStatuses.size shouldBe 1 stagingIndexStatus.replicatedOrAnnouncedSet.isEmpty shouldBe true }) it should "correctly compact the staging revision" in withExtendedSparkAndTmpDir( sparkConfWithSqlAndCatalog.set("spark.qbeast.compact.minFileSize", "1")) { (spark, tmpDir) => { - val df = loadTestData(spark) - - df.repartition(20).write.format("delta").save(tmpDir) + writeHybridTable(spark, tmpDir) val deltaLog = DeltaLog.forTable(spark, tmpDir) - val snapshot = deltaLog.snapshot - val numFilesBefore = snapshot.numOfFiles + val qsBefore = DeltaQbeastSnapshot(deltaLog.snapshot) + val numFilesBefore = qsBefore.loadIndexStatus(stagingID).cubesStatuses.head._2.files.size val table = QbeastTable.forPath(spark, tmpDir) - table.compact(stagingRevisionID) + table.compact(stagingID) - deltaLog.update() - val numFilesAfter = deltaLog.snapshot.numOfFiles + val qsAfter = DeltaQbeastSnapshot(deltaLog.update()) + val numFilesAfter = qsAfter.loadIndexStatus(stagingID).cubesStatuses.head._2.files.size numFilesAfter shouldBe <(numFilesBefore) val deltaCount = spark.read.format("delta").load(tmpDir).count() val qbeastCount = spark.read.format("qbeast").load(tmpDir).count() - deltaCount shouldBe 99986 - qbeastCount shouldBe 99986 + deltaCount shouldBe totalSize + qbeastCount shouldBe totalSize } } - it should "sample a qbeast staging table correctly" in withSparkAndTmpDir((spark, tmpDir) => { - val dataSize = 99986 - val df = loadTestData(spark) // 99986 - - df.write.format("delta").save(tmpDir) + it should "sample correctly" in withSparkAndTmpDir((spark, tmpDir) => { + writeHybridTable(spark, tmpDir) val qdf = spark.read.format("qbeast").load(tmpDir) - // We allow a 1% of tolerance in the sampling - val tolerance = 0.01 + val tolerance = 0.05 List(0.1, 0.2, 0.5, 0.7, 0.99).foreach(f => { - val result = qdf.sample(withReplacement = false, f).count().toDouble - result shouldBe (dataSize * f) +- dataSize * f * tolerance + val sampleSize = qdf.sample(withReplacement = false, f).count().toDouble + val margin = totalSize * f * tolerance + + sampleSize shouldBe (totalSize * f) +- margin }) }) From 4c4ae5934e5197e8818752b5c61302c77dfe836e Mon Sep 17 00:00:00 2001 From: Jiawei Date: Wed, 25 Jan 2023 11:01:56 +0100 Subject: [PATCH 58/76] Code comment --- .../commands/ConvertToQbeastCommand.scala | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala b/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala index b79279784..ce3e0a8ab 100644 --- a/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala +++ b/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala @@ -18,6 +18,21 @@ import org.apache.spark.sql.{AnalysisExceptionFactory, Row, SparkSession} import java.util.Locale +/** + * Command to convert a parquet or a delta table, partitioned or not, into a qbeast table. + * The command creates the an empty revision for the metadata, the qbeast options provided + * should be those with which the user want to index the table. When the source table is + * partitioned, the partitionColumns should be provided in the form of "columnName DataType", + * e.g. "col1 STRING". + * @param identifier STRING, table identifier consisting of "format.`tablePath`" + * e.g. parquet.`/tmp/test/` + * @param columnsToIndex Seq[STRING], the columns on which the index is built + * e.g. Seq("col1", "col2") + * @param cubeSize INT, the desired cube size for the index + * e.g. 5000 + * @param partitionColumns STRING, the columns with which the source table is partitioned + * "col1 STRING, col2 INT" + */ @Experimental case class ConvertToQbeastCommand( identifier: String, From 44245df92efd40d7d4ecef5139587c2103e290b7 Mon Sep 17 00:00:00 2001 From: Jiawei Date: Wed, 25 Jan 2023 11:02:54 +0100 Subject: [PATCH 59/76] Update documentation to include the staging revision and the ConvertToQbeatCommand --- docs/QbeastFormat.md | 30 ++++++++++++++++++++++++++++++ 1 file changed, 30 insertions(+) diff --git a/docs/QbeastFormat.md b/docs/QbeastFormat.md index d57f53652..c80d69cdf 100644 --- a/docs/QbeastFormat.md +++ b/docs/QbeastFormat.md @@ -153,6 +153,36 @@ In Revision, you can find different information about the tree status and config In this case, we index columns `user_id` and `product_id` (which are both `Integers`) with a linear transformation. This means that they will not suffer any transformation besides the normalization. +### Staging Revision and ConvertToQbeastConvertToQbeastCommand +The introduction of the staging revision allows users to read a delta or a hybrid `qbeast + delta` table using qbeast format. +The non-qbeast `AddFile`s are considered as part of this staging revision. + +Its RevisionID is fixed to `stagingID = 0`, and it has `EmptyTransformer`s and `EmptyTransformation`s. +This Revision is normally created during the first write or when overwriting a table using qbeast. +For a table that is entirely written in `delta` or `parquet`, partitioned or not, we can use the `ConvertToQbeastCommand` to create this revision: +```scala +import io.qbeast.spark.internal.commands.ConvertToQbeastCommand + +val path = "/pathToTable/" +val tableIdentifier = s"parquet.`$path`" +val columnsToIndex = Seq("col1", "col2", "col3") +val desiredCubeSize = 50000 + +ConvertToQbeastCommand(tableIdentifier, columnsToIndex, desiredCubeSize).run(spark) + +val qTable = spark.read.format("qbeast").load(path) +``` + +By doing so, we also enable appends using either delta or the qbeast format. + +`Compaction` is allowed to group small files in the staging revision: +```scala +import io.qbeast.spark.QbeastTable + +val table = QbeastTable.forPath(spark, "/pathToTable/") +table.compact(0) +``` + ### State changes in MetaData **Data de-normalization** is a crucial component behind our multi-dimensional index. Instead of storing an index in a separate tree-like data structure, we reorganize the data and their replications in an `OTree`, whose **hierarchical structure** is the actual index. From 5895f978296cf8a588e4b1f0127dd3729fee401f Mon Sep 17 00:00:00 2001 From: Jiawei Date: Wed, 25 Jan 2023 11:16:37 +0100 Subject: [PATCH 60/76] Optimize imports --- src/test/scala/io/qbeast/spark/index/NewRevisionTest.scala | 3 +-- .../scala/io/qbeast/spark/index/query/QueryExecutorTest.scala | 1 - 2 files changed, 1 insertion(+), 3 deletions(-) diff --git a/src/test/scala/io/qbeast/spark/index/NewRevisionTest.scala b/src/test/scala/io/qbeast/spark/index/NewRevisionTest.scala index 261055d8c..4eaa118bb 100644 --- a/src/test/scala/io/qbeast/spark/index/NewRevisionTest.scala +++ b/src/test/scala/io/qbeast/spark/index/NewRevisionTest.scala @@ -3,14 +3,13 @@ */ package io.qbeast.spark.index +import io.qbeast.TestClasses._ import io.qbeast.spark.{QbeastIntegrationTestSpec, delta} import org.apache.spark.sql.SparkSession import org.apache.spark.sql.delta.DeltaLog import org.scalatest.PrivateMethodTester import org.scalatest.flatspec.AnyFlatSpec import org.scalatest.matchers.should.Matchers -import io.qbeast.TestClasses._ -import io.qbeast.core.model.Revision.isStaging class NewRevisionTest extends AnyFlatSpec diff --git a/src/test/scala/io/qbeast/spark/index/query/QueryExecutorTest.scala b/src/test/scala/io/qbeast/spark/index/query/QueryExecutorTest.scala index ac6d11b7e..c60318d4e 100644 --- a/src/test/scala/io/qbeast/spark/index/query/QueryExecutorTest.scala +++ b/src/test/scala/io/qbeast/spark/index/query/QueryExecutorTest.scala @@ -1,7 +1,6 @@ package io.qbeast.spark.index.query import io.qbeast.TestClasses.T2 -import io.qbeast.core.model.Revision.isStaging import io.qbeast.core.model.{CubeId, QbeastBlock, Weight, WeightRange} import io.qbeast.spark.QbeastIntegrationTestSpec import io.qbeast.spark.delta.DeltaQbeastSnapshot From 2c6711d09413b163b2bd26fef9a5a3ffb95cc02b Mon Sep 17 00:00:00 2001 From: Jiawei Date: Wed, 25 Jan 2023 11:17:22 +0100 Subject: [PATCH 61/76] Check total num revisions after append on a converted table --- .../qbeast/spark/utils/ConvertToQbeastTest.scala | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala b/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala index 4ff8905d5..448199076 100644 --- a/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala +++ b/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala @@ -89,7 +89,7 @@ class ConvertToQbeastTest extends QbeastIntegrationTestSpec with PrivateMethodTe indexStatus.cubesStatuses.size shouldBe 1 }) - it should "add a staging revision for a qbeast table" in withSparkAndTmpDir((spark, tmpDir) => { + it should "not change a qbeast table" in withSparkAndTmpDir((spark, tmpDir) => { val data = loadTestData(spark).limit(dataSize) data.write .format("qbeast") @@ -97,13 +97,11 @@ class ConvertToQbeastTest extends QbeastIntegrationTestSpec with PrivateMethodTe .option("cubeSize", dcs) .save(tmpDir) - val qsBefore = getQbeastSnapshot(spark, tmpDir) - val revisionsBefore = qsBefore.loadAllRevisions + val revisionsBefore = getQbeastSnapshot(spark, tmpDir).loadAllRevisions ConvertToQbeastCommand(s"parquet.`$tmpDir`", columnsToIndex, dcs) - val qsAfter = getQbeastSnapshot(spark, tmpDir) - val revisionsAfter = qsAfter.loadAllRevisions + val revisionsAfter = getQbeastSnapshot(spark, tmpDir).loadAllRevisions spark.read.parquet(tmpDir).count shouldBe dataSize revisionsAfter shouldBe revisionsBefore @@ -138,8 +136,8 @@ class ConvertToQbeastTest extends QbeastIntegrationTestSpec with PrivateMethodTe }) }) - "Appending to a converted table" should "create a new non-staging revision" in withSparkAndTmpDir( - (spark, tmpDir) => { + "Appending to a converted table" should "create a new, non-staging revision" in + withSparkAndTmpDir((spark, tmpDir) => { convertFromFormat(spark, "parquet", tmpDir) val df = loadTestData(spark).limit(dataSize) df.write @@ -150,6 +148,8 @@ class ConvertToQbeastTest extends QbeastIntegrationTestSpec with PrivateMethodTe .save(tmpDir) val qs = getQbeastSnapshot(spark, tmpDir) + qs.loadAllRevisions.size shouldBe 2 + val rev = qs.loadLatestRevision isStaging(rev) shouldBe false }) From ab91ff07fce2c471db04adce18145686da5f1c6d Mon Sep 17 00:00:00 2001 From: Jiawei Date: Wed, 25 Jan 2023 15:07:49 +0100 Subject: [PATCH 62/76] Place staging revision utils under RevisionUtils --- .../qbeast/core/model/RevisionClasses.scala | 36 ++++++++++--------- .../spark/delta/DeltaQbeastSnapshot.scala | 2 +- .../spark/delta/IndexStatusBuilder.scala | 17 +++++---- .../spark/delta/QbeastMetadataOperation.scala | 2 +- .../qbeast/spark/delta/writer/Compactor.scala | 2 +- .../spark/index/query/QuerySpecBuilder.scala | 2 +- .../io/qbeast/spark/table/IndexedTable.scala | 2 +- 7 files changed, 33 insertions(+), 30 deletions(-) diff --git a/core/src/main/scala/io/qbeast/core/model/RevisionClasses.scala b/core/src/main/scala/io/qbeast/core/model/RevisionClasses.scala index 3ac5e470d..acc05431d 100644 --- a/core/src/main/scala/io/qbeast/core/model/RevisionClasses.scala +++ b/core/src/main/scala/io/qbeast/core/model/RevisionClasses.scala @@ -4,7 +4,7 @@ import com.fasterxml.jackson.annotation.{JsonCreator, JsonValue} import com.fasterxml.jackson.databind.annotation.JsonSerialize import com.fasterxml.jackson.databind.annotation.JsonSerialize.Typing import io.qbeast.IISeq -import io.qbeast.core.model.Revision.isStaging +import io.qbeast.core.model.RevisionUtils.{isStaging, stagingID} import io.qbeast.core.transform.{EmptyTransformer, Transformation, Transformer} import scala.collection.immutable.SortedMap @@ -42,17 +42,6 @@ final class QTableID(_id: String) extends Serializable { * Companion object for Revision */ object Revision { - val stagingID: RevisionID = 0 - - def isStaging(revisionID: RevisionID): Boolean = - revisionID == stagingID - - def isStaging(revision: Revision): Boolean = - isStaging(revision.revisionID) && - revision.columnTransformers.forall { - case _: EmptyTransformer => true - case _ => false - } /** * Create a new first revision for a table @@ -83,20 +72,35 @@ object Revision { tableID: QTableID, desiredCubeSize: Int, columnsToIndex: Seq[String]): Revision = { - val transformers = columnsToIndex.map(s => EmptyTransformer(s)).toIndexedSeq - val transformations = transformers.map(_.makeTransformation(r => r)) + val emptyTransformers = columnsToIndex.map(s => EmptyTransformer(s)).toIndexedSeq + val emptyTransformations = emptyTransformers.map(_.makeTransformation(r => r)) Revision( stagingID, System.currentTimeMillis(), tableID, desiredCubeSize, - transformers, - transformations) + emptyTransformers, + emptyTransformations) } } +object RevisionUtils { + val stagingID: RevisionID = 0 + + def isStaging(revisionID: RevisionID): Boolean = + revisionID == stagingID + + def isStaging(revision: Revision): Boolean = + isStaging(revision.revisionID) && + revision.columnTransformers.forall { + case _: EmptyTransformer => true + case _ => false + } + +} + /** * A revision of a QTable. * @param revisionID the identifier of the revision diff --git a/src/main/scala/io/qbeast/spark/delta/DeltaQbeastSnapshot.scala b/src/main/scala/io/qbeast/spark/delta/DeltaQbeastSnapshot.scala index 3987c6b11..3f5d6a067 100644 --- a/src/main/scala/io/qbeast/spark/delta/DeltaQbeastSnapshot.scala +++ b/src/main/scala/io/qbeast/spark/delta/DeltaQbeastSnapshot.scala @@ -4,7 +4,7 @@ package io.qbeast.spark.delta import io.qbeast.IISeq -import io.qbeast.core.model.Revision.{isStaging, stagingID} +import io.qbeast.core.model.RevisionUtils.{isStaging, stagingID} import io.qbeast.core.model._ import io.qbeast.spark.utils.Staging.isStagingFile import io.qbeast.spark.utils.{MetadataConfig, TagColumns} diff --git a/src/main/scala/io/qbeast/spark/delta/IndexStatusBuilder.scala b/src/main/scala/io/qbeast/spark/delta/IndexStatusBuilder.scala index 321dea183..524af78e0 100644 --- a/src/main/scala/io/qbeast/spark/delta/IndexStatusBuilder.scala +++ b/src/main/scala/io/qbeast/spark/delta/IndexStatusBuilder.scala @@ -3,7 +3,7 @@ */ package io.qbeast.spark.delta -import io.qbeast.core.model.Revision.isStaging +import io.qbeast.core.model.RevisionUtils.isStaging import io.qbeast.core.model._ import io.qbeast.spark.delta.QbeastMetadataSQL._ import io.qbeast.spark.utils.State.FLOODED @@ -73,14 +73,14 @@ private[delta] class IndexStatusBuilder( * @return Dataset containing cube information */ def buildCubesStatuses: SortedMap[CubeId, CubeStatus] = { - val spark = SparkSession.active - import spark.implicits._ + val spark = SparkSession.active val builder = SortedMap.newBuilder[CubeId, CubeStatus] - val dimCount: Int = revision.transformations.size - val dcs = revision.desiredCubeSize + val rev = revision + import spark.implicits._ + val ndims: Int = rev.transformations.size revisionFiles .groupBy(TagColumns.cube) .agg( @@ -88,15 +88,14 @@ private[delta] class IndexStatusBuilder( sum(TagColumns.elementCount).as("elementCount"), collect_list(qblock).as("files")) .select( - createCube(col("cube"), lit(dimCount)).as("cubeId"), + createCube(col("cube"), lit(ndims)).as("cubeId"), col("maxWeight"), - normalizeWeight(col("maxWeight"), col("elementCount"), lit(dcs)) - .as("normalizedWeight"), + normalizeWeight(col("maxWeight"), col("elementCount"), lit(rev.desiredCubeSize)).as( + "normalizedWeight"), col("files")) .as[CubeStatus] .collect() .foreach(row => builder += row.cubeId -> row) - builder.result() } diff --git a/src/main/scala/io/qbeast/spark/delta/QbeastMetadataOperation.scala b/src/main/scala/io/qbeast/spark/delta/QbeastMetadataOperation.scala index 1c7136472..4e8e07568 100644 --- a/src/main/scala/io/qbeast/spark/delta/QbeastMetadataOperation.scala +++ b/src/main/scala/io/qbeast/spark/delta/QbeastMetadataOperation.scala @@ -3,7 +3,7 @@ */ package io.qbeast.spark.delta -import io.qbeast.core.model.Revision.stagingID +import io.qbeast.core.model.RevisionUtils.stagingID import io.qbeast.core.model.{ReplicatedSet, Revision, TableChanges, mapper} import io.qbeast.spark.utils.MetadataConfig import io.qbeast.spark.utils.MetadataConfig.{lastRevisionID, revision} diff --git a/src/main/scala/io/qbeast/spark/delta/writer/Compactor.scala b/src/main/scala/io/qbeast/spark/delta/writer/Compactor.scala index 9ea07756f..194f4970c 100644 --- a/src/main/scala/io/qbeast/spark/delta/writer/Compactor.scala +++ b/src/main/scala/io/qbeast/spark/delta/writer/Compactor.scala @@ -4,7 +4,7 @@ package io.qbeast.spark.delta.writer import io.qbeast.IISeq -import io.qbeast.core.model.Revision.isStaging +import io.qbeast.core.model.RevisionUtils.isStaging import io.qbeast.core.model.{CubeId, QTableID, QbeastBlock, TableChanges, Weight} import io.qbeast.spark.utils.{State, TagUtils} import org.apache.hadoop.fs.Path diff --git a/src/main/scala/io/qbeast/spark/index/query/QuerySpecBuilder.scala b/src/main/scala/io/qbeast/spark/index/query/QuerySpecBuilder.scala index fbe09d1a1..8076eedb0 100644 --- a/src/main/scala/io/qbeast/spark/index/query/QuerySpecBuilder.scala +++ b/src/main/scala/io/qbeast/spark/index/query/QuerySpecBuilder.scala @@ -3,7 +3,7 @@ */ package io.qbeast.spark.index.query -import io.qbeast.core.model.Revision.isStaging +import io.qbeast.core.model.RevisionUtils.isStaging import io.qbeast.core.model._ import io.qbeast.spark.internal.expressions.QbeastMurmur3Hash import org.apache.spark.sql.SparkSession diff --git a/src/main/scala/io/qbeast/spark/table/IndexedTable.scala b/src/main/scala/io/qbeast/spark/table/IndexedTable.scala index 370f91652..34344a06a 100644 --- a/src/main/scala/io/qbeast/spark/table/IndexedTable.scala +++ b/src/main/scala/io/qbeast/spark/table/IndexedTable.scala @@ -4,7 +4,7 @@ package io.qbeast.spark.table import io.qbeast.core.keeper.Keeper -import io.qbeast.core.model.Revision.isStaging +import io.qbeast.core.model.RevisionUtils.isStaging import io.qbeast.core.model._ import io.qbeast.spark.delta.CubeDataLoader import io.qbeast.spark.index.QbeastColumns From d550497beb35853bf54c90363c407e67cf8b7e65 Mon Sep 17 00:00:00 2001 From: Jiawei Date: Wed, 25 Jan 2023 15:54:24 +0100 Subject: [PATCH 63/76] Test EmptyTransformer/ation --- .../transform/EmptyTransformationTest.scala | 35 +++++++++++++++++++ .../core/transform/TransformerTest.scala | 8 +++++ 2 files changed, 43 insertions(+) create mode 100644 core/src/test/scala/io/qbeast/core/transform/EmptyTransformationTest.scala diff --git a/core/src/test/scala/io/qbeast/core/transform/EmptyTransformationTest.scala b/core/src/test/scala/io/qbeast/core/transform/EmptyTransformationTest.scala new file mode 100644 index 000000000..fa172af3e --- /dev/null +++ b/core/src/test/scala/io/qbeast/core/transform/EmptyTransformationTest.scala @@ -0,0 +1,35 @@ +package io.qbeast.core.transform + +import io.qbeast.core.model.DoubleDataType +import org.scalatest.flatspec.AnyFlatSpec +import org.scalatest.matchers.should.Matchers + +class EmptyTransformationTest extends AnyFlatSpec with Matchers { + it should "always map to the same value" in { + val t = EmptyTransformation() + + (1 to 100).foreach { i => + t.transform(i) shouldBe 0d + } + + t.transform(null) shouldBe 0d + } + + it should "be superseded by another Transformation" in { + val et = EmptyTransformation() + val ht = HashTransformation() + val lt = LinearTransformation(1d, 1.1, DoubleDataType) + + et.isSupersededBy(ht) shouldBe true + et.isSupersededBy(lt) shouldBe true + } + + it should "return the other Transformation when merging" in { + val et = EmptyTransformation() + val ht = HashTransformation() + val lt = LinearTransformation(1d, 1.1, DoubleDataType) + + et.merge(ht) shouldBe ht + et.merge(lt) shouldBe lt + } +} diff --git a/core/src/test/scala/io/qbeast/core/transform/TransformerTest.scala b/core/src/test/scala/io/qbeast/core/transform/TransformerTest.scala index fcfbf17fc..5a753b6d8 100644 --- a/core/src/test/scala/io/qbeast/core/transform/TransformerTest.scala +++ b/core/src/test/scala/io/qbeast/core/transform/TransformerTest.scala @@ -94,4 +94,12 @@ class TransformerTest extends AnyFlatSpec with Matchers { transformer.maybeUpdateTransformation(currentTransformation, transformation) shouldBe None } + + "An EmptyTransformer" should "create an EmptyTransformation without stats" in { + val colName = "a" + val transformer = EmptyTransformer(colName) + val transformation = transformer.makeTransformation(r => r) + + transformation shouldBe a[EmptyTransformation] + } } From 6bc7ceba1a1e1793677273a2cedae55d5e590009 Mon Sep 17 00:00:00 2001 From: Jiawei Date: Wed, 25 Jan 2023 16:11:07 +0100 Subject: [PATCH 64/76] Add comment and remove unnecessary changes --- src/main/scala/io/qbeast/spark/delta/OTreeIndex.scala | 8 +------- .../io/qbeast/spark/delta/QbeastMetadataOperation.scala | 6 +++++- 2 files changed, 6 insertions(+), 8 deletions(-) diff --git a/src/main/scala/io/qbeast/spark/delta/OTreeIndex.scala b/src/main/scala/io/qbeast/spark/delta/OTreeIndex.scala index efc63b800..048b6745d 100644 --- a/src/main/scala/io/qbeast/spark/delta/OTreeIndex.scala +++ b/src/main/scala/io/qbeast/spark/delta/OTreeIndex.scala @@ -104,13 +104,7 @@ object OTreeIndex { def apply(spark: SparkSession, path: Path): OTreeIndex = { val deltaLog = DeltaLog.forTable(spark, path) - val tahoe = TahoeLogFileIndex( - spark, - deltaLog, - path, - deltaLog.snapshot, - Seq.empty, - isTimeTravelQuery = false) + val tahoe = TahoeLogFileIndex(spark, deltaLog, path, deltaLog.snapshot, Seq.empty, false) OTreeIndex(tahoe) } diff --git a/src/main/scala/io/qbeast/spark/delta/QbeastMetadataOperation.scala b/src/main/scala/io/qbeast/spark/delta/QbeastMetadataOperation.scala index 4e8e07568..1ed336dcc 100644 --- a/src/main/scala/io/qbeast/spark/delta/QbeastMetadataOperation.scala +++ b/src/main/scala/io/qbeast/spark/delta/QbeastMetadataOperation.scala @@ -82,18 +82,22 @@ private[delta] class QbeastMetadataOperation extends ImplicitMetadataOperation { newRevision: Revision): Configuration = { val newRevisionID = newRevision.revisionID - // Add staging revision if necessary + // Add staging revision, if necessary. The qbeast metadata configuration + // should always have a revision with RevisionID = stagingID. val stagingRevisionKey = s"$revision.$stagingID" val addStagingRevision = newRevisionID == 1 && !baseConfiguration.contains(stagingRevisionKey) val configuration = if (!addStagingRevision) baseConfiguration else { + // Create staging revision with EmptyTransformers(and EmptyTransformations) val stagingRevision = Revision.emptyRevision( newRevision.tableID, newRevision.desiredCubeSize, newRevision.columnTransformers.map(_.columnName)) + // Add the staging revision to the revisionMap without overwriting + // the latestRevisionID baseConfiguration .updated(stagingRevisionKey, mapper.writeValueAsString(stagingRevision)) } From a7cbe6ef9f600ccf4e2bac653d2a2a685c04a0a9 Mon Sep 17 00:00:00 2001 From: Jiawei Date: Wed, 25 Jan 2023 16:29:51 +0100 Subject: [PATCH 65/76] Move params and methods to RevisionUtils --- src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala | 2 +- .../scala/io/qbeast/spark/utils/QbeastDeltaStagingTest.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala b/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala index 448199076..80979c11d 100644 --- a/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala +++ b/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala @@ -1,6 +1,6 @@ package io.qbeast.spark.utils -import io.qbeast.core.model.Revision.{isStaging, stagingID} +import io.qbeast.core.model.RevisionUtils.{isStaging, stagingID} import io.qbeast.spark.delta.DeltaQbeastSnapshot import io.qbeast.spark.internal.commands.ConvertToQbeastCommand import io.qbeast.spark.{QbeastIntegrationTestSpec, QbeastTable} diff --git a/src/test/scala/io/qbeast/spark/utils/QbeastDeltaStagingTest.scala b/src/test/scala/io/qbeast/spark/utils/QbeastDeltaStagingTest.scala index 4a57ff08f..ece323088 100644 --- a/src/test/scala/io/qbeast/spark/utils/QbeastDeltaStagingTest.scala +++ b/src/test/scala/io/qbeast/spark/utils/QbeastDeltaStagingTest.scala @@ -1,7 +1,7 @@ package io.qbeast.spark.utils import io.qbeast.TestClasses.T2 -import io.qbeast.core.model.Revision.stagingID +import io.qbeast.core.model.RevisionUtils.stagingID import io.qbeast.spark.delta.DeltaQbeastSnapshot import io.qbeast.spark.{QbeastIntegrationTestSpec, QbeastTable} import org.apache.spark.sql.SparkSession From b01e8065693f6e0d07c43bca2176728ebb8b0954 Mon Sep 17 00:00:00 2001 From: Jiawei Date: Wed, 25 Jan 2023 16:35:14 +0100 Subject: [PATCH 66/76] isStagingFile is specific of QbeastSnapshot --- .../scala/io/qbeast/spark/delta/DeltaQbeastSnapshot.scala | 3 ++- src/main/scala/io/qbeast/spark/delta/OTreeIndex.scala | 4 ++-- src/main/scala/io/qbeast/spark/utils/Params.scala | 4 ---- 3 files changed, 4 insertions(+), 7 deletions(-) diff --git a/src/main/scala/io/qbeast/spark/delta/DeltaQbeastSnapshot.scala b/src/main/scala/io/qbeast/spark/delta/DeltaQbeastSnapshot.scala index 3f5d6a067..77e124eff 100644 --- a/src/main/scala/io/qbeast/spark/delta/DeltaQbeastSnapshot.scala +++ b/src/main/scala/io/qbeast/spark/delta/DeltaQbeastSnapshot.scala @@ -6,7 +6,6 @@ package io.qbeast.spark.delta import io.qbeast.IISeq import io.qbeast.core.model.RevisionUtils.{isStaging, stagingID} import io.qbeast.core.model._ -import io.qbeast.spark.utils.Staging.isStagingFile import io.qbeast.spark.utils.{MetadataConfig, TagColumns} import org.apache.spark.sql.delta.Snapshot import org.apache.spark.sql.delta.actions.AddFile @@ -22,6 +21,8 @@ case class DeltaQbeastSnapshot(private val snapshot: Snapshot) extends QbeastSna def isInitial: Boolean = snapshot.version == -1 + private val isStagingFile = "tags IS NULL" + private val metadataMap: Map[String, String] = snapshot.metadata.configuration /** diff --git a/src/main/scala/io/qbeast/spark/delta/OTreeIndex.scala b/src/main/scala/io/qbeast/spark/delta/OTreeIndex.scala index 048b6745d..9a83a7775 100644 --- a/src/main/scala/io/qbeast/spark/delta/OTreeIndex.scala +++ b/src/main/scala/io/qbeast/spark/delta/OTreeIndex.scala @@ -4,8 +4,8 @@ package io.qbeast.spark.delta import io.qbeast.core.model.QbeastBlock +import io.qbeast.core.model.RevisionUtils.stagingID import io.qbeast.spark.index.query.{QueryExecutor, QuerySpecBuilder} -import io.qbeast.spark.utils.Staging.isStagingFile import org.apache.hadoop.fs.{FileStatus, Path} import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.expressions.{Expression, GenericInternalRow} @@ -56,7 +56,7 @@ case class OTreeIndex(index: TahoeLogFileIndex) extends FileIndex { * @return */ private def stagingFiles: Seq[FileStatus] = { - snapshot.allFiles.where(isStagingFile).collect().map { a: AddFile => + qbeastSnapshot.loadRevisionBlocks(stagingID).collect().map { a: AddFile => new FileStatus( /* length */ a.size, /* isDir */ false, diff --git a/src/main/scala/io/qbeast/spark/utils/Params.scala b/src/main/scala/io/qbeast/spark/utils/Params.scala index d3b1b0f91..c8feb4949 100644 --- a/src/main/scala/io/qbeast/spark/utils/Params.scala +++ b/src/main/scala/io/qbeast/spark/utils/Params.scala @@ -40,7 +40,3 @@ object MetadataConfig { final val replicatedSet = "qbeast.replicatedSet" final val lastRevisionID = "qbeast.lastRevisionID" } - -object Staging { - final val isStagingFile = "tags IS NULL" -} From ba00ec44cc07871c61d471f39f2539c9cecb945c Mon Sep 17 00:00:00 2001 From: Jiawei Date: Wed, 25 Jan 2023 17:02:39 +0100 Subject: [PATCH 67/76] Correct typo --- docs/QbeastFormat.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/QbeastFormat.md b/docs/QbeastFormat.md index c80d69cdf..b2aa00d3f 100644 --- a/docs/QbeastFormat.md +++ b/docs/QbeastFormat.md @@ -153,7 +153,7 @@ In Revision, you can find different information about the tree status and config In this case, we index columns `user_id` and `product_id` (which are both `Integers`) with a linear transformation. This means that they will not suffer any transformation besides the normalization. -### Staging Revision and ConvertToQbeastConvertToQbeastCommand +### Staging Revision and ConvertToQbeastCommand The introduction of the staging revision allows users to read a delta or a hybrid `qbeast + delta` table using qbeast format. The non-qbeast `AddFile`s are considered as part of this staging revision. From 8969983f1079e72ebe4583b195403e871ab0fe80 Mon Sep 17 00:00:00 2001 From: Jiawei Date: Wed, 25 Jan 2023 17:32:13 +0100 Subject: [PATCH 68/76] Fix loadRevisionAt --- .../io/qbeast/spark/delta/DeltaQbeastSnapshot.scala | 9 ++++++--- .../spark/delta/QbeastMetadataOperation.scala | 13 ++++++++----- 2 files changed, 14 insertions(+), 8 deletions(-) diff --git a/src/main/scala/io/qbeast/spark/delta/DeltaQbeastSnapshot.scala b/src/main/scala/io/qbeast/spark/delta/DeltaQbeastSnapshot.scala index 77e124eff..cad5a847f 100644 --- a/src/main/scala/io/qbeast/spark/delta/DeltaQbeastSnapshot.scala +++ b/src/main/scala/io/qbeast/spark/delta/DeltaQbeastSnapshot.scala @@ -4,7 +4,7 @@ package io.qbeast.spark.delta import io.qbeast.IISeq -import io.qbeast.core.model.RevisionUtils.{isStaging, stagingID} +import io.qbeast.core.model.RevisionUtils.isStaging import io.qbeast.core.model._ import io.qbeast.spark.utils.{MetadataConfig, TagColumns} import org.apache.spark.sql.delta.Snapshot @@ -154,8 +154,11 @@ case class DeltaQbeastSnapshot(private val snapshot: Snapshot) extends QbeastSna * @return the latest Revision at a concrete timestamp */ override def loadRevisionAt(timestamp: Long): Revision = { - (revisionsMap - stagingID).values.find(_.timestamp <= timestamp).getOrElse { - throw AnalysisExceptionFactory.create(s"No space revision available before $timestamp") + val candidateRevisions = revisionsMap.values.filter(_.timestamp <= timestamp) + if (candidateRevisions.nonEmpty) candidateRevisions.maxBy(_.timestamp) + else { + throw AnalysisExceptionFactory + .create(s"No space revision available before $timestamp") } } diff --git a/src/main/scala/io/qbeast/spark/delta/QbeastMetadataOperation.scala b/src/main/scala/io/qbeast/spark/delta/QbeastMetadataOperation.scala index 1ed336dcc..6a17d4e78 100644 --- a/src/main/scala/io/qbeast/spark/delta/QbeastMetadataOperation.scala +++ b/src/main/scala/io/qbeast/spark/delta/QbeastMetadataOperation.scala @@ -90,11 +90,14 @@ private[delta] class QbeastMetadataOperation extends ImplicitMetadataOperation { val configuration = if (!addStagingRevision) baseConfiguration else { - // Create staging revision with EmptyTransformers(and EmptyTransformations) - val stagingRevision = Revision.emptyRevision( - newRevision.tableID, - newRevision.desiredCubeSize, - newRevision.columnTransformers.map(_.columnName)) + // Create staging revision with EmptyTransformers (and EmptyTransformations). + // We modify its timestamp to secure loadRevisionAt + val stagingRevision = Revision + .emptyRevision( + newRevision.tableID, + newRevision.desiredCubeSize, + newRevision.columnTransformers.map(_.columnName)) + .copy(timestamp = newRevision.timestamp - 1) // Add the staging revision to the revisionMap without overwriting // the latestRevisionID From d61538f784df19237fa1ffa5d2a0bc0d2cd4efee Mon Sep 17 00:00:00 2001 From: Jiawei Date: Thu, 26 Jan 2023 16:06:30 +0100 Subject: [PATCH 69/76] 'Remove' partitioned table conversion support --- .../commands/ConvertToQbeastCommand.scala | 72 ++++--- .../spark/utils/ConvertToQbeastTest.scala | 200 ++++++++++-------- .../spark/utils/QbeastDeltaStagingTest.scala | 40 +++- 3 files changed, 183 insertions(+), 129 deletions(-) diff --git a/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala b/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala index ce3e0a8ab..3928a6fe5 100644 --- a/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala +++ b/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala @@ -5,6 +5,10 @@ package io.qbeast.spark.internal.commands import io.qbeast.core.model._ import io.qbeast.spark.delta.DeltaQbeastSnapshot +import io.qbeast.spark.internal.commands.ConvertToQbeastCommand.{ + partitionedTableExceptionMsg, + unsupportedFormatExceptionMsg +} import io.qbeast.spark.utils.MetadataConfig.{lastRevisionID, revision} import org.apache.http.annotation.Experimental import org.apache.spark.internal.Logging @@ -12,38 +16,30 @@ import org.apache.spark.qbeast.config.DEFAULT_CUBE_SIZE import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.delta.DeltaLog import org.apache.spark.sql.delta.DeltaOperations.Convert -import org.apache.spark.sql.delta.metering.DeltaLogging import org.apache.spark.sql.execution.command.LeafRunnableCommand -import org.apache.spark.sql.{AnalysisExceptionFactory, Row, SparkSession} +import org.apache.spark.sql.{AnalysisException, AnalysisExceptionFactory, Row, SparkSession} import java.util.Locale /** - * Command to convert a parquet or a delta table, partitioned or not, into a qbeast table. + * Command to convert a parquet or a delta table into a qbeast table. * The command creates the an empty revision for the metadata, the qbeast options provided - * should be those with which the user want to index the table. When the source table is - * partitioned, the partitionColumns should be provided in the form of "columnName DataType", - * e.g. "col1 STRING". + * should be those with which the user want to index the table. Partitioned tables are not + * supported. * @param identifier STRING, table identifier consisting of "format.`tablePath`" * e.g. parquet.`/tmp/test/` * @param columnsToIndex Seq[STRING], the columns on which the index is built * e.g. Seq("col1", "col2") * @param cubeSize INT, the desired cube size for the index * e.g. 5000 - * @param partitionColumns STRING, the columns with which the source table is partitioned - * "col1 STRING, col2 INT" */ @Experimental case class ConvertToQbeastCommand( identifier: String, columnsToIndex: Seq[String], - cubeSize: Int = DEFAULT_CUBE_SIZE, - partitionColumns: Option[String] = None) + cubeSize: Int = DEFAULT_CUBE_SIZE) extends LeafRunnableCommand - with Logging - with DeltaLogging { - - private val isPartitioned: Boolean = partitionColumns.isDefined + with Logging { private def resolveTableFormat(spark: SparkSession): (String, TableIdentifier) = identifier.split("\\.") match { @@ -52,17 +48,6 @@ case class ConvertToQbeastCommand( case _ => throw new RuntimeException(s"Table doesn't exists at $identifier") } - /** - * Convert the parquet table using ConvertToDeltaCommand from Delta Lake. - */ - private def convertParquetToDelta(spark: SparkSession, path: String): Unit = { - val conversionCommand = - if (!isPartitioned) s"CONVERT TO DELTA parquet.$path" - else s"CONVERT TO DELTA parquet.$path PARTITIONED BY (${partitionColumns.get})" - - spark.sql(conversionCommand) - } - override def run(spark: SparkSession): Seq[Row] = { val (fileFormat, tableId) = resolveTableFormat(spark) @@ -75,9 +60,18 @@ case class ConvertToQbeastCommand( } else { fileFormat match { // Convert parquet to delta - case "parquet" => convertParquetToDelta(spark, tableId.quotedString) + case "parquet" => + try { + spark.sql(s"CONVERT TO DELTA parquet.${tableId.quotedString}") + } catch { + case e: AnalysisException => + val deltaMsg = e.getMessage() + throw AnalysisExceptionFactory.create( + partitionedTableExceptionMsg + + s"Failed to convert the parquet table into delta: $deltaMsg") + } case "delta" => - case _ => throw AnalysisExceptionFactory.create(s"Unsupported file format: $fileFormat") + case _ => throw AnalysisExceptionFactory.create(unsupportedFormatExceptionMsg(fileFormat)) } // Convert delta to qbeast @@ -85,13 +79,15 @@ case class ConvertToQbeastCommand( val txn = deltaLog.startTransaction() + // Converting a partitioned delta table is not supported, for qbeast files + // are not partitioned. + val isPartitionedDelta = txn.metadata.partitionColumns.nonEmpty + if (isPartitionedDelta) { + throw AnalysisExceptionFactory.create(partitionedTableExceptionMsg) + } val convRevision = Revision.emptyRevision(QTableID(tableId.table), cubeSize, columnsToIndex) val revisionID = convRevision.revisionID - // If the table has partition columns, its conversion to qbeast will - // remove them by overwriting the schema - val isOverwritingSchema = txn.metadata.partitionColumns.nonEmpty - // Update revision map val updatedConf = txn.metadata.configuration @@ -99,10 +95,9 @@ case class ConvertToQbeastCommand( .updated(s"$revision.$revisionID", mapper.writeValueAsString(convRevision)) val newMetadata = - txn.metadata.copy(configuration = updatedConf, partitionColumns = Seq.empty) + txn.metadata.copy(configuration = updatedConf) // , partitionColumns = Seq.empty txn.updateMetadata(newMetadata) - if (isOverwritingSchema) recordDeltaEvent(txn.deltaLog, "delta.ddl.overwriteSchema") txn.commit(Seq.empty, Convert(0, Seq.empty, collectStats = false, None)) } Seq.empty[Row] @@ -110,4 +105,13 @@ case class ConvertToQbeastCommand( } -case class ColumnMinMax(minValue: Any, maxValue: Any) +object ConvertToQbeastCommand { + + val partitionedTableExceptionMsg: String = + """Converting a partitioned table into qbeast is not supported. + |Consider overwriting the entire data using qbeast.""".stripMargin.replaceAll("\n", " ") + + def unsupportedFormatExceptionMsg: String => String = (fileFormat: String) => + s"Unsupported file format: $fileFormat" + +} diff --git a/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala b/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala index 80979c11d..3cdb4f58f 100644 --- a/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala +++ b/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala @@ -3,37 +3,46 @@ package io.qbeast.spark.utils import io.qbeast.core.model.RevisionUtils.{isStaging, stagingID} import io.qbeast.spark.delta.DeltaQbeastSnapshot import io.qbeast.spark.internal.commands.ConvertToQbeastCommand +import io.qbeast.spark.internal.commands.ConvertToQbeastCommand.{ + partitionedTableExceptionMsg, + unsupportedFormatExceptionMsg +} import io.qbeast.spark.{QbeastIntegrationTestSpec, QbeastTable} import org.apache.spark.sql.delta.DeltaLog -import org.apache.spark.sql.{AnalysisException, DataFrame, SparkSession} +import org.apache.spark.sql.{AnalysisException, SparkSession} import org.scalatest.PrivateMethodTester class ConvertToQbeastTest extends QbeastIntegrationTestSpec with PrivateMethodTester { - val columnsToIndex: Seq[String] = Seq("user_id", "price", "event_type") - val partitionColumns: Option[String] = Some("event_type STRING") - val numSparkPartitions = 20 val dataSize = 50000 + val numSparkPartitions = 20 + + val columnsToIndex: Seq[String] = Seq("user_id", "price", "event_type") val dcs = 5000 + val partitionedParquetExceptionMsg: String = + partitionedTableExceptionMsg + "Failed to convert the parquet table into delta: " + def convertFromFormat( spark: SparkSession, format: String, tablePath: String, - partitionColumns: Option[String] = None, - columnsToIndex: Seq[String] = columnsToIndex, - desiredCubeSize: Int = dcs): DataFrame = { + isPartitioned: Boolean = false): Unit = { val data = loadTestData(spark).limit(dataSize).repartition(numSparkPartitions) - if (partitionColumns.isDefined) { - val cols = partitionColumns.get.split(", ").map(_.split(" ").head) - data.write.mode("overwrite").partitionBy(cols: _*).format(format).save(tablePath) - } else data.write.mode("overwrite").format(format).save(tablePath) + // Write source data + if (isPartitioned) { + data.write + .mode("overwrite") + .partitionBy("event_type") + .format(format) + .save(tablePath) + } else { + data.write.mode("overwrite").format(format).save(tablePath) + } + // Convert source data to qbeast val tableIdentifier = s"$format.`$tablePath`" - ConvertToQbeastCommand(tableIdentifier, columnsToIndex, desiredCubeSize, partitionColumns) - .run(spark) - - spark.read.format("qbeast").load(tablePath) + ConvertToQbeastCommand(tableIdentifier, columnsToIndex, dcs).run(spark) } def getQbeastSnapshot(spark: SparkSession, dir: String): DeltaQbeastSnapshot = { @@ -41,90 +50,95 @@ class ConvertToQbeastTest extends QbeastIntegrationTestSpec with PrivateMethodTe DeltaQbeastSnapshot(deltaLog.snapshot) } - "ConvertToQbeastCommand" should "convert a delta table" in - withSparkAndTmpDir((spark, tmpDir) => { - val convertedTable = convertFromFormat(spark, "delta", tmpDir) + behavior of "ConvertToQbeastCommand" - convertedTable.count shouldBe dataSize - spark.read.parquet(tmpDir).count shouldBe dataSize + it should "convert a delta table" in withSparkAndTmpDir((spark, tmpDir) => { + val fileFormat = "delta" + convertFromFormat(spark, fileFormat, tmpDir) - val indexStatus = getQbeastSnapshot(spark, tmpDir).loadIndexStatus(stagingID) - // All non-qbeast files are considered staging files and are placed - // directly into the staging revision(RevisionID = 0) - indexStatus.cubesStatuses.size shouldBe 1 - indexStatus.cubesStatuses.head._2.files.size shouldBe numSparkPartitions - }) + val sourceDf = spark.read.format(fileFormat).load(tmpDir) + val qbeastDf = spark.read.format("qbeast").load(tmpDir) - it should "convert a PARTITIONED delta table" in - withSparkAndTmpDir((spark, tmpDir) => { - val convertedTable = convertFromFormat(spark, "delta", tmpDir, partitionColumns) + assertLargeDatasetEquality(qbeastDf, sourceDf) - convertedTable.count shouldBe dataSize - spark.read.parquet(tmpDir).count shouldBe dataSize + // All non-qbeast files are considered staging files and are placed + // directly into the staging revision(RevisionID = 0) + val indexStatus = getQbeastSnapshot(spark, tmpDir).loadIndexStatus(stagingID) + indexStatus.cubesStatuses.size shouldBe 1 + indexStatus.cubesStatuses.head._2.files.size shouldBe numSparkPartitions + }) - val indexStatus = getQbeastSnapshot(spark, tmpDir).loadIndexStatus(stagingID) - indexStatus.cubesStatuses.size shouldBe 1 - }) + it should "convert a parquet table" in withSparkAndTmpDir((spark, tmpDir) => { + val fileFormat = "parquet" + convertFromFormat(spark, fileFormat, tmpDir) - it should "convert a parquet table" in - withSparkAndTmpDir((spark, tmpDir) => { - val convertedTable = convertFromFormat(spark, "parquet", tmpDir) + val sourceDf = spark.read.format(fileFormat).load(tmpDir) + val qbeastDf = spark.read.format("qbeast").load(tmpDir) - convertedTable.count shouldBe dataSize - spark.read.parquet(tmpDir).count shouldBe dataSize + assertLargeDatasetEquality(qbeastDf, sourceDf) - val indexStatus = getQbeastSnapshot(spark, tmpDir).loadIndexStatus(stagingID) - indexStatus.cubesStatuses.size shouldBe 1 - indexStatus.cubesStatuses.head._2.files.size shouldBe numSparkPartitions - }) + // All non-qbeast files are considered staging files and are placed + // directly into the staging revision(RevisionID = 0) + val indexStatus = getQbeastSnapshot(spark, tmpDir).loadIndexStatus(stagingID) + indexStatus.cubesStatuses.size shouldBe 1 + indexStatus.cubesStatuses.head._2.files.size shouldBe numSparkPartitions + }) - it should "convert a PARTITIONED parquet table" in - withSparkAndTmpDir((spark, tmpDir) => { - val convertedTable = convertFromFormat(spark, "parquet", tmpDir, partitionColumns) + it should "fail to convert a PARTITIONED delta table" in withSparkAndTmpDir((spark, tmpDir) => { + val fileFormat = "delta" - convertedTable.count shouldBe dataSize - spark.read.parquet(tmpDir).count shouldBe dataSize + val thrown = + the[AnalysisException] thrownBy + convertFromFormat(spark, fileFormat, tmpDir, isPartitioned = true) - val indexStatus = getQbeastSnapshot(spark, tmpDir).loadIndexStatus(stagingID) - indexStatus.cubesStatuses.size shouldBe 1 - }) + thrown.getMessage() should startWith(partitionedTableExceptionMsg) + }) - it should "not change a qbeast table" in withSparkAndTmpDir((spark, tmpDir) => { - val data = loadTestData(spark).limit(dataSize) - data.write - .format("qbeast") - .option("columnsToIndex", columnsToIndex.mkString(",")) - .option("cubeSize", dcs) - .save(tmpDir) + it should "fail to convert a PARTITIONED parquet table" in withSparkAndTmpDir( + (spark, tmpDir) => { + val fileFormat = "parquet" - val revisionsBefore = getQbeastSnapshot(spark, tmpDir).loadAllRevisions + val thrown = + the[AnalysisException] thrownBy + convertFromFormat(spark, fileFormat, tmpDir, isPartitioned = true) - ConvertToQbeastCommand(s"parquet.`$tmpDir`", columnsToIndex, dcs) + thrown.getMessage() should startWith(partitionedParquetExceptionMsg) + }) + + it should "fail to convert an unsupported format" in withSparkAndTmpDir((spark, tmpDir) => { + val fileFormat = "json" - val revisionsAfter = getQbeastSnapshot(spark, tmpDir).loadAllRevisions + val thrown = + the[AnalysisException] thrownBy convertFromFormat(spark, fileFormat, tmpDir) - spark.read.parquet(tmpDir).count shouldBe dataSize - revisionsAfter shouldBe revisionsBefore + // json not supported + thrown.getMessage() should startWith(unsupportedFormatExceptionMsg("json")) }) - it should "throw an exception when attempting to convert an unsupported format" in - withSparkAndTmpDir((spark, tmpDir) => { - val df = loadTestData(spark) - df.write.mode("overwrite").json(tmpDir) + it should "not create new revisions for a qbeast table" in withSparkAndTmpDir( + (spark, tmpDir) => { + loadTestData(spark) + .limit(dataSize) + .write + .format("qbeast") + .option("columnsToIndex", columnsToIndex.mkString(",")) + .option("cubeSize", dcs) + .save(tmpDir) + + val revisionsBefore = getQbeastSnapshot(spark, tmpDir).loadAllRevisions + ConvertToQbeastCommand(s"qbeast.`$tmpDir`", columnsToIndex, dcs) + val revisionsAfter = getQbeastSnapshot(spark, tmpDir).loadAllRevisions - an[AnalysisException] shouldBe thrownBy( - ConvertToQbeastCommand(s"json.`$tmpDir`", columnsToIndex).run(spark)) + // Revisions should not modify + revisionsAfter shouldBe revisionsBefore }) it should "preserve sampling accuracy" in withSparkAndTmpDir((spark, tmpDir) => { convertFromFormat(spark, "parquet", tmpDir) - val qbeastTable = QbeastTable.forPath(spark, tmpDir) - - qbeastTable.analyze() - qbeastTable.optimize() val convertedTable = spark.read.format("qbeast").load(tmpDir) val tolerance = 0.01 + List(0.1, 0.2, 0.5, 0.7, 0.99).foreach(f => { val sampleSize = convertedTable .sample(withReplacement = false, f) @@ -139,40 +153,51 @@ class ConvertToQbeastTest extends QbeastIntegrationTestSpec with PrivateMethodTe "Appending to a converted table" should "create a new, non-staging revision" in withSparkAndTmpDir((spark, tmpDir) => { convertFromFormat(spark, "parquet", tmpDir) - val df = loadTestData(spark).limit(dataSize) - df.write + + // Append qbeast data + loadTestData(spark) + .limit(dataSize) + .write .mode("append") .format("qbeast") - .option("columnsToIndex", columnsToIndex.mkString(",")) - .option("cubeSize", dcs) .save(tmpDir) + // Should add new revision val qs = getQbeastSnapshot(spark, tmpDir) - qs.loadAllRevisions.size shouldBe 2 - + val allRevisions = qs.loadAllRevisions val rev = qs.loadLatestRevision + + allRevisions.size shouldBe 2 isStaging(rev) shouldBe false }) "Analyzing the staging revision" should "not change the ANNOUNCED set" in withSparkAndTmpDir((spark, tmpDir) => { convertFromFormat(spark, "parquet", tmpDir) + + // Analyze the staging revision val qbeastTable = QbeastTable.forPath(spark, tmpDir) qbeastTable.analyze() + // Preserve empty ANNOUNCED set val qs = getQbeastSnapshot(spark, tmpDir) qs.loadLatestIndexStatus.announcedSet.isEmpty shouldBe true }) "Optimizing the staging revision" should "not replicate any data" in withSparkAndTmpDir((spark, tmpDir) => { - convertFromFormat(spark, "parquet", tmpDir) - val qbeastTable = QbeastTable.forPath(spark, tmpDir) + val fileFormat = "parquet" + convertFromFormat(spark, fileFormat, tmpDir) + // Analyze and optimize + val qbeastTable = QbeastTable.forPath(spark, tmpDir) qbeastTable.analyze() qbeastTable.optimize() - spark.read.parquet(tmpDir).count shouldBe dataSize + // Compare DataFrames + val sourceDf = spark.read.format(fileFormat).load(tmpDir) + val qbeastDf = spark.read.format("qbeast").load(tmpDir) + assertLargeDatasetEquality(qbeastDf, sourceDf) }) "Compacting the staging revision" should "reduce the number of delta AddFiles" in @@ -181,12 +206,19 @@ class ConvertToQbeastTest extends QbeastIntegrationTestSpec with PrivateMethodTe .set("spark.qbeast.compact.minFileSize", "1") .set("spark.qbeast.compact.maxFileSize", "2000000")) { (spark, tmpDir) => { - convertFromFormat(spark, "parquet", tmpDir) + val fileFormat = "delta" + convertFromFormat(spark, fileFormat, tmpDir) + + // Perform compaction val qbeastTable = QbeastTable.forPath(spark, tmpDir) qbeastTable.compact() - spark.read.parquet(tmpDir).count shouldBe >(dataSize.toLong) + // Compare DataFrames + val sourceDf = spark.read.format(fileFormat).load(tmpDir) + val qbeastDf = spark.read.format("qbeast").load(tmpDir) + assertLargeDatasetEquality(qbeastDf, sourceDf) + // Standard staging revision behavior val qs = getQbeastSnapshot(spark, tmpDir) val stagingCs = qs.loadLatestIndexStatus.cubesStatuses diff --git a/src/test/scala/io/qbeast/spark/utils/QbeastDeltaStagingTest.scala b/src/test/scala/io/qbeast/spark/utils/QbeastDeltaStagingTest.scala index ece323088..8f268b776 100644 --- a/src/test/scala/io/qbeast/spark/utils/QbeastDeltaStagingTest.scala +++ b/src/test/scala/io/qbeast/spark/utils/QbeastDeltaStagingTest.scala @@ -16,6 +16,8 @@ class QbeastDeltaStagingTest extends QbeastIntegrationTestSpec { def writeHybridTable(spark: SparkSession, dir: String): Unit = { import spark.implicits._ + + // Write qbeast data val qdf = (0 until qDataSize).map(i => T2(i, i)).toDF("a", "b") qdf.write .format("qbeast") @@ -23,38 +25,51 @@ class QbeastDeltaStagingTest extends QbeastIntegrationTestSpec { .option("cubeSize", "5000") .save(dir) + // Create hybrid table by appending delta data val ddf = (qDataSize until qDataSize + dDataSize).map(i => T2(i, i)).toDF("a", "b") - ddf.repartition(numSparkPartitions).write.mode("append").format("delta").save(dir) + ddf + .repartition(numSparkPartitions) + .write + .mode("append") + .format("delta") + .save(dir) } "A qbeast + delta hybrid table" should "be read correctly" in withSparkAndTmpDir( (spark, tmpDir) => { writeHybridTable(spark, tmpDir) - val numStagingRows = spark.read.format("qbeast").load(tmpDir).count - numStagingRows shouldBe totalSize + val qbeastDf = spark.read.format("qbeast").load(tmpDir) + val deltaDf = spark.read.format("delta").load(tmpDir) + assertLargeDatasetEquality(qbeastDf, deltaDf) + // Should have the staging revision and the first revision val snapshot = DeltaLog.forTable(spark, tmpDir).snapshot val qs = DeltaQbeastSnapshot(snapshot) qs.loadAllRevisions.size shouldBe 2 qs.existsRevision(stagingID) }) - it should "not be altered by Optimizing the staging revision" in withSparkAndTmpDir( + it should "be readable using both formats after Analyze and Optimize" in withSparkAndTmpDir( (spark, tmpDir) => { writeHybridTable(spark, tmpDir) + // Analyze and Optimize the staging revision val table = QbeastTable.forPath(spark, tmpDir) table.analyze(stagingID) table.optimize(stagingID) - val allElements = spark.read.parquet(tmpDir).count - allElements shouldBe totalSize + // DataFrame should not change by optimizing the staging revision + val qbeastDf = spark.read.format("qbeast").load(tmpDir) + val deltaDf = spark.read.format("delta").load(tmpDir) + qbeastDf.count() shouldBe deltaDf.count() + + assertLargeDatasetEquality(qbeastDf, deltaDf) + // Should preserve standing staging revision behavior val snapshot = DeltaLog.forTable(spark, tmpDir).snapshot val qbeastSnapshot = DeltaQbeastSnapshot(snapshot) val stagingIndexStatus = qbeastSnapshot.loadIndexStatus(stagingID) - stagingIndexStatus.cubesStatuses.size shouldBe 1 stagingIndexStatus.replicatedOrAnnouncedSet.isEmpty shouldBe true }) @@ -64,22 +79,25 @@ class QbeastDeltaStagingTest extends QbeastIntegrationTestSpec { { writeHybridTable(spark, tmpDir) + // Number of delta files before compaction val deltaLog = DeltaLog.forTable(spark, tmpDir) val qsBefore = DeltaQbeastSnapshot(deltaLog.snapshot) val numFilesBefore = qsBefore.loadIndexStatus(stagingID).cubesStatuses.head._2.files.size + // Perform compaction val table = QbeastTable.forPath(spark, tmpDir) table.compact(stagingID) + // Number of delta files after compaction val qsAfter = DeltaQbeastSnapshot(deltaLog.update()) val numFilesAfter = qsAfter.loadIndexStatus(stagingID).cubesStatuses.head._2.files.size + numFilesAfter shouldBe <(numFilesBefore) - val deltaCount = spark.read.format("delta").load(tmpDir).count() - val qbeastCount = spark.read.format("qbeast").load(tmpDir).count() + val deltaDf = spark.read.format("delta").load(tmpDir) + val qbeastDf = spark.read.format("qbeast").load(tmpDir) - deltaCount shouldBe totalSize - qbeastCount shouldBe totalSize + assertLargeDatasetEquality(qbeastDf, deltaDf) } } From 5620d25d155531c61e023fd835bfe180e5c0718c Mon Sep 17 00:00:00 2001 From: Jiawei Date: Thu, 26 Jan 2023 16:13:55 +0100 Subject: [PATCH 70/76] Create exception message object --- .../commands/ConvertToQbeastCommand.scala | 18 ++++----------- .../spark/utils/QbeastExceptionMessages.scala | 22 +++++++++++++++++++ .../spark/utils/ConvertToQbeastTest.scala | 2 +- 3 files changed, 27 insertions(+), 15 deletions(-) create mode 100644 src/main/scala/io/qbeast/spark/utils/QbeastExceptionMessages.scala diff --git a/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala b/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala index 3928a6fe5..a81a1c7d1 100644 --- a/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala +++ b/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala @@ -5,11 +5,11 @@ package io.qbeast.spark.internal.commands import io.qbeast.core.model._ import io.qbeast.spark.delta.DeltaQbeastSnapshot -import io.qbeast.spark.internal.commands.ConvertToQbeastCommand.{ +import io.qbeast.spark.utils.MetadataConfig.{lastRevisionID, revision} +import io.qbeast.spark.utils.QbeastExceptionMessages.{ partitionedTableExceptionMsg, unsupportedFormatExceptionMsg } -import io.qbeast.spark.utils.MetadataConfig.{lastRevisionID, revision} import org.apache.http.annotation.Experimental import org.apache.spark.internal.Logging import org.apache.spark.qbeast.config.DEFAULT_CUBE_SIZE @@ -85,6 +85,7 @@ case class ConvertToQbeastCommand( if (isPartitionedDelta) { throw AnalysisExceptionFactory.create(partitionedTableExceptionMsg) } + val convRevision = Revision.emptyRevision(QTableID(tableId.table), cubeSize, columnsToIndex) val revisionID = convRevision.revisionID @@ -95,7 +96,7 @@ case class ConvertToQbeastCommand( .updated(s"$revision.$revisionID", mapper.writeValueAsString(convRevision)) val newMetadata = - txn.metadata.copy(configuration = updatedConf) // , partitionColumns = Seq.empty + txn.metadata.copy(configuration = updatedConf) txn.updateMetadata(newMetadata) txn.commit(Seq.empty, Convert(0, Seq.empty, collectStats = false, None)) @@ -104,14 +105,3 @@ case class ConvertToQbeastCommand( } } - -object ConvertToQbeastCommand { - - val partitionedTableExceptionMsg: String = - """Converting a partitioned table into qbeast is not supported. - |Consider overwriting the entire data using qbeast.""".stripMargin.replaceAll("\n", " ") - - def unsupportedFormatExceptionMsg: String => String = (fileFormat: String) => - s"Unsupported file format: $fileFormat" - -} diff --git a/src/main/scala/io/qbeast/spark/utils/QbeastExceptionMessages.scala b/src/main/scala/io/qbeast/spark/utils/QbeastExceptionMessages.scala new file mode 100644 index 000000000..328470465 --- /dev/null +++ b/src/main/scala/io/qbeast/spark/utils/QbeastExceptionMessages.scala @@ -0,0 +1,22 @@ +/* + * Copyright 2021 Qbeast Analytics, S.L. + */ +package io.qbeast.spark.utils + +object QbeastExceptionMessages { + + /** + * Conversion error for attempting to convert a partitioned table + */ + val partitionedTableExceptionMsg: String = + """Converting a partitioned table into qbeast is not supported. + |Consider overwriting the entire data using qbeast.""".stripMargin.replaceAll("\n", " ") + + /** + * Conversion error for unsupported file format + * @return Exception message with the input file format + */ + def unsupportedFormatExceptionMsg: String => String = (fileFormat: String) => + s"Unsupported file format: $fileFormat" + +} diff --git a/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala b/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala index 3cdb4f58f..4e7548fec 100644 --- a/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala +++ b/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala @@ -3,7 +3,7 @@ package io.qbeast.spark.utils import io.qbeast.core.model.RevisionUtils.{isStaging, stagingID} import io.qbeast.spark.delta.DeltaQbeastSnapshot import io.qbeast.spark.internal.commands.ConvertToQbeastCommand -import io.qbeast.spark.internal.commands.ConvertToQbeastCommand.{ +import io.qbeast.spark.utils.QbeastExceptionMessages.{ partitionedTableExceptionMsg, unsupportedFormatExceptionMsg } From c64fb5c6af4d071206db7a2315aeca34475582c3 Mon Sep 17 00:00:00 2001 From: Jiawei Date: Thu, 26 Jan 2023 17:54:50 +0100 Subject: [PATCH 71/76] Update documentation --- docs/QbeastFormat.md | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/docs/QbeastFormat.md b/docs/QbeastFormat.md index b2aa00d3f..cff83895a 100644 --- a/docs/QbeastFormat.md +++ b/docs/QbeastFormat.md @@ -154,12 +154,12 @@ In Revision, you can find different information about the tree status and config In this case, we index columns `user_id` and `product_id` (which are both `Integers`) with a linear transformation. This means that they will not suffer any transformation besides the normalization. ### Staging Revision and ConvertToQbeastCommand -The introduction of the staging revision allows users to read a delta or a hybrid `qbeast + delta` table using qbeast format. -The non-qbeast `AddFile`s are considered as part of this staging revision. +The introduction of the staging revision enables reading tables in a hybrid `qbeast + delta` state. +The non-qbeast `AddFile`s are considered as part of this staging revision, all belonging to the root. Its RevisionID is fixed to `stagingID = 0`, and it has `EmptyTransformer`s and `EmptyTransformation`s. -This Revision is normally created during the first write or when overwriting a table using qbeast. -For a table that is entirely written in `delta` or `parquet`, partitioned or not, we can use the `ConvertToQbeastCommand` to create this revision: +It is automatically created during the first write or when overwriting a table using qbeast. +For a table that is entirely written in `delta` or `parquet`, we can use the `ConvertToQbeastCommand` to create this revision: ```scala import io.qbeast.spark.internal.commands.ConvertToQbeastCommand @@ -172,10 +172,10 @@ ConvertToQbeastCommand(tableIdentifier, columnsToIndex, desiredCubeSize).run(spa val qTable = spark.read.format("qbeast").load(path) ``` +By doing so, we also enable subsequent appends using either delta or qbeast. +Conversion on a partitioned table is not supported. -By doing so, we also enable appends using either delta or the qbeast format. - -`Compaction` is allowed to group small files in the staging revision: +`Compaction` can be performed on the staging revision to group small delta files: ```scala import io.qbeast.spark.QbeastTable From f089c5056310b40914ddd6564caaeadc621ef07c Mon Sep 17 00:00:00 2001 From: Jiawei Date: Thu, 26 Jan 2023 18:16:27 +0100 Subject: [PATCH 72/76] Test table identifier format --- .../internal/commands/ConvertToQbeastCommand.scala | 6 ++++-- .../qbeast/spark/utils/QbeastExceptionMessages.scala | 8 ++++++++ .../io/qbeast/spark/utils/ConvertToQbeastTest.scala | 12 +++++++++++- 3 files changed, 23 insertions(+), 3 deletions(-) diff --git a/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala b/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala index a81a1c7d1..7209c12f2 100644 --- a/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala +++ b/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala @@ -7,6 +7,7 @@ import io.qbeast.core.model._ import io.qbeast.spark.delta.DeltaQbeastSnapshot import io.qbeast.spark.utils.MetadataConfig.{lastRevisionID, revision} import io.qbeast.spark.utils.QbeastExceptionMessages.{ + incorrectIdentifierFormat, partitionedTableExceptionMsg, unsupportedFormatExceptionMsg } @@ -43,9 +44,10 @@ case class ConvertToQbeastCommand( private def resolveTableFormat(spark: SparkSession): (String, TableIdentifier) = identifier.split("\\.") match { - case Array(f, p) => + case Array(f, p) if f.nonEmpty && p.nonEmpty => (f.toLowerCase(Locale.ROOT), spark.sessionState.sqlParser.parseTableIdentifier(p)) - case _ => throw new RuntimeException(s"Table doesn't exists at $identifier") + case _ => + throw new RuntimeException(incorrectIdentifierFormat(identifier)) } override def run(spark: SparkSession): Seq[Row] = { diff --git a/src/main/scala/io/qbeast/spark/utils/QbeastExceptionMessages.scala b/src/main/scala/io/qbeast/spark/utils/QbeastExceptionMessages.scala index 328470465..16a4bc42e 100644 --- a/src/main/scala/io/qbeast/spark/utils/QbeastExceptionMessages.scala +++ b/src/main/scala/io/qbeast/spark/utils/QbeastExceptionMessages.scala @@ -19,4 +19,12 @@ object QbeastExceptionMessages { def unsupportedFormatExceptionMsg: String => String = (fileFormat: String) => s"Unsupported file format: $fileFormat" + /** + * Conversion error for incorrect identifier format + * @return + */ + def incorrectIdentifierFormat: String => String = (identifier: String) => + "Required table identifier format: fileFormat.`` " + + s"identifier received: $identifier" + } diff --git a/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala b/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala index 4e7548fec..e2edef264 100644 --- a/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala +++ b/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala @@ -4,6 +4,7 @@ import io.qbeast.core.model.RevisionUtils.{isStaging, stagingID} import io.qbeast.spark.delta.DeltaQbeastSnapshot import io.qbeast.spark.internal.commands.ConvertToQbeastCommand import io.qbeast.spark.utils.QbeastExceptionMessages.{ + incorrectIdentifierFormat, partitionedTableExceptionMsg, unsupportedFormatExceptionMsg } @@ -126,13 +127,22 @@ class ConvertToQbeastTest extends QbeastIntegrationTestSpec with PrivateMethodTe .save(tmpDir) val revisionsBefore = getQbeastSnapshot(spark, tmpDir).loadAllRevisions - ConvertToQbeastCommand(s"qbeast.`$tmpDir`", columnsToIndex, dcs) + ConvertToQbeastCommand(s"qbeast.`$tmpDir`", columnsToIndex, dcs).run(spark) val revisionsAfter = getQbeastSnapshot(spark, tmpDir).loadAllRevisions // Revisions should not modify revisionsAfter shouldBe revisionsBefore }) + it should "fail to convert when the identifier format is not correct" in withSparkAndTmpDir( + (spark, tmpDir) => { + val identifier = s"parquet`$tmpDir`" + val thrown = the[RuntimeException] thrownBy + ConvertToQbeastCommand(identifier, columnsToIndex, dcs).run(spark) + + thrown.getMessage shouldBe incorrectIdentifierFormat(identifier) + }) + it should "preserve sampling accuracy" in withSparkAndTmpDir((spark, tmpDir) => { convertFromFormat(spark, "parquet", tmpDir) From b69320f82aac14975d101b8582536f3bf086f2c3 Mon Sep 17 00:00:00 2001 From: Jiawei Date: Thu, 26 Jan 2023 18:21:02 +0100 Subject: [PATCH 73/76] Test loadRevisionAt with invalid timestamp --- .../spark/delta/QbeastSnapshotTest.scala | 27 ++++++++++++++++++- 1 file changed, 26 insertions(+), 1 deletion(-) diff --git a/src/test/scala/io/qbeast/spark/delta/QbeastSnapshotTest.scala b/src/test/scala/io/qbeast/spark/delta/QbeastSnapshotTest.scala index 3ab1754e5..980b01710 100644 --- a/src/test/scala/io/qbeast/spark/delta/QbeastSnapshotTest.scala +++ b/src/test/scala/io/qbeast/spark/delta/QbeastSnapshotTest.scala @@ -8,7 +8,7 @@ import io.qbeast.core.model.{CubeStatus, QTableID} import io.qbeast.spark.index.SparkRevisionFactory import io.qbeast.spark.QbeastIntegrationTestSpec import org.apache.spark.sql.delta.DeltaLog -import org.apache.spark.sql.{Dataset, SparkSession} +import org.apache.spark.sql.{AnalysisException, Dataset, SparkSession} import org.scalatest.AppendedClues.convertToClueful class QbeastSnapshotTest extends QbeastIntegrationTestSpec { @@ -99,6 +99,31 @@ class QbeastSnapshotTest extends QbeastIntegrationTestSpec { } } + it should "throw an exception when no revision satisfy timestamp requirement" in + withQbeastContextSparkAndTmpDir { (spark, tmpDir) => + { + val invalidRevisionTimestamp = System.currentTimeMillis() + + val df = createDF(1000) + val names = List("age", "val2") + val cubeSize = 10 + val options = + Map("columnsToIndex" -> names.mkString(","), "cubeSize" -> cubeSize.toString) + + df.write + .format("qbeast") + .mode("overwrite") + .options(options) + .save(tmpDir) + + val deltaLog = DeltaLog.forTable(spark, tmpDir) + val qbeastSnapshot = DeltaQbeastSnapshot(deltaLog.snapshot) + an[AnalysisException] shouldBe thrownBy( + qbeastSnapshot.loadRevisionAt(invalidRevisionTimestamp)) + + } + } + "Overflowed set" should "contain only cubes that surpass desiredCubeSize" in withQbeastContextSparkAndTmpDir { (spark, tmpDir) => From 60d36aabf048da0596ee16b8bc68c1d395367f83 Mon Sep 17 00:00:00 2001 From: Jiawei Date: Thu, 26 Jan 2023 18:36:56 +0100 Subject: [PATCH 74/76] Skip Analyze and Optimize for the staging RevisionID --- .../qbeast/core/model/RevisionClasses.scala | 5 ++-- .../scala/io/qbeast/spark/QbeastTable.scala | 27 ++++++++++++------- 2 files changed, 20 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/io/qbeast/core/model/RevisionClasses.scala b/core/src/main/scala/io/qbeast/core/model/RevisionClasses.scala index acc05431d..8f978ec03 100644 --- a/core/src/main/scala/io/qbeast/core/model/RevisionClasses.scala +++ b/core/src/main/scala/io/qbeast/core/model/RevisionClasses.scala @@ -4,7 +4,7 @@ import com.fasterxml.jackson.annotation.{JsonCreator, JsonValue} import com.fasterxml.jackson.databind.annotation.JsonSerialize import com.fasterxml.jackson.databind.annotation.JsonSerialize.Typing import io.qbeast.IISeq -import io.qbeast.core.model.RevisionUtils.{isStaging, stagingID} +import io.qbeast.core.model.RevisionUtils.stagingID import io.qbeast.core.transform.{EmptyTransformer, Transformation, Transformer} import scala.collection.immutable.SortedMap @@ -229,8 +229,7 @@ case class IndexStatus( extends Serializable { def addAnnouncements(newAnnouncedSet: Set[CubeId]): IndexStatus = { - if (isStaging(revision)) this - else copy(announcedSet = announcedSet ++ newAnnouncedSet) + copy(announcedSet = announcedSet ++ newAnnouncedSet) } def cubesToOptimize: Set[CubeId] = announcedSet.diff(replicatedSet) diff --git a/src/main/scala/io/qbeast/spark/QbeastTable.scala b/src/main/scala/io/qbeast/spark/QbeastTable.scala index 37871c5b5..caaa24691 100644 --- a/src/main/scala/io/qbeast/spark/QbeastTable.scala +++ b/src/main/scala/io/qbeast/spark/QbeastTable.scala @@ -4,6 +4,7 @@ package io.qbeast.spark import io.qbeast.context.QbeastContext +import io.qbeast.core.model.RevisionUtils.isStaging import io.qbeast.core.model.{CubeId, CubeStatus, QTableID, RevisionID} import io.qbeast.spark.delta.DeltaQbeastSnapshot import io.qbeast.spark.internal.commands.{ @@ -56,13 +57,17 @@ class QbeastTable private ( * If doesn't exist or none is specified, would be the last available */ def optimize(revisionID: RevisionID): Unit = { - checkRevisionAvailable(revisionID) - OptimizeTableCommand(revisionID, indexedTable) - .run(sparkSession) + if (!isStaging(revisionID)) { + checkRevisionAvailable(revisionID) + OptimizeTableCommand(revisionID, indexedTable) + .run(sparkSession) + } } def optimize(): Unit = { - optimize(latestRevisionAvailableID) + if (!isStaging(latestRevisionAvailableID)) { + optimize(latestRevisionAvailableID) + } } /** @@ -73,14 +78,18 @@ class QbeastTable private ( * @return the sequence of cubes to optimize in string representation */ def analyze(revisionID: RevisionID): Seq[String] = { - checkRevisionAvailable(revisionID) - AnalyzeTableCommand(revisionID, indexedTable) - .run(sparkSession) - .map(_.getString(0)) + if (isStaging(revisionID)) Seq.empty + else { + checkRevisionAvailable(revisionID) + AnalyzeTableCommand(revisionID, indexedTable) + .run(sparkSession) + .map(_.getString(0)) + } } def analyze(): Seq[String] = { - analyze(latestRevisionAvailableID) + if (isStaging(latestRevisionAvailableID)) Seq.empty + else analyze(latestRevisionAvailableID) } /** From 57cacd4ff2016c40f3ee40d5a431b337f008fa4f Mon Sep 17 00:00:00 2001 From: Jiawei Date: Thu, 26 Jan 2023 18:38:22 +0100 Subject: [PATCH 75/76] Use AnalysisException, more test for EmptyTransformer --- .../scala/io/qbeast/core/transform/TransformerTest.scala | 9 ++++++--- .../spark/internal/commands/ConvertToQbeastCommand.scala | 2 +- .../io/qbeast/spark/utils/ConvertToQbeastTest.scala | 2 +- 3 files changed, 8 insertions(+), 5 deletions(-) diff --git a/core/src/test/scala/io/qbeast/core/transform/TransformerTest.scala b/core/src/test/scala/io/qbeast/core/transform/TransformerTest.scala index 5a753b6d8..2ac139524 100644 --- a/core/src/test/scala/io/qbeast/core/transform/TransformerTest.scala +++ b/core/src/test/scala/io/qbeast/core/transform/TransformerTest.scala @@ -1,8 +1,9 @@ package io.qbeast.core.transform -import io.qbeast.core.model.{DateDataType, IntegerDataType, TimestampDataType} +import io.qbeast.core.model.{DateDataType, IntegerDataType, StringDataType, TimestampDataType} import org.scalatest.flatspec.AnyFlatSpec import org.scalatest.matchers.should.Matchers + import java.sql.{Date, Timestamp} class TransformerTest extends AnyFlatSpec with Matchers { @@ -96,10 +97,12 @@ class TransformerTest extends AnyFlatSpec with Matchers { } "An EmptyTransformer" should "create an EmptyTransformation without stats" in { + EmptyTransformer.transformerSimpleName shouldBe "empty" + val colName = "a" - val transformer = EmptyTransformer(colName) - val transformation = transformer.makeTransformation(r => r) + val transformer = EmptyTransformer(colName, StringDataType) + val transformation = transformer.makeTransformation(r => r) transformation shouldBe a[EmptyTransformation] } } diff --git a/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala b/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala index 7209c12f2..222cacaee 100644 --- a/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala +++ b/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala @@ -47,7 +47,7 @@ case class ConvertToQbeastCommand( case Array(f, p) if f.nonEmpty && p.nonEmpty => (f.toLowerCase(Locale.ROOT), spark.sessionState.sqlParser.parseTableIdentifier(p)) case _ => - throw new RuntimeException(incorrectIdentifierFormat(identifier)) + throw AnalysisExceptionFactory.create(incorrectIdentifierFormat(identifier)) } override def run(spark: SparkSession): Seq[Row] = { diff --git a/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala b/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala index e2edef264..6013b68a6 100644 --- a/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala +++ b/src/test/scala/io/qbeast/spark/utils/ConvertToQbeastTest.scala @@ -137,7 +137,7 @@ class ConvertToQbeastTest extends QbeastIntegrationTestSpec with PrivateMethodTe it should "fail to convert when the identifier format is not correct" in withSparkAndTmpDir( (spark, tmpDir) => { val identifier = s"parquet`$tmpDir`" - val thrown = the[RuntimeException] thrownBy + val thrown = the[AnalysisException] thrownBy ConvertToQbeastCommand(identifier, columnsToIndex, dcs).run(spark) thrown.getMessage shouldBe incorrectIdentifierFormat(identifier) From 1332bc97dd460a0a70ca3ca3c010cf2ceeea9a5c Mon Sep 17 00:00:00 2001 From: Jiawei Date: Fri, 27 Jan 2023 13:26:14 +0100 Subject: [PATCH 76/76] Update metadata in MetadataManager --- .../qbeast/core/model/MetadataManager.scala | 11 ++++++ .../spark/delta/DeltaMetadataWriter.scala | 27 ++++++++++---- .../delta/SparkDeltaMetadataManager.scala | 18 +++++++-- .../commands/ConvertToQbeastCommand.scala | 37 ++++++------------- 4 files changed, 58 insertions(+), 35 deletions(-) diff --git a/core/src/main/scala/io/qbeast/core/model/MetadataManager.scala b/core/src/main/scala/io/qbeast/core/model/MetadataManager.scala index 4dc394925..897461113 100644 --- a/core/src/main/scala/io/qbeast/core/model/MetadataManager.scala +++ b/core/src/main/scala/io/qbeast/core/model/MetadataManager.scala @@ -8,6 +8,7 @@ import io.qbeast.IISeq * @tparam FileDescriptor type of file descriptor */ trait MetadataManager[DataSchema, FileDescriptor] { + type Configuration = Map[String, String] /** * Gets the Snapshot for a given table @@ -33,6 +34,16 @@ trait MetadataManager[DataSchema, FileDescriptor] { def updateWithTransaction(tableID: QTableID, schema: DataSchema, append: Boolean)( writer: => (TableChanges, IISeq[FileDescriptor])): Unit + /** + * Updates the table metadata by overwriting the metadata configurations + * with the provided key-value pairs. + * @param tableID QTableID + * @param schema table schema + * @param update configurations used to overwrite the existing metadata + */ + def updateMetadataWithTransaction(tableID: QTableID, schema: DataSchema)( + update: => Configuration): Unit + /** * Updates the Revision with the given RevisionChanges * @param tableID the QTableID diff --git a/src/main/scala/io/qbeast/spark/delta/DeltaMetadataWriter.scala b/src/main/scala/io/qbeast/spark/delta/DeltaMetadataWriter.scala index 1501f51db..af76b47f2 100644 --- a/src/main/scala/io/qbeast/spark/delta/DeltaMetadataWriter.scala +++ b/src/main/scala/io/qbeast/spark/delta/DeltaMetadataWriter.scala @@ -5,14 +5,9 @@ package io.qbeast.spark.delta import io.qbeast.core.model.{QTableID, RevisionID, TableChanges} import io.qbeast.spark.delta.writer.StatsTracker.registerStatsTrackers +import io.qbeast.spark.utils.QbeastExceptionMessages.partitionedTableExceptionMsg import io.qbeast.spark.utils.TagColumns -import org.apache.spark.sql.delta.actions.{ - Action, - AddFile, - FileAction, - RemoveFile, - SetTransaction -} +import org.apache.spark.sql.delta.actions._ import org.apache.spark.sql.delta.commands.DeltaCommand import org.apache.spark.sql.delta.{DeltaLog, DeltaOperations, DeltaOptions, OptimisticTransaction} import org.apache.spark.sql.execution.datasources.{ @@ -84,6 +79,24 @@ private[delta] case class DeltaMetadataWriter( } } + def updateMetadataWithTransaction(update: => Configuration): Unit = { + deltaLog.withNewTransaction { txn => + if (txn.metadata.partitionColumns.nonEmpty) { + throw AnalysisExceptionFactory.create(partitionedTableExceptionMsg) + } + + val config = update + val updatedConfig = config.foldLeft(txn.metadata.configuration) { case (accConf, (k, v)) => + accConf.updated(k, v) + } + val updatedMetadata = txn.metadata.copy(configuration = updatedConfig) + + val op = DeltaOperations.SetTableProperties(config) + txn.updateMetadata(updatedMetadata) + txn.commit(Seq.empty, op) + } + } + private def updateReplicatedFiles(tableChanges: TableChanges): Seq[Action] = { val revision = tableChanges.updatedRevision diff --git a/src/main/scala/io/qbeast/spark/delta/SparkDeltaMetadataManager.scala b/src/main/scala/io/qbeast/spark/delta/SparkDeltaMetadataManager.scala index 2c70c3395..fe4e36307 100644 --- a/src/main/scala/io/qbeast/spark/delta/SparkDeltaMetadataManager.scala +++ b/src/main/scala/io/qbeast/spark/delta/SparkDeltaMetadataManager.scala @@ -4,11 +4,11 @@ package io.qbeast.spark.delta import io.qbeast.IISeq -import io.qbeast.core.model.{MetadataManager, _} -import org.apache.spark.sql.{SaveMode, SparkSession} -import org.apache.spark.sql.delta.{DeltaLog, DeltaOptions} +import io.qbeast.core.model._ import org.apache.spark.sql.delta.actions.FileAction +import org.apache.spark.sql.delta.{DeltaLog, DeltaOptions} import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.{SaveMode, SparkSession} /** * Spark+Delta implementation of the MetadataManager interface @@ -26,6 +26,18 @@ object SparkDeltaMetadataManager extends MetadataManager[StructType, FileAction] metadataWriter.writeWithTransaction(writer) } + override def updateMetadataWithTransaction(tableID: QTableID, schema: StructType)( + update: => Configuration): Unit = { + val deltaLog = loadDeltaQbeastLog(tableID).deltaLog + val options = + new DeltaOptions(Map("path" -> tableID.id), SparkSession.active.sessionState.conf) + + val metadataWriter = + DeltaMetadataWriter(tableID, mode = SaveMode.Append, deltaLog, options, schema) + + metadataWriter.updateMetadataWithTransaction(update) + } + override def loadSnapshot(tableID: QTableID): DeltaQbeastSnapshot = { DeltaQbeastSnapshot(loadDeltaQbeastLog(tableID).deltaLog.update()) } diff --git a/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala b/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala index 222cacaee..48dfeae5e 100644 --- a/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala +++ b/src/main/scala/io/qbeast/spark/internal/commands/ConvertToQbeastCommand.scala @@ -4,7 +4,7 @@ package io.qbeast.spark.internal.commands import io.qbeast.core.model._ -import io.qbeast.spark.delta.DeltaQbeastSnapshot +import io.qbeast.spark.delta.{DeltaQbeastSnapshot, SparkDeltaMetadataManager} import io.qbeast.spark.utils.MetadataConfig.{lastRevisionID, revision} import io.qbeast.spark.utils.QbeastExceptionMessages.{ incorrectIdentifierFormat, @@ -16,7 +16,6 @@ import org.apache.spark.internal.Logging import org.apache.spark.qbeast.config.DEFAULT_CUBE_SIZE import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.delta.DeltaLog -import org.apache.spark.sql.delta.DeltaOperations.Convert import org.apache.spark.sql.execution.command.LeafRunnableCommand import org.apache.spark.sql.{AnalysisException, AnalysisExceptionFactory, Row, SparkSession} @@ -76,33 +75,21 @@ case class ConvertToQbeastCommand( case _ => throw AnalysisExceptionFactory.create(unsupportedFormatExceptionMsg(fileFormat)) } - // Convert delta to qbeast - deltaLog.update() + // Convert delta to qbeast through metadata modification + val tableID = QTableID(tableId.table) + val schema = deltaLog.snapshot.schema - val txn = deltaLog.startTransaction() + SparkDeltaMetadataManager.updateMetadataWithTransaction(tableID, schema) { + val convRevision = Revision.emptyRevision(tableID, cubeSize, columnsToIndex) + val revisionID = convRevision.revisionID - // Converting a partitioned delta table is not supported, for qbeast files - // are not partitioned. - val isPartitionedDelta = txn.metadata.partitionColumns.nonEmpty - if (isPartitionedDelta) { - throw AnalysisExceptionFactory.create(partitionedTableExceptionMsg) + // Add staging revision to Revision Map, set it as the latestRevision + Map( + lastRevisionID -> revisionID.toString, + s"$revision.$revisionID" -> mapper.writeValueAsString(convRevision)) } - - val convRevision = Revision.emptyRevision(QTableID(tableId.table), cubeSize, columnsToIndex) - val revisionID = convRevision.revisionID - - // Update revision map - val updatedConf = - txn.metadata.configuration - .updated(lastRevisionID, revisionID.toString) - .updated(s"$revision.$revisionID", mapper.writeValueAsString(convRevision)) - - val newMetadata = - txn.metadata.copy(configuration = updatedConf) - - txn.updateMetadata(newMetadata) - txn.commit(Seq.empty, Convert(0, Seq.empty, collectStats = false, None)) } + Seq.empty[Row] }