From b627bab4901b59f667f5c6467a7860c992e06733 Mon Sep 17 00:00:00 2001 From: osopardo1 Date: Wed, 27 Jul 2022 10:17:04 +0200 Subject: [PATCH 01/49] Wip on adding datasourceV2 compatibility --- .../internal/sources/QbeastDataSource.scala | 1 + .../sources/v2/QbeastTableDatasource.scala | 37 ++++ .../sources/v2/QbeastWriteBuilder.scala | 5 + .../spark/QbeastIntegrationTestSpec.scala | 5 + .../QbeastDataSourceIntegrationTest.scala | 163 +++--------------- .../spark/utils/QbeastIntegrationTest.scala | 160 +++++++++++++++++ .../utils/QbeastSQLIntegrationTest.scala | 5 + .../spark/utils/QbeastSamplingTest.scala | 36 ++++ 8 files changed, 270 insertions(+), 142 deletions(-) create mode 100644 src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastTableDatasource.scala create mode 100644 src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastWriteBuilder.scala create mode 100644 src/test/scala/io/qbeast/spark/utils/QbeastIntegrationTest.scala create mode 100644 src/test/scala/io/qbeast/spark/utils/QbeastSQLIntegrationTest.scala diff --git a/src/main/scala/io/qbeast/spark/internal/sources/QbeastDataSource.scala b/src/main/scala/io/qbeast/spark/internal/sources/QbeastDataSource.scala index c45c35dcf..9c1628edd 100644 --- a/src/main/scala/io/qbeast/spark/internal/sources/QbeastDataSource.scala +++ b/src/main/scala/io/qbeast/spark/internal/sources/QbeastDataSource.scala @@ -10,6 +10,7 @@ import org.apache.hadoop.fs.FileStatus import org.apache.spark.sql.connector.catalog.TableCapability._ import org.apache.spark.sql.connector.catalog._ import org.apache.spark.sql.connector.expressions.Transform +import org.apache.spark.sql.connector.write.{LogicalWriteInfo, WriteBuilder} import org.apache.spark.sql.sources.{ BaseRelation, CreatableRelationProvider, diff --git a/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastTableDatasource.scala b/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastTableDatasource.scala new file mode 100644 index 000000000..cd95e8acb --- /dev/null +++ b/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastTableDatasource.scala @@ -0,0 +1,37 @@ +package io.qbeast.spark.internal.sources.v2 + +import io.qbeast.context.QbeastContext.metadataManager +import io.qbeast.core.model.QTableID +import io.qbeast.spark.table.IndexedTableFactory +import org.apache.hadoop.fs.Path +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.connector.catalog.{SupportsWrite, Table, TableCapability} +import org.apache.spark.sql.connector.write.{LogicalWriteInfo, WriteBuilder} +import org.apache.spark.sql.types.StructType + +import java.util +import scala.collection.JavaConverters._ + +class QbeastTableDatasource private[sources] ( + spark: SparkSession, + path: Path, + options: Map[String, String]) + extends Table + with SupportsWrite { + private val tableID = QTableID(path.toString) + + override def name(): String = tableID.id + + override def schema(): StructType = metadataManager.loadCurrentSchema(tableID) + + override def capabilities(): util.Set[TableCapability] = Set( + TableCapability.BATCH_READ, + TableCapability.BATCH_WRITE, + TableCapability.V1_BATCH_WRITE, + TableCapability.ACCEPT_ANY_SCHEMA).asJava + + override def newWriteBuilder(info: LogicalWriteInfo): WriteBuilder = { + new QbeastWriteBuilder + } + +} diff --git a/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastWriteBuilder.scala b/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastWriteBuilder.scala new file mode 100644 index 000000000..3ad3a32f0 --- /dev/null +++ b/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastWriteBuilder.scala @@ -0,0 +1,5 @@ +package io.qbeast.spark.internal.sources.v2 + +import org.apache.spark.sql.connector.write.WriteBuilder + +class QbeastWriteBuilder extends WriteBuilder {} diff --git a/src/test/scala/io/qbeast/spark/QbeastIntegrationTestSpec.scala b/src/test/scala/io/qbeast/spark/QbeastIntegrationTestSpec.scala index 298a1d597..b6aa154b8 100644 --- a/src/test/scala/io/qbeast/spark/QbeastIntegrationTestSpec.scala +++ b/src/test/scala/io/qbeast/spark/QbeastIntegrationTestSpec.scala @@ -145,6 +145,11 @@ trait QbeastIntegrationTestSpec extends AnyFlatSpec with Matchers with DatasetCo def withQbeastContextSparkAndTmpDir[T](testCode: (SparkSession, String) => T): T = withTmpDir(tmpDir => withQbeastAndSparkContext()(spark => testCode(spark, tmpDir))) + def withQbeastContextSparkAndTmpWarehouse[T](testCode: (SparkSession, String) => T): T = + withTmpDir(tmpDir => + withExtendedSpark(new SparkConf().set("spark.sql.warehouse.dir", tmpDir))(spark => + testCode(spark, tmpDir))) + def withOTreeAlgorithm[T](code: IndexManager[DataFrame] => T): T = { code(SparkOTreeManager) } diff --git a/src/test/scala/io/qbeast/spark/utils/QbeastDataSourceIntegrationTest.scala b/src/test/scala/io/qbeast/spark/utils/QbeastDataSourceIntegrationTest.scala index 9e92ab726..85e9a592d 100644 --- a/src/test/scala/io/qbeast/spark/utils/QbeastDataSourceIntegrationTest.scala +++ b/src/test/scala/io/qbeast/spark/utils/QbeastDataSourceIntegrationTest.scala @@ -4,168 +4,47 @@ package io.qbeast.spark.utils import io.qbeast.spark.QbeastIntegrationTestSpec -import io.qbeast.spark.delta.DeltaQbeastSnapshot -import org.apache.spark.sql.delta.DeltaLog -import org.apache.spark.sql.functions._ +import org.apache.spark.sql.{DataFrame, SparkSession} class QbeastDataSourceIntegrationTest extends QbeastIntegrationTestSpec { - "the Qbeast data source" should - "expose the original number of columns and rows" in withQbeastContextSparkAndTmpDir { - (spark, tmpDir) => - { - val data = loadTestData(spark) - writeTestData(data, Seq("user_id", "product_id"), 10000, tmpDir) - - val indexed = spark.read.format("qbeast").load(tmpDir) - - data.count() shouldBe indexed.count() - - assertLargeDatasetEquality(indexed, data, orderedComparison = false) - - data.columns.toSet shouldBe indexed.columns.toSet - - } - } + def createTestData(spark: SparkSession): DataFrame = { + import spark.implicits._ + Seq(1, 2, 3, 4).toDF("id") + } - it should "index correctly on bigger spaces" in withQbeastContextSparkAndTmpDir { - (spark, tmpDir) => + "The QbeastDataSource" should + "work with DataFrame API" in withQbeastContextSparkAndTmpDir { (spark, tmpDir) => { - val data = loadTestData(spark) - .withColumn("user_id", lit(col("user_id") * Long.MaxValue)) - // WRITE SOME DATA - data.write - .mode("overwrite") - .format("qbeast") - .option("columnsToIndex", "user_id,product_id") - .save(tmpDir) + val data = createTestData(spark) + data.write.format("qbeast").option("columnsToIndex", "id").save(tmpDir) val indexed = spark.read.format("qbeast").load(tmpDir) - data.count() shouldBe indexed.count() + indexed.count() shouldBe 4 - assertLargeDatasetEquality(indexed, data, orderedComparison = false) + indexed.columns shouldBe Seq("id") - data.columns.toSet shouldBe indexed.columns.toSet + indexed.orderBy("id").collect() shouldBe data.orderBy("id").collect() } - } - - it should "index correctly on overwrite" in withQbeastContextSparkAndTmpDir { (spark, tmpDir) => - { - val data = loadTestData(spark) - // WRITE SOME DATA - writeTestData(data, Seq("user_id", "product_id"), 10000, tmpDir) - - // OVERWRITE - writeTestData(data, Seq("user_id", "product_id"), 10000, tmpDir) - - val indexed = spark.read.format("qbeast").load(tmpDir) - - data.count() shouldBe indexed.count() - - assertLargeDatasetEquality(indexed, data, orderedComparison = false) - - data.columns.toSet shouldBe indexed.columns.toSet } - } - it should "clean previous metadata on overwrite" in withQbeastContextSparkAndTmpDir { - (spark, tmpDir) => + it should "work with SaveAsTable" in withQbeastContextSparkAndTmpWarehouse { + (spark, tmpDirWarehouse) => { - val data = loadTestData(spark) - // WRITE SOME DATA - writeTestData(data, Seq("user_id", "product_id"), 10000, tmpDir) + // TODO + // DataFrame API + val data = createTestData(spark) + data.write.format("qbeast").saveAsTable("default.qbeast") - // OVERWRITE - writeTestData(data, Seq("user_id", "product_id"), 10000, tmpDir) + val indexed = spark.read.load("default.qbeast") - val deltaLog = DeltaLog.forTable(spark, tmpDir) - val qbeastSnapshot = DeltaQbeastSnapshot(deltaLog.snapshot) + indexed.count() shouldBe data.count() - qbeastSnapshot.loadAllRevisions.size shouldBe 1 - qbeastSnapshot.loadLatestRevision.revisionID shouldBe 1L + indexed.columns.toSet shouldBe data.columns.toSet } } - it should - "work with indexed columns within 0 and 1" in withQbeastContextSparkAndTmpDir { - (spark, tmpDir) => - { - import org.apache.spark.sql.functions._ - import spark.implicits._ - val data = loadTestData(spark) - val stats = data - .agg( - max('user_id).as("max_user_id"), - min('user_id).as("min_user_id"), - max('product_id).as("max_product_id"), - min('product_id).as("min_product_id")) - .collect() - .head - val (max_user, min_user, max_p, min_p) = { - (stats.getInt(0), stats.getInt(1), stats.getInt(2), stats.getInt(3)) - } - val norm_user = udf((v: Int) => (v - min_user).toDouble / (max_user - min_user)) - val norm_p = udf((v: Int) => (v - min_p).toDouble / (max_p - min_p)) - - val normalizedData = data - .withColumn("tmp_user_id", norm_user('user_id)) - .withColumn("tmp_norm_p", norm_p('product_id)) - .drop("user_id", "product_id") - .withColumnRenamed("tmp_user_id", "user_id") - .withColumnRenamed("tmp_norm_p", "product_id") - - writeTestData(normalizedData, Seq("user_id", "product_id"), 10000, tmpDir) - - val indexed = spark.read.format("qbeast").load(tmpDir) - - normalizedData.count() shouldBe indexed.count() - - assertLargeDatasetEquality(indexed, normalizedData, orderedComparison = false) - - normalizedData.columns.toSet shouldBe indexed.columns.toSet - - } - } - - it should - "append data to the original dataset" in withQbeastContextSparkAndTmpDir { (spark, tmpDir) => - { - val data = loadTestData(spark) - val columnsToIndex = Seq("user_id", "product_id") - val cubeSize = 10000 - writeTestData(data, columnsToIndex, cubeSize, tmpDir) - - val appendData = spark.read - .format("csv") - .option("header", "true") - .option("inferSchema", "true") - .load("src/test/resources/ecommerce300k_2019_Nov.csv") - - appendData.write - .mode("append") - .format("qbeast") - .options( - Map( - "columnsToIndex" -> columnsToIndex.mkString(","), - "cubeSize" -> cubeSize.toString)) - .save(tmpDir) - - val df = spark.read.format("qbeast").load(tmpDir) - val dataSize = data.count() + appendData.count() - - df.count() shouldBe dataSize - - val precision = 0.1 - val tolerance = 0.01 - // We allow a 1% of tolerance in the sampling - df.sample(withReplacement = false, precision) - .count() - .toDouble shouldBe (dataSize * precision) +- dataSize * precision * tolerance - - } - } - } diff --git a/src/test/scala/io/qbeast/spark/utils/QbeastIntegrationTest.scala b/src/test/scala/io/qbeast/spark/utils/QbeastIntegrationTest.scala new file mode 100644 index 000000000..2108ec67e --- /dev/null +++ b/src/test/scala/io/qbeast/spark/utils/QbeastIntegrationTest.scala @@ -0,0 +1,160 @@ +package io.qbeast.spark.utils + +import io.qbeast.spark.QbeastIntegrationTestSpec +import io.qbeast.spark.delta.DeltaQbeastSnapshot +import org.apache.spark.sql.delta.DeltaLog +import org.apache.spark.sql.functions.{col, lit} + +class QbeastIntegrationTest extends QbeastIntegrationTestSpec { + + "Qbeast datasource" should + "expose the original number of columns and rows" in withQbeastContextSparkAndTmpDir { + (spark, tmpDir) => + { + val data = loadTestData(spark) + writeTestData(data, Seq("user_id", "product_id"), 10000, tmpDir) + + val indexed = spark.read.format("qbeast").load(tmpDir) + + data.count() shouldBe indexed.count() + + assertLargeDatasetEquality(indexed, data, orderedComparison = false) + + data.columns.toSet shouldBe indexed.columns.toSet + + } + } + + "Qbeast index" should "index correctly on bigger spaces" in withQbeastContextSparkAndTmpDir { + (spark, tmpDir) => + { + val data = loadTestData(spark) + .withColumn("user_id", lit(col("user_id") * Long.MaxValue)) + // WRITE SOME DATA + data.write + .mode("overwrite") + .format("qbeast") + .option("columnsToIndex", "user_id,product_id") + .save(tmpDir) + + val indexed = spark.read.format("qbeast").load(tmpDir) + + data.count() shouldBe indexed.count() + + assertLargeDatasetEquality(indexed, data, orderedComparison = false) + + data.columns.toSet shouldBe indexed.columns.toSet + + } + } + + it should "index correctly on overwrite" in withQbeastContextSparkAndTmpDir { (spark, tmpDir) => + { + val data = loadTestData(spark) + // WRITE SOME DATA + writeTestData(data, Seq("user_id", "product_id"), 10000, tmpDir) + + // OVERWRITE + writeTestData(data, Seq("user_id", "product_id"), 10000, tmpDir) + + val indexed = spark.read.format("qbeast").load(tmpDir) + + data.count() shouldBe indexed.count() + + assertLargeDatasetEquality(indexed, data, orderedComparison = false) + + data.columns.toSet shouldBe indexed.columns.toSet + } + } + + it should "clean previous metadata on overwrite" in withQbeastContextSparkAndTmpDir { + (spark, tmpDir) => + { + val data = loadTestData(spark) + // WRITE SOME DATA + writeTestData(data, Seq("user_id", "product_id"), 10000, tmpDir) + + // OVERWRITE + writeTestData(data, Seq("user_id", "product_id"), 10000, tmpDir) + + val deltaLog = DeltaLog.forTable(spark, tmpDir) + val qbeastSnapshot = DeltaQbeastSnapshot(deltaLog.snapshot) + + qbeastSnapshot.loadAllRevisions.size shouldBe 1 + qbeastSnapshot.loadLatestRevision.revisionID shouldBe 1L + + } + } + + it should + "work with indexed columns within 0 and 1" in withQbeastContextSparkAndTmpDir { + (spark, tmpDir) => + { + import org.apache.spark.sql.functions._ + import spark.implicits._ + val data = loadTestData(spark) + val stats = data + .agg( + max('user_id).as("max_user_id"), + min('user_id).as("min_user_id"), + max('product_id).as("max_product_id"), + min('product_id).as("min_product_id")) + .collect() + .head + val (max_user, min_user, max_p, min_p) = { + (stats.getInt(0), stats.getInt(1), stats.getInt(2), stats.getInt(3)) + } + val norm_user = udf((v: Int) => (v - min_user).toDouble / (max_user - min_user)) + val norm_p = udf((v: Int) => (v - min_p).toDouble / (max_p - min_p)) + + val normalizedData = data + .withColumn("tmp_user_id", norm_user('user_id)) + .withColumn("tmp_norm_p", norm_p('product_id)) + .drop("user_id", "product_id") + .withColumnRenamed("tmp_user_id", "user_id") + .withColumnRenamed("tmp_norm_p", "product_id") + + writeTestData(normalizedData, Seq("user_id", "product_id"), 10000, tmpDir) + + val indexed = spark.read.format("qbeast").load(tmpDir) + + normalizedData.count() shouldBe indexed.count() + + assertLargeDatasetEquality(indexed, normalizedData, orderedComparison = false) + + normalizedData.columns.toSet shouldBe indexed.columns.toSet + + } + } + + it should + "append data to the original dataset" in withQbeastContextSparkAndTmpDir { (spark, tmpDir) => + { + val data = loadTestData(spark) + val columnsToIndex = Seq("user_id", "product_id") + val cubeSize = 10000 + writeTestData(data, columnsToIndex, cubeSize, tmpDir) + + val appendData = spark.read + .format("csv") + .option("header", "true") + .option("inferSchema", "true") + .load("src/test/resources/ecommerce300k_2019_Nov.csv") + + appendData.write + .mode("append") + .format("qbeast") + .options( + Map( + "columnsToIndex" -> columnsToIndex.mkString(","), + "cubeSize" -> cubeSize.toString)) + .save(tmpDir) + + val df = spark.read.format("qbeast").load(tmpDir) + val dataSize = data.count() + appendData.count() + + df.count() shouldBe dataSize + + } + } +} diff --git a/src/test/scala/io/qbeast/spark/utils/QbeastSQLIntegrationTest.scala b/src/test/scala/io/qbeast/spark/utils/QbeastSQLIntegrationTest.scala new file mode 100644 index 000000000..1255d4e5f --- /dev/null +++ b/src/test/scala/io/qbeast/spark/utils/QbeastSQLIntegrationTest.scala @@ -0,0 +1,5 @@ +package io.qbeast.spark.utils + +import io.qbeast.spark.QbeastIntegrationTestSpec + +class QbeastSQLIntegrationTest extends QbeastIntegrationTestSpec {} diff --git a/src/test/scala/io/qbeast/spark/utils/QbeastSamplingTest.scala b/src/test/scala/io/qbeast/spark/utils/QbeastSamplingTest.scala index d219fc9d2..f5b346788 100644 --- a/src/test/scala/io/qbeast/spark/utils/QbeastSamplingTest.scala +++ b/src/test/scala/io/qbeast/spark/utils/QbeastSamplingTest.scala @@ -111,4 +111,40 @@ class QbeastSamplingTest extends QbeastIntegrationTestSpec { }) } } + + "An appended dataset" should "sample correctly" in withQbeastContextSparkAndTmpDir { + (spark, tmpDir) => + { + val data = loadTestData(spark) + val columnsToIndex = Seq("user_id", "product_id") + val cubeSize = 10000 + writeTestData(data, columnsToIndex, cubeSize, tmpDir) + + val appendData = spark.read + .format("csv") + .option("header", "true") + .option("inferSchema", "true") + .load("src/test/resources/ecommerce300k_2019_Nov.csv") + + appendData.write + .mode("append") + .format("qbeast") + .options( + Map( + "columnsToIndex" -> columnsToIndex.mkString(","), + "cubeSize" -> cubeSize.toString)) + .save(tmpDir) + + val df = spark.read.format("qbeast").load(tmpDir) + val dataSize = data.count() + appendData.count() + + val precision = 0.1 + val tolerance = 0.01 + // We allow a 1% of tolerance in the sampling + df.sample(withReplacement = false, precision) + .count() + .toDouble shouldBe (dataSize * precision) +- dataSize * precision * tolerance + + } + } } From 99adfd7cd5a91cef16fa5856c602a5f71a977611 Mon Sep 17 00:00:00 2001 From: osopardo1 Date: Wed, 3 Aug 2022 14:20:44 +0200 Subject: [PATCH 02/49] WIP on merging V1 and V2 methods --- .../qbeast/spark/internal/QbeastOptions.scala | 17 +++ .../QbeastSparkSessionExtension.scala | 6 +- .../internal/rules/SaveAsTableRule.scala | 22 +++ .../internal/sources/QbeastBaseRelation.scala | 6 +- .../internal/sources/QbeastDataSource.scala | 59 ++++---- .../sources/catalog/QbeastCatalog.scala | 80 +++++++++++ .../sources/v2/QbeastTableDatasource.scala | 37 ----- .../internal/sources/v2/QbeastTableImpl.scala | 78 +++++++++++ .../sources/v2/QbeastWriteBuilder.scala | 30 +++- .../io/qbeast/spark/table/IndexedTable.scala | 2 +- .../spark/utils/SparkToQTypesUtils.scala | 10 -- .../org/apache/spark/sql/V2AndV1Traits.scala | 12 ++ src/test/scala/io/qbeast/TestClasses.scala | 2 + .../spark/QbeastIntegrationTestSpec.scala | 24 +++- .../QbeastDataSourceIntegrationTest.scala | 49 +++++-- .../utils/QbeastSQLIntegrationTest.scala | 128 +++++++++++++++++- 16 files changed, 462 insertions(+), 100 deletions(-) create mode 100644 src/main/scala/io/qbeast/spark/internal/rules/SaveAsTableRule.scala create mode 100644 src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalog.scala delete mode 100644 src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastTableDatasource.scala create mode 100644 src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastTableImpl.scala create mode 100644 src/main/scala/org/apache/spark/sql/V2AndV1Traits.scala diff --git a/src/main/scala/io/qbeast/spark/internal/QbeastOptions.scala b/src/main/scala/io/qbeast/spark/internal/QbeastOptions.scala index 48001b019..f7b5c33cf 100644 --- a/src/main/scala/io/qbeast/spark/internal/QbeastOptions.scala +++ b/src/main/scala/io/qbeast/spark/internal/QbeastOptions.scala @@ -3,6 +3,7 @@ */ package io.qbeast.spark.internal +import io.qbeast.core.model.QTableID import io.qbeast.spark.index.ColumnsToIndex import org.apache.spark.qbeast.config.DEFAULT_CUBE_SIZE import org.apache.spark.sql.AnalysisExceptionFactory @@ -18,8 +19,10 @@ case class QbeastOptions(columnsToIndex: Seq[String], cubeSize: Int) * Options available when trying to write in qbeast format */ object QbeastOptions { + val COLUMNS_TO_INDEX = "columnsToIndex" val CUBE_SIZE = "cubeSize" + val PATH = "path" private def getColumnsToIndex(options: Map[String, String]): Seq[String] = { val encodedColumnsToIndex = options.getOrElse( @@ -49,4 +52,18 @@ object QbeastOptions { QbeastOptions(columnsToIndex, desiredCubeSize) } + def loadTableIDFromParameters(parameters: Map[String, String]): QTableID = { + new QTableID( + parameters.getOrElse( + PATH, { + throw AnalysisExceptionFactory.create("'path' is not specified") + })) + } + + def checkQbeastProperties(parameters: Map[String, String]): Unit = { + require( + parameters.contains("columnsToIndex") || parameters.contains("columnstoindex"), + throw AnalysisExceptionFactory.create("'columnsToIndex is not specified")) + } + } diff --git a/src/main/scala/io/qbeast/spark/internal/QbeastSparkSessionExtension.scala b/src/main/scala/io/qbeast/spark/internal/QbeastSparkSessionExtension.scala index 829da0df6..caa37eb7d 100644 --- a/src/main/scala/io/qbeast/spark/internal/QbeastSparkSessionExtension.scala +++ b/src/main/scala/io/qbeast/spark/internal/QbeastSparkSessionExtension.scala @@ -4,7 +4,7 @@ package io.qbeast.spark.internal import io.delta.sql.DeltaSparkSessionExtension -import io.qbeast.spark.internal.rules.{SampleRule} +import io.qbeast.spark.internal.rules.{SampleRule, SaveAsTableRule} import org.apache.spark.sql.SparkSessionExtensions /** @@ -19,6 +19,10 @@ class QbeastSparkSessionExtension extends DeltaSparkSessionExtension { extensions.injectOptimizerRule { session => new SampleRule(session) } + + extensions.injectOptimizerRule { session => + new SaveAsTableRule(session) + } } } diff --git a/src/main/scala/io/qbeast/spark/internal/rules/SaveAsTableRule.scala b/src/main/scala/io/qbeast/spark/internal/rules/SaveAsTableRule.scala new file mode 100644 index 000000000..ebdb06655 --- /dev/null +++ b/src/main/scala/io/qbeast/spark/internal/rules/SaveAsTableRule.scala @@ -0,0 +1,22 @@ +package io.qbeast.spark.internal.rules + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.plans.logical.{CreateTableAsSelect, LogicalPlan} +import org.apache.spark.sql.catalyst.rules.Rule + +class SaveAsTableRule(spark: SparkSession) extends Rule[LogicalPlan] with Logging { + + override def apply(plan: LogicalPlan): LogicalPlan = { + // When CreateTableAsSelect statement is in place for qbeast + // We need to pass the writeOptions as properties to the creation of the table + plan transformDown { + case saveAsSelect: CreateTableAsSelect + if saveAsSelect.properties.get("provider").contains("qbeast") => + val options = saveAsSelect.writeOptions + val finalProperties = saveAsSelect.properties ++ options + saveAsSelect.copy(properties = finalProperties) + } + } + +} diff --git a/src/main/scala/io/qbeast/spark/internal/sources/QbeastBaseRelation.scala b/src/main/scala/io/qbeast/spark/internal/sources/QbeastBaseRelation.scala index 5cf3b2c71..11e1751df 100644 --- a/src/main/scala/io/qbeast/spark/internal/sources/QbeastBaseRelation.scala +++ b/src/main/scala/io/qbeast/spark/internal/sources/QbeastBaseRelation.scala @@ -3,7 +3,6 @@ */ package io.qbeast.spark.internal.sources -import io.qbeast.core.model.{QTableID} import org.apache.spark.sql.sources.BaseRelation import org.apache.spark.sql.sources.InsertableRelation @@ -49,7 +48,7 @@ object QbeastBaseRelation { val columnsToIndex = revision.columnTransformers.map(row => row.columnName).mkString(",") val cubeSize = revision.desiredCubeSize val parameters = - Map[String, String]("columnsToIndex" -> columnsToIndex, "cubeSize" -> cubeSize.toString()) + Map[String, String]("columnsToIndex" -> columnsToIndex, "cubeSize" -> cubeSize.toString) val path = new Path(tableID.id) val fileIndex = OTreeIndex(spark, path) @@ -72,11 +71,10 @@ object QbeastBaseRelation { /** * Function that can be called from a QbeastBaseRelation object to create a * new QbeastBaseRelation with a new tableID. - * @param tableID the identifier of the table * @param indexedTable the indexed table * @return BaseRelation for the new table in Qbeast format */ - def forQbeastTable(tableID: QTableID, indexedTable: IndexedTable): BaseRelation = { + def forQbeastTable(indexedTable: IndexedTable): BaseRelation = { val spark = SparkSession.active createRelation(spark.sqlContext, indexedTable) diff --git a/src/main/scala/io/qbeast/spark/internal/sources/QbeastDataSource.scala b/src/main/scala/io/qbeast/spark/internal/sources/QbeastDataSource.scala index 9c1628edd..28f267188 100644 --- a/src/main/scala/io/qbeast/spark/internal/sources/QbeastDataSource.scala +++ b/src/main/scala/io/qbeast/spark/internal/sources/QbeastDataSource.scala @@ -4,13 +4,13 @@ package io.qbeast.spark.internal.sources import io.qbeast.context.QbeastContext +import io.qbeast.spark.internal.QbeastOptions +import io.qbeast.spark.internal.QbeastOptions.checkQbeastProperties +import io.qbeast.spark.internal.sources.v2.QbeastTableImpl import io.qbeast.spark.table.IndexedTableFactory -import io.qbeast.spark.utils.SparkToQTypesUtils -import org.apache.hadoop.fs.FileStatus -import org.apache.spark.sql.connector.catalog.TableCapability._ +import org.apache.hadoop.fs.{FileStatus, Path} import org.apache.spark.sql.connector.catalog._ import org.apache.spark.sql.connector.expressions.Transform -import org.apache.spark.sql.connector.write.{LogicalWriteInfo, WriteBuilder} import org.apache.spark.sql.sources.{ BaseRelation, CreatableRelationProvider, @@ -28,7 +28,7 @@ import org.apache.spark.sql.{ } import java.util -import scala.collection.JavaConverters._ +import scala.collection.JavaConverters.mapAsScalaMapConverter /** * Qbeast data source is implementation of Spark data source API V1. @@ -48,10 +48,18 @@ class QbeastDataSource private[sources] (private val tableFactory: IndexedTableF override def inferSchema(options: CaseInsensitiveStringMap): StructType = StructType(Seq()) + // Used to get the table of an existing one override def getTable( schema: StructType, partitioning: Array[Transform], - properties: util.Map[String, String]): Table = new TableImpl(schema) + properties: util.Map[String, String]): Table = { + val tableId = QbeastOptions.loadTableIDFromParameters(properties.asScala.toMap) + new QbeastTableImpl( + new Path(tableId.id), + properties.asScala.toMap, + Some(schema), + tableFactory) + } def inferSchema( sparkSession: SparkSession, @@ -66,10 +74,9 @@ class QbeastDataSource private[sources] (private val tableFactory: IndexedTableF mode: SaveMode, parameters: Map[String, String], data: DataFrame): BaseRelation = { - require( - parameters.contains("columnsToIndex"), - throw AnalysisExceptionFactory.create("'columnsToIndex is not specified")) - val tableId = SparkToQTypesUtils.loadFromParameters(parameters) + + checkQbeastProperties(parameters) + val tableId = QbeastOptions.loadTableIDFromParameters(parameters) val table = tableFactory.getIndexedTable(tableId) mode match { case SaveMode.Append => table.save(data, parameters, append = true) @@ -85,24 +92,22 @@ class QbeastDataSource private[sources] (private val tableFactory: IndexedTableF override def createRelation( sqlContext: SQLContext, parameters: Map[String, String]): BaseRelation = { - val tableID = SparkToQTypesUtils.loadFromParameters(parameters) - val table = tableFactory.getIndexedTable(tableID) - if (table.exists) { - table.load() - } else { - throw AnalysisExceptionFactory.create( - s"'$tableID' is not a Qbeast formatted data directory.") + val tableID = QbeastOptions.loadTableIDFromParameters(parameters) + val indexedTable = tableFactory.getIndexedTable(tableID) + + // If the table has data registered on the snapshot, we can load from the IndexedTable factory + // Otherwise, the table can be loaded from the catalog + if (indexedTable.exists) indexedTable.load() + else { + // If indexedTable does not contain data + // Check if it's registered on the catalog + val tableImpl = new QbeastTableImpl(new Path(tableID.id), parameters, None, tableFactory) + if (tableImpl.isCatalogTable) { tableImpl.toBaseRelation } + else { + throw AnalysisExceptionFactory.create( + s"'$tableID' is not a Qbeast formatted data directory.") + } } } } - -private class TableImpl(schema: StructType) extends Table { - override def name(): String = "qbeast" - - override def schema(): StructType = schema - - override def capabilities(): util.Set[TableCapability] = - Set(ACCEPT_ANY_SCHEMA, BATCH_READ, V1_BATCH_WRITE, OVERWRITE_BY_FILTER, TRUNCATE).asJava - -} diff --git a/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalog.scala b/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalog.scala new file mode 100644 index 000000000..3b60054fd --- /dev/null +++ b/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalog.scala @@ -0,0 +1,80 @@ +package io.qbeast.spark.internal.sources.catalog + +import io.qbeast.context.QbeastContext +import io.qbeast.spark.internal.QbeastOptions.checkQbeastProperties +import io.qbeast.spark.internal.sources.v2.QbeastTableImpl +import org.apache.hadoop.fs.Path +import org.apache.spark.sql.{AnalysisExceptionFactory, V1TableQbeast} +import org.apache.spark.sql.catalyst.catalog.CatalogTableType +import org.apache.spark.sql.connector.catalog.{DelegatingCatalogExtension, Identifier, Table} +import org.apache.spark.sql.connector.expressions.Transform +import org.apache.spark.sql.types.StructType + +import java.util +import scala.collection.JavaConverters.mapAsScalaMapConverter + +class QbeastCatalog extends DelegatingCatalogExtension { + + private val tableFactory = QbeastContext.indexedTableFactory + + /** + * Checks if the Table is created with Qbeast Format + * @param properties the map of properties of the table + * @return a boolean set to true in the case it's a Qbeast formatted table + */ + protected def isQbeastTableProvider(properties: Map[String, String]): Boolean = { + properties.get("provider") match { + case Some("qbeast") => true + case _ => false + } + } + + /** + * Creates a qbeast table based on the underlying table + * @param table the underlying table + * @return a Table with Qbeast information and implementations + */ + protected def qbeastTable(table: Table): Table = { + + val prop = table.properties().asScala.toMap + val schema = table.schema() + + if (isQbeastTableProvider(prop)) { + table match { + case V1TableQbeast(v1Table) => + checkQbeastProperties(prop) + val catalogTable = v1Table.v1Table + + val path: String = if (catalogTable.tableType == CatalogTableType.EXTERNAL) { + // If it's an EXTERNAL TABLE, we can find the path through the Storage Properties + catalogTable.storage.locationUri.get.toString + } else if (catalogTable.tableType == CatalogTableType.MANAGED) { + // If it's a MANAGED TABLE, the location is set in the former catalogTable + catalogTable.location.toString + } else { + // Otherwise, TODO + throw AnalysisExceptionFactory.create("No path found for table " + table.name()) + } + new QbeastTableImpl(new Path(path), prop, Some(schema), tableFactory) + + case _ => table + } + } else table + } + + override def loadTable(ident: Identifier): Table = { + val superTable = super.loadTable(ident) + qbeastTable(superTable) + } + + override def createTable( + ident: Identifier, + schema: StructType, + partitions: Array[Transform], + properties: util.Map[String, String]): Table = { + + val superTable = super.createTable(ident, schema, partitions, properties) + qbeastTable(superTable) + } + +} diff --git a/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastTableDatasource.scala b/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastTableDatasource.scala deleted file mode 100644 index cd95e8acb..000000000 --- a/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastTableDatasource.scala +++ /dev/null @@ -1,37 +0,0 @@ -package io.qbeast.spark.internal.sources.v2 - -import io.qbeast.context.QbeastContext.metadataManager -import io.qbeast.core.model.QTableID -import io.qbeast.spark.table.IndexedTableFactory -import org.apache.hadoop.fs.Path -import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.connector.catalog.{SupportsWrite, Table, TableCapability} -import org.apache.spark.sql.connector.write.{LogicalWriteInfo, WriteBuilder} -import org.apache.spark.sql.types.StructType - -import java.util -import scala.collection.JavaConverters._ - -class QbeastTableDatasource private[sources] ( - spark: SparkSession, - path: Path, - options: Map[String, String]) - extends Table - with SupportsWrite { - private val tableID = QTableID(path.toString) - - override def name(): String = tableID.id - - override def schema(): StructType = metadataManager.loadCurrentSchema(tableID) - - override def capabilities(): util.Set[TableCapability] = Set( - TableCapability.BATCH_READ, - TableCapability.BATCH_WRITE, - TableCapability.V1_BATCH_WRITE, - TableCapability.ACCEPT_ANY_SCHEMA).asJava - - override def newWriteBuilder(info: LogicalWriteInfo): WriteBuilder = { - new QbeastWriteBuilder - } - -} diff --git a/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastTableImpl.scala b/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastTableImpl.scala new file mode 100644 index 000000000..565a64bd4 --- /dev/null +++ b/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastTableImpl.scala @@ -0,0 +1,78 @@ +package io.qbeast.spark.internal.sources.v2 + +import io.qbeast.context.QbeastContext._ +import io.qbeast.core.model.QTableID +import io.qbeast.spark.internal.sources.QbeastBaseRelation +import org.apache.spark.sql.connector.catalog.TableCapability._ +import io.qbeast.spark.table.IndexedTableFactory +import org.apache.hadoop.fs.Path +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.connector.catalog.{SupportsWrite, Table, TableCapability} +import org.apache.spark.sql.connector.write.{LogicalWriteInfo, WriteBuilder} +import org.apache.spark.sql.sources.BaseRelation +import org.apache.spark.sql.types.StructType + +import java.util +import scala.collection.JavaConverters._ + +class QbeastTableImpl private[sources] ( + path: Path, + options: Map[String, String], + schema: Option[StructType] = None, + private val tableFactory: IndexedTableFactory) + extends Table + with SupportsWrite { + + private val spark = SparkSession.active + + private val pathString = path.toString + + private val tableId = QTableID(pathString) + + private val indexedTable = tableFactory.getIndexedTable(tableId) + + /** + * Checks if the table exists on the catalog + * @return true if exists + */ + def isCatalogTable: Boolean = { + // TODO Check if exists on the catalog + // I don't think this is the better way to do so + val tableName = pathString.split("/").last + spark.catalog.tableExists(tableName) + } + + override def name(): String = tableId.id + + override def schema(): StructType = { + if (schema.isDefined) schema.get + else metadataManager.loadCurrentSchema(tableId) + } + + override def capabilities(): util.Set[TableCapability] = + Set(ACCEPT_ANY_SCHEMA, BATCH_READ, V1_BATCH_WRITE, OVERWRITE_BY_FILTER, TRUNCATE).asJava + + // Returns the write builder for the query in info + override def newWriteBuilder(info: LogicalWriteInfo): WriteBuilder = { + new QbeastWriteBuilder(info, indexedTable) + } + + def toBaseRelation: BaseRelation = { + QbeastBaseRelation.forQbeastTable(indexedTable) + } + + override def properties(): util.Map[String, String] = options.asJava + + // TODO extend with SupportRead + /* + override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder = { + val fileIndex = OTreeIndex(SparkSession.active, path) + val partitioningAwareFileIndex = PartitioningAwareFileIndex() + new FileScanBuilder(spark, fileIndex, schema()) { + override def build(): Scan = ??? + } + } + + */ + +} diff --git a/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastWriteBuilder.scala b/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastWriteBuilder.scala index 3ad3a32f0..2b0692084 100644 --- a/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastWriteBuilder.scala +++ b/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastWriteBuilder.scala @@ -1,5 +1,31 @@ package io.qbeast.spark.internal.sources.v2 -import org.apache.spark.sql.connector.write.WriteBuilder +import io.qbeast.spark.table.IndexedTable +import org.apache.spark.sql.{DataFrame} +import org.apache.spark.sql.connector.write.{ + LogicalWriteInfo, + SupportsOverwrite, + V1WriteBuilder, + WriteBuilder +} +import org.apache.spark.sql.sources.{Filter, InsertableRelation} -class QbeastWriteBuilder extends WriteBuilder {} +import scala.collection.convert.ImplicitConversions.`map AsScala` + +class QbeastWriteBuilder(info: LogicalWriteInfo, indexedTable: IndexedTable) + extends WriteBuilder + with V1WriteBuilder + with SupportsOverwrite { + + override def overwrite(filters: Array[Filter]): WriteBuilder = this + + override def buildForV1Write(): InsertableRelation = { + + new InsertableRelation { + def insert(data: DataFrame, overwrite: Boolean): Unit = { + indexedTable.save(data, info.options().asCaseSensitiveMap().toMap, append = !overwrite) + } + } + } + +} diff --git a/src/main/scala/io/qbeast/spark/table/IndexedTable.scala b/src/main/scala/io/qbeast/spark/table/IndexedTable.scala index 77446a2d5..4bf9cc188 100644 --- a/src/main/scala/io/qbeast/spark/table/IndexedTable.scala +++ b/src/main/scala/io/qbeast/spark/table/IndexedTable.scala @@ -196,7 +196,7 @@ private[table] class IndexedTableImpl( * @return the QbeastBaseRelation */ private def createQbeastBaseRelation(): BaseRelation = { - QbeastBaseRelation.forQbeastTable(tableID, this) + QbeastBaseRelation.forQbeastTable(this) } private def write(data: DataFrame, indexStatus: IndexStatus, append: Boolean): BaseRelation = { diff --git a/src/main/scala/io/qbeast/spark/utils/SparkToQTypesUtils.scala b/src/main/scala/io/qbeast/spark/utils/SparkToQTypesUtils.scala index c48b9e854..e8d82e360 100644 --- a/src/main/scala/io/qbeast/spark/utils/SparkToQTypesUtils.scala +++ b/src/main/scala/io/qbeast/spark/utils/SparkToQTypesUtils.scala @@ -3,21 +3,11 @@ */ package io.qbeast.spark.utils -import io.qbeast.core.model.QTableID import io.qbeast.core.{model => qmodel} -import org.apache.spark.sql.AnalysisExceptionFactory import org.apache.spark.sql.types._ object SparkToQTypesUtils { - def loadFromParameters(parameters: Map[String, String]): QTableID = { - new QTableID( - parameters.getOrElse( - "path", { - throw AnalysisExceptionFactory.create("'path' is not specified") - })) - } - def convertDataTypes(sparkType: DataType): io.qbeast.core.model.QDataType = sparkType match { case _: DoubleType => qmodel.DoubleDataType case _: IntegerType => qmodel.IntegerDataType diff --git a/src/main/scala/org/apache/spark/sql/V2AndV1Traits.scala b/src/main/scala/org/apache/spark/sql/V2AndV1Traits.scala new file mode 100644 index 000000000..0fe0b9815 --- /dev/null +++ b/src/main/scala/org/apache/spark/sql/V2AndV1Traits.scala @@ -0,0 +1,12 @@ +package org.apache.spark.sql + +import org.apache.spark.sql.connector.catalog.{Table, V1Table} + +object V1TableQbeast { + + def unapply(table: Table): Option[V1Table] = table match { + case v1T @ V1Table(_) => Some(v1T) + case _ => None + } + +} diff --git a/src/test/scala/io/qbeast/TestClasses.scala b/src/test/scala/io/qbeast/TestClasses.scala index db2963c4a..e0399baa3 100644 --- a/src/test/scala/io/qbeast/TestClasses.scala +++ b/src/test/scala/io/qbeast/TestClasses.scala @@ -34,4 +34,6 @@ object TestClasses { val2: Option[Long], val3: Option[Double]) + case class Student(id: Int, name: String, age: Int) + } diff --git a/src/test/scala/io/qbeast/spark/QbeastIntegrationTestSpec.scala b/src/test/scala/io/qbeast/spark/QbeastIntegrationTestSpec.scala index b6aa154b8..45fbac587 100644 --- a/src/test/scala/io/qbeast/spark/QbeastIntegrationTestSpec.scala +++ b/src/test/scala/io/qbeast/spark/QbeastIntegrationTestSpec.scala @@ -10,10 +10,10 @@ import io.qbeast.core.model.IndexManager import io.qbeast.spark.delta.SparkDeltaMetadataManager import io.qbeast.spark.index.{SparkOTreeManager, SparkRevisionFactory} import io.qbeast.spark.index.writer.SparkDataWriter -import io.qbeast.spark.internal.QbeastSparkSessionExtension import io.qbeast.spark.table.IndexedTableFactoryImpl import org.apache.log4j.{Level, Logger} import org.apache.spark.SparkConf +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.{DataFrame, SparkSession} import org.scalatest.flatspec.AnyFlatSpec import org.scalatest.matchers.should.Matchers @@ -66,12 +66,16 @@ trait QbeastIntegrationTestSpec extends AnyFlatSpec with Matchers with DatasetCo */ def withExtendedSpark[T](sparkConf: SparkConf = new SparkConf())( testCode: SparkSession => T): T = { + val conf = sparkConf + .setMaster("local[8]") + .set("spark.sql.extensions", "io.qbeast.spark.internal.QbeastSparkSessionExtension") + .set( + SQLConf.V2_SESSION_CATALOG_IMPLEMENTATION.key, + "io.qbeast.spark.internal.sources.catalog.QbeastCatalog") val spark = SparkSession .builder() - .master("local[8]") .appName("QbeastDataSource") - .withExtensions(new QbeastSparkSessionExtension()) - .config(sparkConf) + .config(conf) .getOrCreate() try { testCode(spark) @@ -87,11 +91,19 @@ trait QbeastIntegrationTestSpec extends AnyFlatSpec with Matchers with DatasetCo * @return */ def withSpark[T](testCode: SparkSession => T): T = { + // Spark Configuration + // Including Session Extensions and Catalog + val conf = new SparkConf() + .setMaster("local[8]") + .set("spark.sql.extensions", "io.qbeast.spark.internal.QbeastSparkSessionExtension") + .set( + SQLConf.V2_SESSION_CATALOG_IMPLEMENTATION.key, + "io.qbeast.spark.internal.sources.catalog.QbeastCatalog") + val spark = SparkSession .builder() - .master("local[8]") + .config(conf) .appName("QbeastDataSource") - .withExtensions(new QbeastSparkSessionExtension()) .getOrCreate() try { testCode(spark) diff --git a/src/test/scala/io/qbeast/spark/utils/QbeastDataSourceIntegrationTest.scala b/src/test/scala/io/qbeast/spark/utils/QbeastDataSourceIntegrationTest.scala index 85e9a592d..530c8a1a5 100644 --- a/src/test/scala/io/qbeast/spark/utils/QbeastDataSourceIntegrationTest.scala +++ b/src/test/scala/io/qbeast/spark/utils/QbeastDataSourceIntegrationTest.scala @@ -30,21 +30,48 @@ class QbeastDataSourceIntegrationTest extends QbeastIntegrationTestSpec { } } - it should "work with SaveAsTable" in withQbeastContextSparkAndTmpWarehouse { - (spark, tmpDirWarehouse) => - { - // TODO - // DataFrame API - val data = createTestData(spark) - data.write.format("qbeast").saveAsTable("default.qbeast") + it should "work with InsertInto" in withQbeastContextSparkAndTmpDir { (spark, tmpDir) => + { + // TODO + val data = createTestData(spark) + data.write.format("qbeast").option("columnsToIndex", "id").save(tmpDir) - val indexed = spark.read.load("default.qbeast") + // Create the table + val indexedTable = spark.read.format("qbeast").load(tmpDir) + indexedTable.createOrReplaceTempView("qbeast") - indexed.count() shouldBe data.count() + // Insert Into table + data.write.format("qbeast").insertInto("default.qbeast") - indexed.columns.toSet shouldBe data.columns.toSet + val indexed = spark.read.format("qbeast").load(tmpDir) - } + indexed.count() shouldBe data.count() + + indexed.columns.toSet shouldBe data.columns.toSet + } + } + + it should "work with SaveAsTable" in withQbeastContextSparkAndTmpWarehouse { (spark, tmpDir) => + { + + val data = createTestData(spark) + val location = tmpDir + "/external" + data.write + .format("qbeast") + .option("columnsToIndex", "id") + .option("location", location) + .saveAsTable("qbeast") + + // spark.catalog.listTables().show(false) + + // val indexed = spark.table("qbeast") + val indexed = spark.read.format("qbeast").load(location) + + indexed.count() shouldBe data.count() + + indexed.columns.toSet shouldBe data.columns.toSet + + } } } diff --git a/src/test/scala/io/qbeast/spark/utils/QbeastSQLIntegrationTest.scala b/src/test/scala/io/qbeast/spark/utils/QbeastSQLIntegrationTest.scala index 1255d4e5f..eab85a315 100644 --- a/src/test/scala/io/qbeast/spark/utils/QbeastSQLIntegrationTest.scala +++ b/src/test/scala/io/qbeast/spark/utils/QbeastSQLIntegrationTest.scala @@ -1,5 +1,131 @@ package io.qbeast.spark.utils +import io.qbeast.TestClasses.Student import io.qbeast.spark.QbeastIntegrationTestSpec -class QbeastSQLIntegrationTest extends QbeastIntegrationTestSpec {} +import scala.util.Random + +class QbeastSQLIntegrationTest extends QbeastIntegrationTestSpec { + // ALL CREATE TABLE STATEMENTS + /** + * --Use data source + * CREATE TABLE student (id INT, name STRING, age INT) USING CSV; + * + * --Use data from another table + * CREATE TABLE student_copy USING CSV + * AS SELECT * FROM student; + * + * --Omit the USING clause, which uses the default data source (parquet by default) + * CREATE TABLE student (id INT, name STRING, age INT); + * + * --Specify table comment and properties + * CREATE TABLE student (id INT, name STRING, age INT) USING CSV + * COMMENT 'this is a comment' + * TBLPROPERTIES ('foo'='bar'); + * + * --Specify table comment and properties with different clauses order + * CREATE TABLE student (id INT, name STRING, age INT) USING CSV + * TBLPROPERTIES ('foo'='bar') + * COMMENT 'this is a comment'; + * + * --Create partitioned and bucketed table + * CREATE TABLE student (id INT, name STRING, age INT) + * USING CSV + * PARTITIONED BY (age) + * CLUSTERED BY (Id) INTO 4 buckets; + * + * --Create partitioned and bucketed table through CTAS + * CREATE TABLE student_partition_bucket + * USING parquet + * PARTITIONED BY (age) + * CLUSTERED BY (id) INTO 4 buckets + * AS SELECT * FROM student; + * + * --Create bucketed table through CTAS and CTE + * CREATE TABLE student_bucket + * USING parquet + * CLUSTERED BY (id) INTO 4 buckets ( + * WITH tmpTable AS ( + * SELECT * FROM student WHERE id > 100 + * ) + * SELECT * FROM tmpTable + * ); + */ + + private val students = 1.to(10).map(i => Student(i, i.toString, Random.nextInt())) + + "QbeastSpark" should "work with SQL CREATE TABLE" in withQbeastContextSparkAndTmpWarehouse( + (spark, tmpWarehouse) => { + import spark.implicits._ + val data = students.toDF() + data.createOrReplaceTempView("data") + + spark.sql( + s"CREATE TABLE student (id INT, name STRING, age INT) USING qbeast " + + "OPTIONS ('columnsToIndex'='id')") + + val nonTemporaryTables = spark.sql("SHOW TABLES FROM default") + nonTemporaryTables.count() shouldBe 2 // data table and student table + + val table = spark.sql("DESCRIBE TABLE EXTENDED student") + table.show(false) + // Check provider + table + .where("col_name == 'Provider'") + .select("data_type") + .first() + .getString(0) shouldBe "qbeast" + // Check Location + table + .where("col_name == 'Location'") + .select("data_type") + .first() + .getString(0) shouldBe tmpWarehouse + "/student" + // Check Table Properties + table + .where("col_name == 'Storage Properties'") + .select("data_type") + .first() + .getString(0) shouldBe "[columnsToIndex=id]" + + }) + + it should "work with INSERT INTO" in withQbeastContextSparkAndTmpWarehouse((spark, _) => { + import spark.implicits._ + val data = students.toDF() + data.createOrReplaceTempView("data") + + spark.sql( + s"CREATE TABLE student (id INT, name STRING, age INT) USING qbeast " + + "OPTIONS ('columnsToIndex'='id')") + + spark.sql("INSERT INTO table student SELECT * FROM data") // FAILS + }) + + it should "work with CREATE TABLE AS SELECT statement" in withQbeastContextSparkAndTmpDir( + (spark, _) => { + + import spark.implicits._ + val data = students.toDF() + data.createOrReplaceTempView("data") + + spark.sql( + s"CREATE OR REPLACE TABLE student (id INT, name STRING, age INT) USING qbeast " + + "OPTIONS ('columnsToIndex'='id') " + + "AS SELECT * FROM data;") + }) + + it should "work with LOCATION" in withQbeastContextSparkAndTmpDir((spark, tmpDir) => { + + import spark.implicits._ + val data = students.toDF() + data.createOrReplaceTempView("data") + + spark.sql( + s"CREATE OR REPLACE TABLE student (id INT, name STRING, age INT) USING qbeast " + + "OPTIONS ('columnsToIndex'='id') " + + s"LOCATION '$tmpDir' " + + "AS SELECT * FROM data;") + }) + +} From 037a2903668eb1dd7f130c93d9d98226ea095a9c Mon Sep 17 00:00:00 2001 From: osopardo1 Date: Wed, 3 Aug 2022 14:56:10 +0200 Subject: [PATCH 03/49] Add tests --- .../QbeastDataSourceIntegrationTest.scala | 24 ---------- .../utils/QbeastSQLIntegrationTest.scala | 45 ------------------- 2 files changed, 69 deletions(-) diff --git a/src/test/scala/io/qbeast/spark/utils/QbeastDataSourceIntegrationTest.scala b/src/test/scala/io/qbeast/spark/utils/QbeastDataSourceIntegrationTest.scala index 530c8a1a5..27fb68ad6 100644 --- a/src/test/scala/io/qbeast/spark/utils/QbeastDataSourceIntegrationTest.scala +++ b/src/test/scala/io/qbeast/spark/utils/QbeastDataSourceIntegrationTest.scala @@ -30,27 +30,6 @@ class QbeastDataSourceIntegrationTest extends QbeastIntegrationTestSpec { } } - it should "work with InsertInto" in withQbeastContextSparkAndTmpDir { (spark, tmpDir) => - { - // TODO - val data = createTestData(spark) - data.write.format("qbeast").option("columnsToIndex", "id").save(tmpDir) - - // Create the table - val indexedTable = spark.read.format("qbeast").load(tmpDir) - indexedTable.createOrReplaceTempView("qbeast") - - // Insert Into table - data.write.format("qbeast").insertInto("default.qbeast") - - val indexed = spark.read.format("qbeast").load(tmpDir) - - indexed.count() shouldBe data.count() - - indexed.columns.toSet shouldBe data.columns.toSet - } - } - it should "work with SaveAsTable" in withQbeastContextSparkAndTmpWarehouse { (spark, tmpDir) => { @@ -62,9 +41,6 @@ class QbeastDataSourceIntegrationTest extends QbeastIntegrationTestSpec { .option("location", location) .saveAsTable("qbeast") - // spark.catalog.listTables().show(false) - - // val indexed = spark.table("qbeast") val indexed = spark.read.format("qbeast").load(location) indexed.count() shouldBe data.count() diff --git a/src/test/scala/io/qbeast/spark/utils/QbeastSQLIntegrationTest.scala b/src/test/scala/io/qbeast/spark/utils/QbeastSQLIntegrationTest.scala index eab85a315..84a0452bd 100644 --- a/src/test/scala/io/qbeast/spark/utils/QbeastSQLIntegrationTest.scala +++ b/src/test/scala/io/qbeast/spark/utils/QbeastSQLIntegrationTest.scala @@ -6,51 +6,6 @@ import io.qbeast.spark.QbeastIntegrationTestSpec import scala.util.Random class QbeastSQLIntegrationTest extends QbeastIntegrationTestSpec { - // ALL CREATE TABLE STATEMENTS - /** - * --Use data source - * CREATE TABLE student (id INT, name STRING, age INT) USING CSV; - * - * --Use data from another table - * CREATE TABLE student_copy USING CSV - * AS SELECT * FROM student; - * - * --Omit the USING clause, which uses the default data source (parquet by default) - * CREATE TABLE student (id INT, name STRING, age INT); - * - * --Specify table comment and properties - * CREATE TABLE student (id INT, name STRING, age INT) USING CSV - * COMMENT 'this is a comment' - * TBLPROPERTIES ('foo'='bar'); - * - * --Specify table comment and properties with different clauses order - * CREATE TABLE student (id INT, name STRING, age INT) USING CSV - * TBLPROPERTIES ('foo'='bar') - * COMMENT 'this is a comment'; - * - * --Create partitioned and bucketed table - * CREATE TABLE student (id INT, name STRING, age INT) - * USING CSV - * PARTITIONED BY (age) - * CLUSTERED BY (Id) INTO 4 buckets; - * - * --Create partitioned and bucketed table through CTAS - * CREATE TABLE student_partition_bucket - * USING parquet - * PARTITIONED BY (age) - * CLUSTERED BY (id) INTO 4 buckets - * AS SELECT * FROM student; - * - * --Create bucketed table through CTAS and CTE - * CREATE TABLE student_bucket - * USING parquet - * CLUSTERED BY (id) INTO 4 buckets ( - * WITH tmpTable AS ( - * SELECT * FROM student WHERE id > 100 - * ) - * SELECT * FROM tmpTable - * ); - */ private val students = 1.to(10).map(i => Student(i, i.toString, Random.nextInt())) From 7baadd2bfa76a1488c97ed0acc35ed6837be506e Mon Sep 17 00:00:00 2001 From: osopardo1 Date: Wed, 3 Aug 2022 14:59:56 +0200 Subject: [PATCH 04/49] Fix SQL in tests --- .../io/qbeast/spark/utils/QbeastSQLIntegrationTest.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/test/scala/io/qbeast/spark/utils/QbeastSQLIntegrationTest.scala b/src/test/scala/io/qbeast/spark/utils/QbeastSQLIntegrationTest.scala index 84a0452bd..ab1e64388 100644 --- a/src/test/scala/io/qbeast/spark/utils/QbeastSQLIntegrationTest.scala +++ b/src/test/scala/io/qbeast/spark/utils/QbeastSQLIntegrationTest.scala @@ -65,7 +65,7 @@ class QbeastSQLIntegrationTest extends QbeastIntegrationTestSpec { data.createOrReplaceTempView("data") spark.sql( - s"CREATE OR REPLACE TABLE student (id INT, name STRING, age INT) USING qbeast " + + s"CREATE OR REPLACE TABLE student USING qbeast " + "OPTIONS ('columnsToIndex'='id') " + "AS SELECT * FROM data;") }) @@ -77,7 +77,7 @@ class QbeastSQLIntegrationTest extends QbeastIntegrationTestSpec { data.createOrReplaceTempView("data") spark.sql( - s"CREATE OR REPLACE TABLE student (id INT, name STRING, age INT) USING qbeast " + + s"CREATE OR REPLACE TABLE student USING qbeast " + "OPTIONS ('columnsToIndex'='id') " + s"LOCATION '$tmpDir' " + "AS SELECT * FROM data;") From 6812a85e869bff77c5bce27f4ebcbc0a3f4f21af Mon Sep 17 00:00:00 2001 From: osopardo1 Date: Thu, 4 Aug 2022 15:10:59 +0200 Subject: [PATCH 05/49] Add ReadSupport and TODOs on the code --- .../internal/sources/QbeastDataSource.scala | 6 +- .../internal/sources/v2/QbeastScanRDD.scala | 72 +++++++++++++++++++ .../internal/sources/v2/QbeastTableImpl.scala | 27 +++---- .../sources/v2/QbeastWriteBuilder.scala | 14 +++- .../utils/QbeastSQLIntegrationTest.scala | 36 ++++++---- 5 files changed, 123 insertions(+), 32 deletions(-) create mode 100644 src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastScanRDD.scala diff --git a/src/main/scala/io/qbeast/spark/internal/sources/QbeastDataSource.scala b/src/main/scala/io/qbeast/spark/internal/sources/QbeastDataSource.scala index 28f267188..f65672d92 100644 --- a/src/main/scala/io/qbeast/spark/internal/sources/QbeastDataSource.scala +++ b/src/main/scala/io/qbeast/spark/internal/sources/QbeastDataSource.scala @@ -54,11 +54,7 @@ class QbeastDataSource private[sources] (private val tableFactory: IndexedTableF partitioning: Array[Transform], properties: util.Map[String, String]): Table = { val tableId = QbeastOptions.loadTableIDFromParameters(properties.asScala.toMap) - new QbeastTableImpl( - new Path(tableId.id), - properties.asScala.toMap, - Some(schema), - tableFactory) + new QbeastTableImpl(new Path(tableId.id), properties.asScala.toMap, None, tableFactory) } def inferSchema( diff --git a/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastScanRDD.scala b/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastScanRDD.scala new file mode 100644 index 000000000..5470f9e2f --- /dev/null +++ b/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastScanRDD.scala @@ -0,0 +1,72 @@ +package io.qbeast.spark.internal.sources.v2 + +import io.qbeast.context.QbeastContext._ +import io.qbeast.spark.internal.sources.QbeastBaseRelation +import io.qbeast.spark.table.IndexedTable +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.encoders.RowEncoder +import org.apache.spark.sql.{Row, SQLContext} +import org.apache.spark.sql.connector.read.V1Scan +import org.apache.spark.sql.execution.FileSourceScanExec +import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation} +import org.apache.spark.sql.sources.{BaseRelation, TableScan} +import org.apache.spark.sql.types.StructType + +import scala.collection.JavaConverters.asScalaIteratorConverter + +/** + * Extends Scan Builder for V1 DataSource + * It allows Spark to read from a Qbeast Formatted table + * TODO include here the logic to get rid of the QbeastHash while reading the records + * @param indexedTable the IndexedTable + */ +class QbeastScanRDD(indexedTable: IndexedTable) extends V1Scan { + + private lazy val qbeastBaseRelation = + QbeastBaseRelation.forQbeastTable(indexedTable).asInstanceOf[HadoopFsRelation] + + override def toV1TableScan[T <: BaseRelation with TableScan](context: SQLContext): T = { + + // TODO add PrunedFilteredScan as an extension and implement the methods + new BaseRelation with TableScan { + override def sqlContext: SQLContext = context + + override def schema: StructType = qbeastBaseRelation.schema + + override def buildScan(): RDD[Row] = { + val schema = qbeastBaseRelation.schema + + // Ugly hack to convert the schema fields into Attributes + val output = LogicalRelation + .apply(qbeastBaseRelation) + .resolve(schema, context.sparkSession.sessionState.analyzer.resolver) + + // TODO check the parameters passed to this method + FileSourceScanExec( + relation = qbeastBaseRelation, + output = output, + requiredSchema = schema, + partitionFilters = Seq.empty, + optionalBucketSet = None, + optionalNumCoalescedBuckets = None, + dataFilters = Seq.empty, + tableIdentifier = None) + .executeColumnar() + .mapPartitions { batches => + batches.flatMap { batch => + batch + .rowIterator() + .asScala + .map(internal => { + val encoder = RowEncoder(schema).resolveAndBind() + encoder.createDeserializer().apply(internal) + }) + } + } + } + }.asInstanceOf[T] + } + + override def readSchema(): StructType = metadataManager.loadCurrentSchema(indexedTable.tableID) + +} diff --git a/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastTableImpl.scala b/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastTableImpl.scala index 565a64bd4..bcf1d86a6 100644 --- a/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastTableImpl.scala +++ b/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastTableImpl.scala @@ -7,10 +7,17 @@ import org.apache.spark.sql.connector.catalog.TableCapability._ import io.qbeast.spark.table.IndexedTableFactory import org.apache.hadoop.fs.Path import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.connector.catalog.{SupportsWrite, Table, TableCapability} +import org.apache.spark.sql.connector.catalog.{ + SupportsRead, + SupportsWrite, + Table, + TableCapability +} +import org.apache.spark.sql.connector.read.{Scan, ScanBuilder} import org.apache.spark.sql.connector.write.{LogicalWriteInfo, WriteBuilder} import org.apache.spark.sql.sources.BaseRelation import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.CaseInsensitiveStringMap import java.util import scala.collection.JavaConverters._ @@ -21,7 +28,8 @@ class QbeastTableImpl private[sources] ( schema: Option[StructType] = None, private val tableFactory: IndexedTableFactory) extends Table - with SupportsWrite { + with SupportsWrite + with SupportsRead { private val spark = SparkSession.active @@ -54,7 +62,7 @@ class QbeastTableImpl private[sources] ( // Returns the write builder for the query in info override def newWriteBuilder(info: LogicalWriteInfo): WriteBuilder = { - new QbeastWriteBuilder(info, indexedTable) + new QbeastWriteBuilder(info, options, indexedTable) } def toBaseRelation: BaseRelation = { @@ -63,16 +71,9 @@ class QbeastTableImpl private[sources] ( override def properties(): util.Map[String, String] = options.asJava - // TODO extend with SupportRead - /* - override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder = { - val fileIndex = OTreeIndex(SparkSession.active, path) - val partitioningAwareFileIndex = PartitioningAwareFileIndex() - new FileScanBuilder(spark, fileIndex, schema()) { - override def build(): Scan = ??? + override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder = + new ScanBuilder { + override def build(): Scan = new QbeastScanRDD(indexedTable) } - } - - */ } diff --git a/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastWriteBuilder.scala b/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastWriteBuilder.scala index 2b0692084..8ad21c794 100644 --- a/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastWriteBuilder.scala +++ b/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastWriteBuilder.scala @@ -12,18 +12,28 @@ import org.apache.spark.sql.sources.{Filter, InsertableRelation} import scala.collection.convert.ImplicitConversions.`map AsScala` -class QbeastWriteBuilder(info: LogicalWriteInfo, indexedTable: IndexedTable) +class QbeastWriteBuilder( + info: LogicalWriteInfo, + properties: Map[String, String], + indexedTable: IndexedTable) extends WriteBuilder with V1WriteBuilder with SupportsOverwrite { override def overwrite(filters: Array[Filter]): WriteBuilder = this + /** + * Build an InsertableRelation to be able to write the data in QbeastFormat + * @return the InsertableRelation with the corresponding method + */ override def buildForV1Write(): InsertableRelation = { new InsertableRelation { def insert(data: DataFrame, overwrite: Boolean): Unit = { - indexedTable.save(data, info.options().asCaseSensitiveMap().toMap, append = !overwrite) + // Passing the options in the query plan plus the properties + // because columnsToIndex needs to be included in the contract + val writeOptions = info.options().toMap ++ properties + indexedTable.save(data, writeOptions, append = !overwrite) } } } diff --git a/src/test/scala/io/qbeast/spark/utils/QbeastSQLIntegrationTest.scala b/src/test/scala/io/qbeast/spark/utils/QbeastSQLIntegrationTest.scala index ab1e64388..6565955e4 100644 --- a/src/test/scala/io/qbeast/spark/utils/QbeastSQLIntegrationTest.scala +++ b/src/test/scala/io/qbeast/spark/utils/QbeastSQLIntegrationTest.scala @@ -23,25 +23,19 @@ class QbeastSQLIntegrationTest extends QbeastIntegrationTestSpec { nonTemporaryTables.count() shouldBe 2 // data table and student table val table = spark.sql("DESCRIBE TABLE EXTENDED student") - table.show(false) + // TODO Check the metadata of the table // Check provider table .where("col_name == 'Provider'") .select("data_type") .first() .getString(0) shouldBe "qbeast" - // Check Location - table - .where("col_name == 'Location'") - .select("data_type") - .first() - .getString(0) shouldBe tmpWarehouse + "/student" // Check Table Properties table - .where("col_name == 'Storage Properties'") + .where("col_name == 'Table Properties'") .select("data_type") .first() - .getString(0) shouldBe "[columnsToIndex=id]" + .getString(0) should contain("columnsToIndex=id") }) @@ -52,12 +46,18 @@ class QbeastSQLIntegrationTest extends QbeastIntegrationTestSpec { spark.sql( s"CREATE TABLE student (id INT, name STRING, age INT) USING qbeast " + - "OPTIONS ('columnsToIndex'='id')") + "TBLPROPERTIES ('columnsToIndex'='id')") + + spark.sql("INSERT INTO table student SELECT * FROM data") + + val indexed = spark.table("student") + + indexed.count() shouldBe data.count() - spark.sql("INSERT INTO table student SELECT * FROM data") // FAILS + indexed.columns.toSet shouldBe data.columns.toSet }) - it should "work with CREATE TABLE AS SELECT statement" in withQbeastContextSparkAndTmpDir( + it should "work with CREATE TABLE AS SELECT statement" in withQbeastContextSparkAndTmpWarehouse( (spark, _) => { import spark.implicits._ @@ -68,6 +68,12 @@ class QbeastSQLIntegrationTest extends QbeastIntegrationTestSpec { s"CREATE OR REPLACE TABLE student USING qbeast " + "OPTIONS ('columnsToIndex'='id') " + "AS SELECT * FROM data;") + + val indexed = spark.table("student") + + indexed.count() shouldBe data.count() + + indexed.columns.toSet shouldBe data.columns.toSet }) it should "work with LOCATION" in withQbeastContextSparkAndTmpDir((spark, tmpDir) => { @@ -81,6 +87,12 @@ class QbeastSQLIntegrationTest extends QbeastIntegrationTestSpec { "OPTIONS ('columnsToIndex'='id') " + s"LOCATION '$tmpDir' " + "AS SELECT * FROM data;") + + val indexed = spark.read.format("qbeast").load(tmpDir) + + indexed.count() shouldBe data.count() + + indexed.columns.toSet shouldBe data.columns.toSet }) } From e7fb0dd54b351ef17097de86788956396589f16b Mon Sep 17 00:00:00 2001 From: osopardo1 Date: Thu, 4 Aug 2022 15:12:11 +0200 Subject: [PATCH 06/49] Add headers --- .../scala/io/qbeast/spark/internal/rules/SaveAsTableRule.scala | 3 +++ .../qbeast/spark/internal/sources/catalog/QbeastCatalog.scala | 3 +++ .../io/qbeast/spark/internal/sources/v2/QbeastScanRDD.scala | 3 +++ .../io/qbeast/spark/internal/sources/v2/QbeastTableImpl.scala | 3 +++ .../qbeast/spark/internal/sources/v2/QbeastWriteBuilder.scala | 3 +++ src/main/scala/org/apache/spark/sql/V2AndV1Traits.scala | 3 +++ 6 files changed, 18 insertions(+) diff --git a/src/main/scala/io/qbeast/spark/internal/rules/SaveAsTableRule.scala b/src/main/scala/io/qbeast/spark/internal/rules/SaveAsTableRule.scala index ebdb06655..c2ab35809 100644 --- a/src/main/scala/io/qbeast/spark/internal/rules/SaveAsTableRule.scala +++ b/src/main/scala/io/qbeast/spark/internal/rules/SaveAsTableRule.scala @@ -1,3 +1,6 @@ +/* + * Copyright 2021 Qbeast Analytics, S.L. + */ package io.qbeast.spark.internal.rules import org.apache.spark.internal.Logging diff --git a/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalog.scala b/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalog.scala index 3b60054fd..c0172bda9 100644 --- a/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalog.scala +++ b/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalog.scala @@ -1,3 +1,6 @@ +/* + * Copyright 2021 Qbeast Analytics, S.L. + */ package io.qbeast.spark.internal.sources.catalog import io.qbeast.context.QbeastContext diff --git a/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastScanRDD.scala b/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastScanRDD.scala index 5470f9e2f..fa77a49b8 100644 --- a/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastScanRDD.scala +++ b/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastScanRDD.scala @@ -1,3 +1,6 @@ +/* + * Copyright 2021 Qbeast Analytics, S.L. + */ package io.qbeast.spark.internal.sources.v2 import io.qbeast.context.QbeastContext._ diff --git a/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastTableImpl.scala b/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastTableImpl.scala index bcf1d86a6..a52f4f285 100644 --- a/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastTableImpl.scala +++ b/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastTableImpl.scala @@ -1,3 +1,6 @@ +/* + * Copyright 2021 Qbeast Analytics, S.L. + */ package io.qbeast.spark.internal.sources.v2 import io.qbeast.context.QbeastContext._ diff --git a/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastWriteBuilder.scala b/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastWriteBuilder.scala index 8ad21c794..2dd5425e4 100644 --- a/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastWriteBuilder.scala +++ b/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastWriteBuilder.scala @@ -1,3 +1,6 @@ +/* + * Copyright 2021 Qbeast Analytics, S.L. + */ package io.qbeast.spark.internal.sources.v2 import io.qbeast.spark.table.IndexedTable diff --git a/src/main/scala/org/apache/spark/sql/V2AndV1Traits.scala b/src/main/scala/org/apache/spark/sql/V2AndV1Traits.scala index 0fe0b9815..56a47e7c5 100644 --- a/src/main/scala/org/apache/spark/sql/V2AndV1Traits.scala +++ b/src/main/scala/org/apache/spark/sql/V2AndV1Traits.scala @@ -1,3 +1,6 @@ +/* + * Copyright 2021 Qbeast Analytics, S.L. + */ package org.apache.spark.sql import org.apache.spark.sql.connector.catalog.{Table, V1Table} From c73d4b5c49356c38ab80078bd86ed64212a8cc49 Mon Sep 17 00:00:00 2001 From: osopardo1 Date: Thu, 4 Aug 2022 15:30:28 +0200 Subject: [PATCH 07/49] Ammend some commints and create new test class for QbeastSparkIntegration --- src/main/scala/io/qbeast/spark/internal/QbeastOptions.scala | 1 - .../qbeast/spark/internal/sources/QbeastBaseRelation.scala | 2 +- ...ntegrationTest.scala => QbeastSparkIntegrationTest.scala} | 5 +---- 3 files changed, 2 insertions(+), 6 deletions(-) rename src/test/scala/io/qbeast/spark/utils/{QbeastDataSourceIntegrationTest.scala => QbeastSparkIntegrationTest.scala} (91%) diff --git a/src/main/scala/io/qbeast/spark/internal/QbeastOptions.scala b/src/main/scala/io/qbeast/spark/internal/QbeastOptions.scala index f7b5c33cf..62e514b74 100644 --- a/src/main/scala/io/qbeast/spark/internal/QbeastOptions.scala +++ b/src/main/scala/io/qbeast/spark/internal/QbeastOptions.scala @@ -19,7 +19,6 @@ case class QbeastOptions(columnsToIndex: Seq[String], cubeSize: Int) * Options available when trying to write in qbeast format */ object QbeastOptions { - val COLUMNS_TO_INDEX = "columnsToIndex" val CUBE_SIZE = "cubeSize" val PATH = "path" diff --git a/src/main/scala/io/qbeast/spark/internal/sources/QbeastBaseRelation.scala b/src/main/scala/io/qbeast/spark/internal/sources/QbeastBaseRelation.scala index 11e1751df..ab3f206c3 100644 --- a/src/main/scala/io/qbeast/spark/internal/sources/QbeastBaseRelation.scala +++ b/src/main/scala/io/qbeast/spark/internal/sources/QbeastBaseRelation.scala @@ -48,7 +48,7 @@ object QbeastBaseRelation { val columnsToIndex = revision.columnTransformers.map(row => row.columnName).mkString(",") val cubeSize = revision.desiredCubeSize val parameters = - Map[String, String]("columnsToIndex" -> columnsToIndex, "cubeSize" -> cubeSize.toString) + Map[String, String]("columnsToIndex" -> columnsToIndex, "cubeSize" -> cubeSize.toString()) val path = new Path(tableID.id) val fileIndex = OTreeIndex(spark, path) diff --git a/src/test/scala/io/qbeast/spark/utils/QbeastDataSourceIntegrationTest.scala b/src/test/scala/io/qbeast/spark/utils/QbeastSparkIntegrationTest.scala similarity index 91% rename from src/test/scala/io/qbeast/spark/utils/QbeastDataSourceIntegrationTest.scala rename to src/test/scala/io/qbeast/spark/utils/QbeastSparkIntegrationTest.scala index 27fb68ad6..4b72b8432 100644 --- a/src/test/scala/io/qbeast/spark/utils/QbeastDataSourceIntegrationTest.scala +++ b/src/test/scala/io/qbeast/spark/utils/QbeastSparkIntegrationTest.scala @@ -1,12 +1,9 @@ -/* - * Copyright 2021 Qbeast Analytics, S.L. - */ package io.qbeast.spark.utils import io.qbeast.spark.QbeastIntegrationTestSpec import org.apache.spark.sql.{DataFrame, SparkSession} -class QbeastDataSourceIntegrationTest extends QbeastIntegrationTestSpec { +class QbeastSparkIntegrationTest extends QbeastIntegrationTestSpec { def createTestData(spark: SparkSession): DataFrame = { import spark.implicits._ From 74f70de1666d6ecd76ca6ccec507f55538e19bba Mon Sep 17 00:00:00 2001 From: osopardo1 Date: Thu, 4 Aug 2022 15:33:40 +0200 Subject: [PATCH 08/49] Moved Correctness tests to new class --- ...stIntegrationTest.scala => QbeastSparkCorrectnessTest.scala} | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) rename src/test/scala/io/qbeast/spark/utils/{QbeastIntegrationTest.scala => QbeastSparkCorrectnessTest.scala} (98%) diff --git a/src/test/scala/io/qbeast/spark/utils/QbeastIntegrationTest.scala b/src/test/scala/io/qbeast/spark/utils/QbeastSparkCorrectnessTest.scala similarity index 98% rename from src/test/scala/io/qbeast/spark/utils/QbeastIntegrationTest.scala rename to src/test/scala/io/qbeast/spark/utils/QbeastSparkCorrectnessTest.scala index 2108ec67e..39e5bc76d 100644 --- a/src/test/scala/io/qbeast/spark/utils/QbeastIntegrationTest.scala +++ b/src/test/scala/io/qbeast/spark/utils/QbeastSparkCorrectnessTest.scala @@ -5,7 +5,7 @@ import io.qbeast.spark.delta.DeltaQbeastSnapshot import org.apache.spark.sql.delta.DeltaLog import org.apache.spark.sql.functions.{col, lit} -class QbeastIntegrationTest extends QbeastIntegrationTestSpec { +class QbeastSparkCorrectnessTest extends QbeastIntegrationTestSpec { "Qbeast datasource" should "expose the original number of columns and rows" in withQbeastContextSparkAndTmpDir { From f6188fff39d5727c21580d6f311fceed471ba4bc Mon Sep 17 00:00:00 2001 From: osopardo1 Date: Fri, 5 Aug 2022 09:32:15 +0200 Subject: [PATCH 09/49] Extend DeltaCatalog and modify buildScan() method --- ...Catalog.scala => QbeastDeltaCatalog.scala} | 7 ++- .../internal/sources/v2/QbeastScanRDD.scala | 55 ++++++++----------- .../spark/QbeastIntegrationTestSpec.scala | 4 +- 3 files changed, 29 insertions(+), 37 deletions(-) rename src/main/scala/io/qbeast/spark/internal/sources/catalog/{QbeastCatalog.scala => QbeastDeltaCatalog.scala} (94%) diff --git a/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalog.scala b/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastDeltaCatalog.scala similarity index 94% rename from src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalog.scala rename to src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastDeltaCatalog.scala index c0172bda9..ec8ccacad 100644 --- a/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalog.scala +++ b/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastDeltaCatalog.scala @@ -7,16 +7,17 @@ import io.qbeast.context.QbeastContext import io.qbeast.spark.internal.QbeastOptions.checkQbeastProperties import io.qbeast.spark.internal.sources.v2.QbeastTableImpl import org.apache.hadoop.fs.Path -import org.apache.spark.sql.{AnalysisExceptionFactory, V1TableQbeast} import org.apache.spark.sql.catalyst.catalog.CatalogTableType -import org.apache.spark.sql.connector.catalog.{DelegatingCatalogExtension, Identifier, Table} +import org.apache.spark.sql.connector.catalog.{Identifier, Table} import org.apache.spark.sql.connector.expressions.Transform +import org.apache.spark.sql.delta.catalog.DeltaCatalog import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.{AnalysisExceptionFactory, V1TableQbeast} import java.util import scala.collection.JavaConverters.mapAsScalaMapConverter -class QbeastCatalog extends DelegatingCatalogExtension { +class QbeastDeltaCatalog extends DeltaCatalog { private val tableFactory = QbeastContext.indexedTableFactory diff --git a/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastScanRDD.scala b/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastScanRDD.scala index fa77a49b8..b5962ef02 100644 --- a/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastScanRDD.scala +++ b/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastScanRDD.scala @@ -7,16 +7,13 @@ import io.qbeast.context.QbeastContext._ import io.qbeast.spark.internal.sources.QbeastBaseRelation import io.qbeast.spark.table.IndexedTable import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalyst.encoders.RowEncoder +import org.apache.spark.sql.catalyst.encoders.{RowEncoder} import org.apache.spark.sql.{Row, SQLContext} import org.apache.spark.sql.connector.read.V1Scan -import org.apache.spark.sql.execution.FileSourceScanExec -import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation} +import org.apache.spark.sql.execution.datasources.HadoopFsRelation import org.apache.spark.sql.sources.{BaseRelation, TableScan} import org.apache.spark.sql.types.StructType -import scala.collection.JavaConverters.asScalaIteratorConverter - /** * Extends Scan Builder for V1 DataSource * It allows Spark to read from a Qbeast Formatted table @@ -37,37 +34,31 @@ class QbeastScanRDD(indexedTable: IndexedTable) extends V1Scan { override def schema: StructType = qbeastBaseRelation.schema override def buildScan(): RDD[Row] = { - val schema = qbeastBaseRelation.schema - // Ugly hack to convert the schema fields into Attributes - val output = LogicalRelation - .apply(qbeastBaseRelation) - .resolve(schema, context.sparkSession.sessionState.analyzer.resolver) + val rootPath = indexedTable.tableID.id + + // Get the files to load from the Relation index + val filesToLoad = qbeastBaseRelation.location.listFiles(Seq.empty, Seq.empty) + // Map the paths of the files with the rootPath + val pathsToLoad = + filesToLoad.flatMap(_.files.map(f => { + val path = f.getPath + if (path.isAbsolute) path.toString else rootPath + "/" + path.toString + })) - // TODO check the parameters passed to this method - FileSourceScanExec( - relation = qbeastBaseRelation, - output = output, - requiredSchema = schema, - partitionFilters = Seq.empty, - optionalBucketSet = None, - optionalNumCoalescedBuckets = None, - dataFilters = Seq.empty, - tableIdentifier = None) - .executeColumnar() - .mapPartitions { batches => - batches.flatMap { batch => - batch - .rowIterator() - .asScala - .map(internal => { - val encoder = RowEncoder(schema).resolveAndBind() - encoder.createDeserializer().apply(internal) - }) - } - } + // We output the plan to build scan information + // This is a hack, and the Scan should be done + // with more Qbeast logic + val df = context.sparkSession.read.format("parquet").load(pathsToLoad: _*) + val encoder = RowEncoder(schema).resolveAndBind() + val deserializer = encoder.createDeserializer() + + df.queryExecution.executedPlan.execute().mapPartitions { batches => + batches.map(deserializer.apply) + } } }.asInstanceOf[T] + } override def readSchema(): StructType = metadataManager.loadCurrentSchema(indexedTable.tableID) diff --git a/src/test/scala/io/qbeast/spark/QbeastIntegrationTestSpec.scala b/src/test/scala/io/qbeast/spark/QbeastIntegrationTestSpec.scala index 45fbac587..fadb049e8 100644 --- a/src/test/scala/io/qbeast/spark/QbeastIntegrationTestSpec.scala +++ b/src/test/scala/io/qbeast/spark/QbeastIntegrationTestSpec.scala @@ -71,7 +71,7 @@ trait QbeastIntegrationTestSpec extends AnyFlatSpec with Matchers with DatasetCo .set("spark.sql.extensions", "io.qbeast.spark.internal.QbeastSparkSessionExtension") .set( SQLConf.V2_SESSION_CATALOG_IMPLEMENTATION.key, - "io.qbeast.spark.internal.sources.catalog.QbeastCatalog") + "io.qbeast.spark.internal.sources.catalog.QbeastDeltaCatalog") val spark = SparkSession .builder() .appName("QbeastDataSource") @@ -98,7 +98,7 @@ trait QbeastIntegrationTestSpec extends AnyFlatSpec with Matchers with DatasetCo .set("spark.sql.extensions", "io.qbeast.spark.internal.QbeastSparkSessionExtension") .set( SQLConf.V2_SESSION_CATALOG_IMPLEMENTATION.key, - "io.qbeast.spark.internal.sources.catalog.QbeastCatalog") + "io.qbeast.spark.internal.sources.catalog.QbeastDeltaCatalog") val spark = SparkSession .builder() From 35b7143ae3ff0918a9a2bb26c7aabbc0c48c44db Mon Sep 17 00:00:00 2001 From: osopardo1 Date: Fri, 5 Aug 2022 09:48:04 +0200 Subject: [PATCH 10/49] Rollback QbeastCatalog --- .../{QbeastDeltaCatalog.scala => QbeastCatalog.scala} | 7 +++---- .../scala/io/qbeast/spark/QbeastIntegrationTestSpec.scala | 4 ++-- 2 files changed, 5 insertions(+), 6 deletions(-) rename src/main/scala/io/qbeast/spark/internal/sources/catalog/{QbeastDeltaCatalog.scala => QbeastCatalog.scala} (94%) diff --git a/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastDeltaCatalog.scala b/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalog.scala similarity index 94% rename from src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastDeltaCatalog.scala rename to src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalog.scala index ec8ccacad..c0172bda9 100644 --- a/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastDeltaCatalog.scala +++ b/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalog.scala @@ -7,17 +7,16 @@ import io.qbeast.context.QbeastContext import io.qbeast.spark.internal.QbeastOptions.checkQbeastProperties import io.qbeast.spark.internal.sources.v2.QbeastTableImpl import org.apache.hadoop.fs.Path +import org.apache.spark.sql.{AnalysisExceptionFactory, V1TableQbeast} import org.apache.spark.sql.catalyst.catalog.CatalogTableType -import org.apache.spark.sql.connector.catalog.{Identifier, Table} +import org.apache.spark.sql.connector.catalog.{DelegatingCatalogExtension, Identifier, Table} import org.apache.spark.sql.connector.expressions.Transform -import org.apache.spark.sql.delta.catalog.DeltaCatalog import org.apache.spark.sql.types.StructType -import org.apache.spark.sql.{AnalysisExceptionFactory, V1TableQbeast} import java.util import scala.collection.JavaConverters.mapAsScalaMapConverter -class QbeastDeltaCatalog extends DeltaCatalog { +class QbeastCatalog extends DelegatingCatalogExtension { private val tableFactory = QbeastContext.indexedTableFactory diff --git a/src/test/scala/io/qbeast/spark/QbeastIntegrationTestSpec.scala b/src/test/scala/io/qbeast/spark/QbeastIntegrationTestSpec.scala index fadb049e8..45fbac587 100644 --- a/src/test/scala/io/qbeast/spark/QbeastIntegrationTestSpec.scala +++ b/src/test/scala/io/qbeast/spark/QbeastIntegrationTestSpec.scala @@ -71,7 +71,7 @@ trait QbeastIntegrationTestSpec extends AnyFlatSpec with Matchers with DatasetCo .set("spark.sql.extensions", "io.qbeast.spark.internal.QbeastSparkSessionExtension") .set( SQLConf.V2_SESSION_CATALOG_IMPLEMENTATION.key, - "io.qbeast.spark.internal.sources.catalog.QbeastDeltaCatalog") + "io.qbeast.spark.internal.sources.catalog.QbeastCatalog") val spark = SparkSession .builder() .appName("QbeastDataSource") @@ -98,7 +98,7 @@ trait QbeastIntegrationTestSpec extends AnyFlatSpec with Matchers with DatasetCo .set("spark.sql.extensions", "io.qbeast.spark.internal.QbeastSparkSessionExtension") .set( SQLConf.V2_SESSION_CATALOG_IMPLEMENTATION.key, - "io.qbeast.spark.internal.sources.catalog.QbeastDeltaCatalog") + "io.qbeast.spark.internal.sources.catalog.QbeastCatalog") val spark = SparkSession .builder() From c49d20d3dd3084f5151557969abb8085b185c6b8 Mon Sep 17 00:00:00 2001 From: osopardo1 Date: Fri, 5 Aug 2022 09:55:23 +0200 Subject: [PATCH 11/49] Add select statement to SQLIntegrationTest --- .../io/qbeast/spark/utils/QbeastSQLIntegrationTest.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/test/scala/io/qbeast/spark/utils/QbeastSQLIntegrationTest.scala b/src/test/scala/io/qbeast/spark/utils/QbeastSQLIntegrationTest.scala index 6565955e4..4c2f1239d 100644 --- a/src/test/scala/io/qbeast/spark/utils/QbeastSQLIntegrationTest.scala +++ b/src/test/scala/io/qbeast/spark/utils/QbeastSQLIntegrationTest.scala @@ -50,7 +50,7 @@ class QbeastSQLIntegrationTest extends QbeastIntegrationTestSpec { spark.sql("INSERT INTO table student SELECT * FROM data") - val indexed = spark.table("student") + val indexed = spark.sql("SELECT * FROM student") indexed.count() shouldBe data.count() @@ -69,7 +69,7 @@ class QbeastSQLIntegrationTest extends QbeastIntegrationTestSpec { "OPTIONS ('columnsToIndex'='id') " + "AS SELECT * FROM data;") - val indexed = spark.table("student") + val indexed = spark.sql("SELECT * FROM student") indexed.count() shouldBe data.count() @@ -88,7 +88,7 @@ class QbeastSQLIntegrationTest extends QbeastIntegrationTestSpec { s"LOCATION '$tmpDir' " + "AS SELECT * FROM data;") - val indexed = spark.read.format("qbeast").load(tmpDir) + val indexed = spark.sql("SELECT * FROM student") indexed.count() shouldBe data.count() From 983e5ce38df45bd01c5df79878a7c9c5fdf33182 Mon Sep 17 00:00:00 2001 From: osopardo1 Date: Fri, 5 Aug 2022 09:57:20 +0200 Subject: [PATCH 12/49] Fix Table Properties --- .../scala/io/qbeast/spark/utils/QbeastSQLIntegrationTest.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/test/scala/io/qbeast/spark/utils/QbeastSQLIntegrationTest.scala b/src/test/scala/io/qbeast/spark/utils/QbeastSQLIntegrationTest.scala index 4c2f1239d..175f1427f 100644 --- a/src/test/scala/io/qbeast/spark/utils/QbeastSQLIntegrationTest.scala +++ b/src/test/scala/io/qbeast/spark/utils/QbeastSQLIntegrationTest.scala @@ -35,7 +35,7 @@ class QbeastSQLIntegrationTest extends QbeastIntegrationTestSpec { .where("col_name == 'Table Properties'") .select("data_type") .first() - .getString(0) should contain("columnsToIndex=id") + .getString(0) shouldBe "[columnsToIndex=id,option.columnsToIndex=id]" }) From 275eca1c53836dd67ce967f1ad363ea6dba5cd40 Mon Sep 17 00:00:00 2001 From: osopardo1 Date: Fri, 5 Aug 2022 10:00:42 +0200 Subject: [PATCH 13/49] Return rdd in buildScan() --- .../qbeast/spark/internal/sources/v2/QbeastScanRDD.scala | 9 +++------ 1 file changed, 3 insertions(+), 6 deletions(-) diff --git a/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastScanRDD.scala b/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastScanRDD.scala index b5962ef02..774818106 100644 --- a/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastScanRDD.scala +++ b/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastScanRDD.scala @@ -7,7 +7,6 @@ import io.qbeast.context.QbeastContext._ import io.qbeast.spark.internal.sources.QbeastBaseRelation import io.qbeast.spark.table.IndexedTable import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalyst.encoders.{RowEncoder} import org.apache.spark.sql.{Row, SQLContext} import org.apache.spark.sql.connector.read.V1Scan import org.apache.spark.sql.execution.datasources.HadoopFsRelation @@ -50,13 +49,11 @@ class QbeastScanRDD(indexedTable: IndexedTable) extends V1Scan { // This is a hack, and the Scan should be done // with more Qbeast logic val df = context.sparkSession.read.format("parquet").load(pathsToLoad: _*) - val encoder = RowEncoder(schema).resolveAndBind() - val deserializer = encoder.createDeserializer() - df.queryExecution.executedPlan.execute().mapPartitions { batches => - batches.map(deserializer.apply) - } + // Return the RDD + df.rdd } + }.asInstanceOf[T] } From 569f96f702d3a93005760465667d5aba77357ac3 Mon Sep 17 00:00:00 2001 From: osopardo1 Date: Fri, 5 Aug 2022 14:37:37 +0200 Subject: [PATCH 14/49] Add new spark rule for fallback reads to V1 implementation --- .../QbeastSparkSessionExtension.scala | 6 ++++- .../spark/internal/rules/QbeastAnalysis.scala | 27 +++++++++++++++++++ .../internal/sources/QbeastDataSource.scala | 26 +++++++++++++++--- .../sources/catalog/QbeastCatalog.scala | 13 ++++++--- .../internal/sources/v2/QbeastScanRDD.scala | 24 ++++------------- .../internal/sources/v2/QbeastTableImpl.scala | 23 +++++++--------- .../org/apache/spark/sql/V2AndV1Traits.scala | 4 ++- 7 files changed, 80 insertions(+), 43 deletions(-) create mode 100644 src/main/scala/io/qbeast/spark/internal/rules/QbeastAnalysis.scala diff --git a/src/main/scala/io/qbeast/spark/internal/QbeastSparkSessionExtension.scala b/src/main/scala/io/qbeast/spark/internal/QbeastSparkSessionExtension.scala index caa37eb7d..44f907080 100644 --- a/src/main/scala/io/qbeast/spark/internal/QbeastSparkSessionExtension.scala +++ b/src/main/scala/io/qbeast/spark/internal/QbeastSparkSessionExtension.scala @@ -4,7 +4,7 @@ package io.qbeast.spark.internal import io.delta.sql.DeltaSparkSessionExtension -import io.qbeast.spark.internal.rules.{SampleRule, SaveAsTableRule} +import io.qbeast.spark.internal.rules.{QbeastAnalysis, SampleRule, SaveAsTableRule} import org.apache.spark.sql.SparkSessionExtensions /** @@ -16,6 +16,10 @@ class QbeastSparkSessionExtension extends DeltaSparkSessionExtension { super.apply(extensions) + extensions.injectResolutionRule { session => + new QbeastAnalysis(session) + } + extensions.injectOptimizerRule { session => new SampleRule(session) } diff --git a/src/main/scala/io/qbeast/spark/internal/rules/QbeastAnalysis.scala b/src/main/scala/io/qbeast/spark/internal/rules/QbeastAnalysis.scala new file mode 100644 index 000000000..a96ddec9c --- /dev/null +++ b/src/main/scala/io/qbeast/spark/internal/rules/QbeastAnalysis.scala @@ -0,0 +1,27 @@ +package io.qbeast.spark.internal.rules + +import io.qbeast.spark.internal.sources.v2.QbeastTableImpl +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution.datasources.LogicalRelation +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation + +class QbeastAnalysis(spark: SparkSession) extends Rule[LogicalPlan] { + + private def toV1Relation( + dataSourceV2Relation: DataSourceV2Relation, + table: QbeastTableImpl): LogicalRelation = { + + val underlyingRelation = table.toBaseRelation + LogicalRelation(underlyingRelation, dataSourceV2Relation.output, None, isStreaming = false) + + } + + override def apply(plan: LogicalPlan): LogicalPlan = plan transformDown { + // This rule falls back to V1 nodes, since we don't have a V2 reader for Delta right now + case v2Relation @ DataSourceV2Relation(t: QbeastTableImpl, _, _, _, _) => + toV1Relation(v2Relation, t) + } + +} diff --git a/src/main/scala/io/qbeast/spark/internal/sources/QbeastDataSource.scala b/src/main/scala/io/qbeast/spark/internal/sources/QbeastDataSource.scala index f65672d92..453b8f756 100644 --- a/src/main/scala/io/qbeast/spark/internal/sources/QbeastDataSource.scala +++ b/src/main/scala/io/qbeast/spark/internal/sources/QbeastDataSource.scala @@ -4,6 +4,7 @@ package io.qbeast.spark.internal.sources import io.qbeast.context.QbeastContext +import io.qbeast.context.QbeastContext.metadataManager import io.qbeast.spark.internal.QbeastOptions import io.qbeast.spark.internal.QbeastOptions.checkQbeastProperties import io.qbeast.spark.internal.sources.v2.QbeastTableImpl @@ -34,10 +35,10 @@ import scala.collection.JavaConverters.mapAsScalaMapConverter * Qbeast data source is implementation of Spark data source API V1. */ class QbeastDataSource private[sources] (private val tableFactory: IndexedTableFactory) - extends TableProvider + extends RelationProvider with CreatableRelationProvider with DataSourceRegister - with RelationProvider { + with TableProvider { /** * Constructor to be used by Spark. @@ -54,7 +55,23 @@ class QbeastDataSource private[sources] (private val tableFactory: IndexedTableF partitioning: Array[Transform], properties: util.Map[String, String]): Table = { val tableId = QbeastOptions.loadTableIDFromParameters(properties.asScala.toMap) - new QbeastTableImpl(new Path(tableId.id), properties.asScala.toMap, None, tableFactory) + val indexedTable = tableFactory.getIndexedTable(tableId) + if (indexedTable.exists) { + val currentRevision = metadataManager.loadSnapshot(tableId).loadLatestRevision + val indexProperties = Map( + "columnsToIndex" -> currentRevision.columnTransformers.map(_.columnName).mkString(","), + "cubeSize" -> currentRevision.desiredCubeSize.toString) + val tableProperties = properties.asScala.toMap ++ indexProperties + new QbeastTableImpl(new Path(tableId.id), tableProperties, None, None, tableFactory) + } else { + new QbeastTableImpl( + new Path(tableId.id), + properties.asScala.toMap, + Some(schema), + None, + tableFactory) + } + } def inferSchema( @@ -97,7 +114,8 @@ class QbeastDataSource private[sources] (private val tableFactory: IndexedTableF else { // If indexedTable does not contain data // Check if it's registered on the catalog - val tableImpl = new QbeastTableImpl(new Path(tableID.id), parameters, None, tableFactory) + val tableImpl = + new QbeastTableImpl(new Path(tableID.id), parameters, None, None, tableFactory) if (tableImpl.isCatalogTable) { tableImpl.toBaseRelation } else { throw AnalysisExceptionFactory.create( diff --git a/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalog.scala b/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalog.scala index c0172bda9..57fa01306 100644 --- a/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalog.scala +++ b/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalog.scala @@ -8,7 +8,7 @@ import io.qbeast.spark.internal.QbeastOptions.checkQbeastProperties import io.qbeast.spark.internal.sources.v2.QbeastTableImpl import org.apache.hadoop.fs.Path import org.apache.spark.sql.{AnalysisExceptionFactory, V1TableQbeast} -import org.apache.spark.sql.catalyst.catalog.CatalogTableType +import org.apache.spark.sql.catalyst.catalog.{CatalogTableType} import org.apache.spark.sql.connector.catalog.{DelegatingCatalogExtension, Identifier, Table} import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.types.StructType @@ -44,9 +44,9 @@ class QbeastCatalog extends DelegatingCatalogExtension { if (isQbeastTableProvider(prop)) { table match { - case V1TableQbeast(v1Table) => + case V1TableQbeast(t) => checkQbeastProperties(prop) - val catalogTable = v1Table.v1Table + val catalogTable = t.v1Table val path: String = if (catalogTable.tableType == CatalogTableType.EXTERNAL) { // If it's an EXTERNAL TABLE, we can find the path through the Storage Properties @@ -58,7 +58,12 @@ class QbeastCatalog extends DelegatingCatalogExtension { // Otherwise, TODO throw AnalysisExceptionFactory.create("No path found for table " + table.name()) } - new QbeastTableImpl(new Path(path), prop, Some(schema), tableFactory) + new QbeastTableImpl( + new Path(path), + prop, + Some(schema), + Some(catalogTable), + tableFactory) case _ => table } diff --git a/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastScanRDD.scala b/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastScanRDD.scala index 774818106..1c3296c50 100644 --- a/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastScanRDD.scala +++ b/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastScanRDD.scala @@ -3,13 +3,11 @@ */ package io.qbeast.spark.internal.sources.v2 -import io.qbeast.context.QbeastContext._ import io.qbeast.spark.internal.sources.QbeastBaseRelation import io.qbeast.spark.table.IndexedTable import org.apache.spark.rdd.RDD import org.apache.spark.sql.{Row, SQLContext} import org.apache.spark.sql.connector.read.V1Scan -import org.apache.spark.sql.execution.datasources.HadoopFsRelation import org.apache.spark.sql.sources.{BaseRelation, TableScan} import org.apache.spark.sql.types.StructType @@ -22,7 +20,7 @@ import org.apache.spark.sql.types.StructType class QbeastScanRDD(indexedTable: IndexedTable) extends V1Scan { private lazy val qbeastBaseRelation = - QbeastBaseRelation.forQbeastTable(indexedTable).asInstanceOf[HadoopFsRelation] + QbeastBaseRelation.forQbeastTable(indexedTable) override def toV1TableScan[T <: BaseRelation with TableScan](context: SQLContext): T = { @@ -34,21 +32,9 @@ class QbeastScanRDD(indexedTable: IndexedTable) extends V1Scan { override def buildScan(): RDD[Row] = { - val rootPath = indexedTable.tableID.id - - // Get the files to load from the Relation index - val filesToLoad = qbeastBaseRelation.location.listFiles(Seq.empty, Seq.empty) - // Map the paths of the files with the rootPath - val pathsToLoad = - filesToLoad.flatMap(_.files.map(f => { - val path = f.getPath - if (path.isAbsolute) path.toString else rootPath + "/" + path.toString - })) - - // We output the plan to build scan information - // This is a hack, and the Scan should be done - // with more Qbeast logic - val df = context.sparkSession.read.format("parquet").load(pathsToLoad: _*) + // We output the rdd from the DataFrame scan + // This is a hack, and the implementation should add more Qbeast logic + val df = context.sparkSession.baseRelationToDataFrame(qbeastBaseRelation) // Return the RDD df.rdd @@ -58,6 +44,6 @@ class QbeastScanRDD(indexedTable: IndexedTable) extends V1Scan { } - override def readSchema(): StructType = metadataManager.loadCurrentSchema(indexedTable.tableID) + override def readSchema(): StructType = qbeastBaseRelation.schema } diff --git a/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastTableImpl.scala b/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastTableImpl.scala index a52f4f285..683112ac2 100644 --- a/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastTableImpl.scala +++ b/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastTableImpl.scala @@ -9,18 +9,12 @@ import io.qbeast.spark.internal.sources.QbeastBaseRelation import org.apache.spark.sql.connector.catalog.TableCapability._ import io.qbeast.spark.table.IndexedTableFactory import org.apache.hadoop.fs.Path -import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.connector.catalog.{ - SupportsRead, - SupportsWrite, - Table, - TableCapability -} -import org.apache.spark.sql.connector.read.{Scan, ScanBuilder} +import org.apache.spark.sql.catalyst.catalog.CatalogTable +import org.apache.spark.sql.{AnalysisExceptionFactory, SparkSession, V2toV1Fallback} +import org.apache.spark.sql.connector.catalog.{SupportsWrite, Table, TableCapability} import org.apache.spark.sql.connector.write.{LogicalWriteInfo, WriteBuilder} import org.apache.spark.sql.sources.BaseRelation import org.apache.spark.sql.types.StructType -import org.apache.spark.sql.util.CaseInsensitiveStringMap import java.util import scala.collection.JavaConverters._ @@ -29,10 +23,11 @@ class QbeastTableImpl private[sources] ( path: Path, options: Map[String, String], schema: Option[StructType] = None, + catalogTable: Option[CatalogTable] = None, private val tableFactory: IndexedTableFactory) extends Table with SupportsWrite - with SupportsRead { + with V2toV1Fallback { private val spark = SparkSession.active @@ -74,9 +69,9 @@ class QbeastTableImpl private[sources] ( override def properties(): util.Map[String, String] = options.asJava - override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder = - new ScanBuilder { - override def build(): Scan = new QbeastScanRDD(indexedTable) - } + override def v1Table: CatalogTable = { + if (catalogTable.isDefined) catalogTable.get + else throw AnalysisExceptionFactory.create("no catalog table defined") + } } diff --git a/src/main/scala/org/apache/spark/sql/V2AndV1Traits.scala b/src/main/scala/org/apache/spark/sql/V2AndV1Traits.scala index 56a47e7c5..458d46908 100644 --- a/src/main/scala/org/apache/spark/sql/V2AndV1Traits.scala +++ b/src/main/scala/org/apache/spark/sql/V2AndV1Traits.scala @@ -3,7 +3,9 @@ */ package org.apache.spark.sql -import org.apache.spark.sql.connector.catalog.{Table, V1Table} +import org.apache.spark.sql.connector.catalog.{Table, V1Table, V2TableWithV1Fallback} + +trait V2toV1Fallback extends V2TableWithV1Fallback {} object V1TableQbeast { From 6f29bdd47163c520686f26a9bfd1a7c43935f406 Mon Sep 17 00:00:00 2001 From: osopardo1 Date: Fri, 5 Aug 2022 14:38:37 +0200 Subject: [PATCH 15/49] Create headers --- .../scala/io/qbeast/spark/internal/rules/QbeastAnalysis.scala | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/main/scala/io/qbeast/spark/internal/rules/QbeastAnalysis.scala b/src/main/scala/io/qbeast/spark/internal/rules/QbeastAnalysis.scala index a96ddec9c..325825f07 100644 --- a/src/main/scala/io/qbeast/spark/internal/rules/QbeastAnalysis.scala +++ b/src/main/scala/io/qbeast/spark/internal/rules/QbeastAnalysis.scala @@ -1,3 +1,6 @@ +/* + * Copyright 2021 Qbeast Analytics, S.L. + */ package io.qbeast.spark.internal.rules import io.qbeast.spark.internal.sources.v2.QbeastTableImpl From 1a82a17f9c6a3ee614cbe626af81318750b701c9 Mon Sep 17 00:00:00 2001 From: osopardo1 Date: Fri, 5 Aug 2022 14:48:53 +0200 Subject: [PATCH 16/49] Add path to getTable for tests --- .../io/qbeast/spark/internal/sources/v2/QbeastTableImpl.scala | 2 +- .../qbeast/spark/internal/sources/QbeastDataSourceTest.scala | 3 +-- .../scala/io/qbeast/spark/utils/QbeastSQLIntegrationTest.scala | 2 +- 3 files changed, 3 insertions(+), 4 deletions(-) diff --git a/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastTableImpl.scala b/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastTableImpl.scala index 683112ac2..9c9eb73fe 100644 --- a/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastTableImpl.scala +++ b/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastTableImpl.scala @@ -29,7 +29,7 @@ class QbeastTableImpl private[sources] ( with SupportsWrite with V2toV1Fallback { - private val spark = SparkSession.active + private lazy val spark = SparkSession.active private val pathString = path.toString diff --git a/src/test/scala/io/qbeast/spark/internal/sources/QbeastDataSourceTest.scala b/src/test/scala/io/qbeast/spark/internal/sources/QbeastDataSourceTest.scala index 945efdfff..903b329a9 100644 --- a/src/test/scala/io/qbeast/spark/internal/sources/QbeastDataSourceTest.scala +++ b/src/test/scala/io/qbeast/spark/internal/sources/QbeastDataSourceTest.scala @@ -18,7 +18,6 @@ import org.scalatest.flatspec.FixtureAnyFlatSpec import org.scalatest.matchers.should.Matchers import org.scalatestplus.mockito.MockitoSugar -import java.util.Collections import scala.collection.JavaConverters._ /** @@ -67,7 +66,7 @@ class QbeastDataSourceTest extends FixtureAnyFlatSpec with MockitoSugar with Mat it should "return correct table" in { f => val schema = StructType(Seq()) val partitioning = Array.empty[Transform] - val properties = Collections.emptyMap[String, String]() + val properties = Map("path" -> "path").asJava val table = f.dataSource.getTable(schema, partitioning, properties) table.schema() shouldBe schema table.capabilities() shouldBe Set( diff --git a/src/test/scala/io/qbeast/spark/utils/QbeastSQLIntegrationTest.scala b/src/test/scala/io/qbeast/spark/utils/QbeastSQLIntegrationTest.scala index 175f1427f..78ec0baab 100644 --- a/src/test/scala/io/qbeast/spark/utils/QbeastSQLIntegrationTest.scala +++ b/src/test/scala/io/qbeast/spark/utils/QbeastSQLIntegrationTest.scala @@ -9,7 +9,7 @@ class QbeastSQLIntegrationTest extends QbeastIntegrationTestSpec { private val students = 1.to(10).map(i => Student(i, i.toString, Random.nextInt())) - "QbeastSpark" should "work with SQL CREATE TABLE" in withQbeastContextSparkAndTmpWarehouse( + "QbeastSpark SQL" should "work with CREATE TABLE" in withQbeastContextSparkAndTmpWarehouse( (spark, tmpWarehouse) => { import spark.implicits._ val data = students.toDF() From 2cec7107021960084279fd86f5b025ea673e7e76 Mon Sep 17 00:00:00 2001 From: osopardo1 Date: Fri, 5 Aug 2022 15:30:10 +0200 Subject: [PATCH 17/49] Rollback to old createRelation implementation --- .../spark/internal/sources/QbeastDataSource.scala | 11 ++--------- .../spark/internal/sources/QbeastDataSourceTest.scala | 2 +- 2 files changed, 3 insertions(+), 10 deletions(-) diff --git a/src/main/scala/io/qbeast/spark/internal/sources/QbeastDataSource.scala b/src/main/scala/io/qbeast/spark/internal/sources/QbeastDataSource.scala index 453b8f756..b6da36012 100644 --- a/src/main/scala/io/qbeast/spark/internal/sources/QbeastDataSource.scala +++ b/src/main/scala/io/qbeast/spark/internal/sources/QbeastDataSource.scala @@ -112,15 +112,8 @@ class QbeastDataSource private[sources] (private val tableFactory: IndexedTableF // Otherwise, the table can be loaded from the catalog if (indexedTable.exists) indexedTable.load() else { - // If indexedTable does not contain data - // Check if it's registered on the catalog - val tableImpl = - new QbeastTableImpl(new Path(tableID.id), parameters, None, None, tableFactory) - if (tableImpl.isCatalogTable) { tableImpl.toBaseRelation } - else { - throw AnalysisExceptionFactory.create( - s"'$tableID' is not a Qbeast formatted data directory.") - } + throw AnalysisExceptionFactory.create( + s"'$tableID' is not a Qbeast formatted data directory.") } } diff --git a/src/test/scala/io/qbeast/spark/internal/sources/QbeastDataSourceTest.scala b/src/test/scala/io/qbeast/spark/internal/sources/QbeastDataSourceTest.scala index 903b329a9..e4a043d0b 100644 --- a/src/test/scala/io/qbeast/spark/internal/sources/QbeastDataSourceTest.scala +++ b/src/test/scala/io/qbeast/spark/internal/sources/QbeastDataSourceTest.scala @@ -66,7 +66,7 @@ class QbeastDataSourceTest extends FixtureAnyFlatSpec with MockitoSugar with Mat it should "return correct table" in { f => val schema = StructType(Seq()) val partitioning = Array.empty[Transform] - val properties = Map("path" -> "path").asJava + val properties = Map("path" -> path).asJava val table = f.dataSource.getTable(schema, partitioning, properties) table.schema() shouldBe schema table.capabilities() shouldBe Set( From 2772bfeea493f3dc713eed140cacf0c0a5509e37 Mon Sep 17 00:00:00 2001 From: osopardo1 Date: Sat, 6 Aug 2022 08:17:31 +0200 Subject: [PATCH 18/49] Add comments to the code and rename QbeastScanRDD to QbeastScan --- .../spark/internal/rules/QbeastAnalysis.scala | 13 +++++++++++- .../internal/rules/SaveAsTableRule.scala | 5 +++++ .../internal/sources/QbeastBaseRelation.scala | 6 ------ .../internal/sources/QbeastDataSource.scala | 11 +++++----- .../{QbeastScanRDD.scala => QbeastScan.scala} | 13 ++++++++---- .../internal/sources/v2/QbeastTableImpl.scala | 21 ++++++++----------- .../sources/v2/QbeastWriteBuilder.scala | 6 ++++++ 7 files changed, 46 insertions(+), 29 deletions(-) rename src/main/scala/io/qbeast/spark/internal/sources/v2/{QbeastScanRDD.scala => QbeastScan.scala} (73%) diff --git a/src/main/scala/io/qbeast/spark/internal/rules/QbeastAnalysis.scala b/src/main/scala/io/qbeast/spark/internal/rules/QbeastAnalysis.scala index 325825f07..c831648f4 100644 --- a/src/main/scala/io/qbeast/spark/internal/rules/QbeastAnalysis.scala +++ b/src/main/scala/io/qbeast/spark/internal/rules/QbeastAnalysis.scala @@ -10,8 +10,18 @@ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.datasources.LogicalRelation import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation +/** + * Analyzes and resolves the Spark Plan before Optimization + * @param spark the SparkSession + */ class QbeastAnalysis(spark: SparkSession) extends Rule[LogicalPlan] { + /** + * Returns the V1Relation from a V2Relation + * @param dataSourceV2Relation the V2Relation + * @param table the underlying table + * @return the LogicalRelation + */ private def toV1Relation( dataSourceV2Relation: DataSourceV2Relation, table: QbeastTableImpl): LogicalRelation = { @@ -22,7 +32,8 @@ class QbeastAnalysis(spark: SparkSession) extends Rule[LogicalPlan] { } override def apply(plan: LogicalPlan): LogicalPlan = plan transformDown { - // This rule falls back to V1 nodes, since we don't have a V2 reader for Delta right now + // This rule is a hack to return a V1 relation for reading + // Because we didn't implemented SupportsRead on QbeastTableImpl yet case v2Relation @ DataSourceV2Relation(t: QbeastTableImpl, _, _, _, _) => toV1Relation(v2Relation, t) } diff --git a/src/main/scala/io/qbeast/spark/internal/rules/SaveAsTableRule.scala b/src/main/scala/io/qbeast/spark/internal/rules/SaveAsTableRule.scala index c2ab35809..7c2fe8b03 100644 --- a/src/main/scala/io/qbeast/spark/internal/rules/SaveAsTableRule.scala +++ b/src/main/scala/io/qbeast/spark/internal/rules/SaveAsTableRule.scala @@ -8,11 +8,16 @@ import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.plans.logical.{CreateTableAsSelect, LogicalPlan} import org.apache.spark.sql.catalyst.rules.Rule +/** + * Rule class that enforces to pass all the write options to the Table Implementation + * @param spark the SparkSession + */ class SaveAsTableRule(spark: SparkSession) extends Rule[LogicalPlan] with Logging { override def apply(plan: LogicalPlan): LogicalPlan = { // When CreateTableAsSelect statement is in place for qbeast // We need to pass the writeOptions as properties to the creation of the table + // to make sure columnsToIndex is present plan transformDown { case saveAsSelect: CreateTableAsSelect if saveAsSelect.properties.get("provider").contains("qbeast") => diff --git a/src/main/scala/io/qbeast/spark/internal/sources/QbeastBaseRelation.scala b/src/main/scala/io/qbeast/spark/internal/sources/QbeastBaseRelation.scala index ab3f206c3..633a4e0a8 100644 --- a/src/main/scala/io/qbeast/spark/internal/sources/QbeastBaseRelation.scala +++ b/src/main/scala/io/qbeast/spark/internal/sources/QbeastBaseRelation.scala @@ -23,12 +23,6 @@ import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat */ object QbeastBaseRelation { - /** - * Creates a QbeastBaseRelation instance - * @param tableID the identifier of the table - * @return the QbeastBaseRelation - */ - /** * Returns a HadoopFsRelation that contains all of the data present * in the table. This relation will be continually updated diff --git a/src/main/scala/io/qbeast/spark/internal/sources/QbeastDataSource.scala b/src/main/scala/io/qbeast/spark/internal/sources/QbeastDataSource.scala index b6da36012..bed92e890 100644 --- a/src/main/scala/io/qbeast/spark/internal/sources/QbeastDataSource.scala +++ b/src/main/scala/io/qbeast/spark/internal/sources/QbeastDataSource.scala @@ -57,6 +57,7 @@ class QbeastDataSource private[sources] (private val tableFactory: IndexedTableF val tableId = QbeastOptions.loadTableIDFromParameters(properties.asScala.toMap) val indexedTable = tableFactory.getIndexedTable(tableId) if (indexedTable.exists) { + // If the table exists, we make sure to pass all the properties to QbeastTableImpl val currentRevision = metadataManager.loadSnapshot(tableId).loadLatestRevision val indexProperties = Map( "columnsToIndex" -> currentRevision.columnTransformers.map(_.columnName).mkString(","), @@ -106,12 +107,10 @@ class QbeastDataSource private[sources] (private val tableFactory: IndexedTableF sqlContext: SQLContext, parameters: Map[String, String]): BaseRelation = { val tableID = QbeastOptions.loadTableIDFromParameters(parameters) - val indexedTable = tableFactory.getIndexedTable(tableID) - - // If the table has data registered on the snapshot, we can load from the IndexedTable factory - // Otherwise, the table can be loaded from the catalog - if (indexedTable.exists) indexedTable.load() - else { + val table = tableFactory.getIndexedTable(tableID) + if (table.exists) { + table.load() + } else { throw AnalysisExceptionFactory.create( s"'$tableID' is not a Qbeast formatted data directory.") } diff --git a/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastScanRDD.scala b/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastScan.scala similarity index 73% rename from src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastScanRDD.scala rename to src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastScan.scala index 1c3296c50..d536a1d8e 100644 --- a/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastScanRDD.scala +++ b/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastScan.scala @@ -12,19 +12,24 @@ import org.apache.spark.sql.sources.{BaseRelation, TableScan} import org.apache.spark.sql.types.StructType /** - * Extends Scan Builder for V1 DataSource + * Extends Scan for V1 DataSource * It allows Spark to read from a Qbeast Formatted table - * TODO include here the logic to get rid of the QbeastHash while reading the records * @param indexedTable the IndexedTable */ -class QbeastScanRDD(indexedTable: IndexedTable) extends V1Scan { + +// TODO Disclaimer: This class is not used yet. +// QbeastScan returns an Scan object that could allow QbeastTableImpl to SupportRead +// This can be the object where we can include logic +// to get rid of the QbeastHash while reading the records +class QbeastScan(indexedTable: IndexedTable) extends V1Scan { private lazy val qbeastBaseRelation = QbeastBaseRelation.forQbeastTable(indexedTable) override def toV1TableScan[T <: BaseRelation with TableScan](context: SQLContext): T = { - // TODO add PrunedFilteredScan as an extension and implement the methods + // TODO add PrunedFilteredScan (column pruning + filter pushdown) + // as an extension and implement the methods new BaseRelation with TableScan { override def sqlContext: SQLContext = context diff --git a/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastTableImpl.scala b/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastTableImpl.scala index 9c9eb73fe..5495a8f6c 100644 --- a/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastTableImpl.scala +++ b/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastTableImpl.scala @@ -19,6 +19,14 @@ import org.apache.spark.sql.types.StructType import java.util import scala.collection.JavaConverters._ +/** + * Table Implementation for Qbeast Format + * @param path the Path of the table + * @param options the write options + * @param schema the schema of the table + * @param catalogTable the underlying Catalog Table, if any + * @param tableFactory the IndexedTable Factory + */ class QbeastTableImpl private[sources] ( path: Path, options: Map[String, String], @@ -37,17 +45,6 @@ class QbeastTableImpl private[sources] ( private val indexedTable = tableFactory.getIndexedTable(tableId) - /** - * Checks if the table exists on the catalog - * @return true if exists - */ - def isCatalogTable: Boolean = { - // TODO Check if exists on the catalog - // I don't think this is the better way to do so - val tableName = pathString.split("/").last - spark.catalog.tableExists(tableName) - } - override def name(): String = tableId.id override def schema(): StructType = { @@ -71,7 +68,7 @@ class QbeastTableImpl private[sources] ( override def v1Table: CatalogTable = { if (catalogTable.isDefined) catalogTable.get - else throw AnalysisExceptionFactory.create("no catalog table defined") + else throw AnalysisExceptionFactory.create("No catalog table defined") } } diff --git a/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastWriteBuilder.scala b/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastWriteBuilder.scala index 2dd5425e4..f09361682 100644 --- a/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastWriteBuilder.scala +++ b/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastWriteBuilder.scala @@ -15,6 +15,12 @@ import org.apache.spark.sql.sources.{Filter, InsertableRelation} import scala.collection.convert.ImplicitConversions.`map AsScala` +/** + * WriteBuilder implementation for Qbeast Format + * @param info the write information + * @param properties the table properties + * @param indexedTable the Indexed Table + */ class QbeastWriteBuilder( info: LogicalWriteInfo, properties: Map[String, String], From d1e4db40fe3c97072ab0a321be9b6342979f00da Mon Sep 17 00:00:00 2001 From: osopardo1 Date: Sat, 6 Aug 2022 08:21:29 +0200 Subject: [PATCH 19/49] Remove spark val from QbeastTableImpl --- .../io/qbeast/spark/internal/sources/v2/QbeastTableImpl.scala | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastTableImpl.scala b/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastTableImpl.scala index 5495a8f6c..dfd13d30a 100644 --- a/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastTableImpl.scala +++ b/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastTableImpl.scala @@ -10,7 +10,7 @@ import org.apache.spark.sql.connector.catalog.TableCapability._ import io.qbeast.spark.table.IndexedTableFactory import org.apache.hadoop.fs.Path import org.apache.spark.sql.catalyst.catalog.CatalogTable -import org.apache.spark.sql.{AnalysisExceptionFactory, SparkSession, V2toV1Fallback} +import org.apache.spark.sql.{AnalysisExceptionFactory, V2toV1Fallback} import org.apache.spark.sql.connector.catalog.{SupportsWrite, Table, TableCapability} import org.apache.spark.sql.connector.write.{LogicalWriteInfo, WriteBuilder} import org.apache.spark.sql.sources.BaseRelation @@ -37,8 +37,6 @@ class QbeastTableImpl private[sources] ( with SupportsWrite with V2toV1Fallback { - private lazy val spark = SparkSession.active - private val pathString = path.toString private val tableId = QTableID(pathString) From 7fa0bb102b6030ad99deed26fbd352bee8277b7c Mon Sep 17 00:00:00 2001 From: Jiawei Date: Mon, 8 Aug 2022 17:44:07 +0200 Subject: [PATCH 20/49] Add INSERT INTO tests using more statements, add INSERT OVERWRITE tests, add tests for both INSERT INTO and OVERWRITE using real data --- .../spark/utils/QbeastInsertToTest.scala | 227 +++++++++++++++++- 1 file changed, 224 insertions(+), 3 deletions(-) diff --git a/src/test/scala/io/qbeast/spark/utils/QbeastInsertToTest.scala b/src/test/scala/io/qbeast/spark/utils/QbeastInsertToTest.scala index 964ad98c1..4210317e9 100644 --- a/src/test/scala/io/qbeast/spark/utils/QbeastInsertToTest.scala +++ b/src/test/scala/io/qbeast/spark/utils/QbeastInsertToTest.scala @@ -1,6 +1,7 @@ package io.qbeast.spark.utils -import io.qbeast.spark.{QbeastIntegrationTestSpec} +import io.qbeast.spark.QbeastIntegrationTestSpec +import org.apache.spark.sql.functions.col class QbeastInsertToTest extends QbeastIntegrationTestSpec { @@ -110,9 +111,229 @@ class QbeastInsertToTest extends QbeastIntegrationTestSpec { spark.sql("select * from t").collect() shouldBe initialData .union(Seq(4).toDF()) .collect() + } + } + + it should "support INSERT INTO using a TABLE statement" in withQbeastContextSparkAndTmpDir { + (spark, tmpDir) => + { + import spark.implicits._ + + val cubeSize = 10000 + val initialData = Seq(5, 6, 7, 8).toDF("value") + val dataToInsert = Seq(1, 2, 3, 4).toDF("value") + + initialData.write + .mode("append") + .format("qbeast") + .options(Map("columnsToIndex" -> "value", "cubeSize" -> cubeSize.toString)) + .save(tmpDir) + + val df = spark.read.format("qbeast").load(tmpDir) + df.createOrReplaceTempView("initial") + dataToInsert.createOrReplaceTempView("toInsert") + + spark.sql("INSERT INTO initial TABLE toInsert") + + spark.sql("SELECT * FROM initial").collect() shouldBe initialData + .union(dataToInsert) + .collect() + } + } + + it should "support INSERT INTO with COLUMN LIST" in withQbeastContextSparkAndTmpDir { + (spark, tmpDir) => + { + import spark.implicits._ + + val cubeSize = 10000 + val initialData = Seq(("1", 1), ("2", 2), ("3", 3), ("4", 4)).toDF("a", "b") + val dataToInsert = Seq(("5", 5), ("6", 6)).toDF("a", "b") + + initialData.write + .mode("append") + .format("qbeast") + .options(Map("columnsToIndex" -> "a,b", "cubeSize" -> cubeSize.toString)) + .save(tmpDir) + + val df = spark.read.format("qbeast").load(tmpDir) + df.createOrReplaceTempView("initial") + + spark.sql("INSERT INTO initial (a, b) VALUES ('5', 5), ('6', 6)") + + spark.sql("SELECT * FROM initial").collect() shouldBe initialData + .union(dataToInsert) + .collect() + } + } + + it should "support INSERT OVERWRITE using a VALUE clause" in withQbeastContextSparkAndTmpDir { + (spark, tmpDir) => + { + import spark.implicits._ + + val cubeSize = 10000 + val initialData = Seq(("1", 1), ("2", 2), ("3", 3), ("4", 4)).toDF("a", "b") + val dataToInsert = Seq(("5", 5), ("6", 6)).toDF("a", "b") + + initialData.write + .mode("append") + .format("qbeast") + .options(Map("columnsToIndex" -> "a,b", "cubeSize" -> cubeSize.toString)) + .save(tmpDir) + + val df = spark.read.format("qbeast").load(tmpDir) + df.createOrReplaceTempView("initial") + + spark.sql("INSERT OVERWRITE initial VALUES ('5', 5), ('6', 6)") + + spark.sql("SELECT * FROM initial").collect() shouldBe dataToInsert.collect() + } + } + + it should "support INSERT OVERWRITE using a SELECT statement" in withQbeastContextSparkAndTmpDir { + (spark, tmpDir) => + { + import spark.implicits._ + + val cubeSize = 10000 + val initialData = Seq(("1", 1), ("2", 2), ("3", 3), ("4", 4)).toDF("a", "b") + val dataToInsert = Seq(("5", 5), ("6", 6)).toDF("a", "b") + + initialData.write + .mode("append") + .format("qbeast") + .options(Map("columnsToIndex" -> "a,b", "cubeSize" -> cubeSize.toString)) + .save(tmpDir) + + val df = spark.read.format("qbeast").load(tmpDir) + df.createOrReplaceTempView("initial") + dataToInsert.createOrReplaceTempView("toInsert") + + spark.sql("INSERT OVERWRITE initial SELECT a, b FROM toInsert") + + spark.sql("SELECT * FROM initial").collect() shouldBe dataToInsert.collect() + } + } + + it should "support INSERT OVERWRITE using a TABLE statement" in withQbeastContextSparkAndTmpDir { + (spark, tmpDir) => + { + import spark.implicits._ + + val cubeSize = 10000 + val initialData = Seq(("1", 1), ("2", 2), ("3", 3), ("4", 4)).toDF("a", "b") + val dataToInsert = Seq(("5", 5), ("6", 6)).toDF("a", "b") + + initialData.write + .mode("append") + .format("qbeast") + .options(Map("columnsToIndex" -> "a,b", "cubeSize" -> cubeSize.toString)) + .save(tmpDir) + + val df = spark.read.format("qbeast").load(tmpDir) + df.createOrReplaceTempView("initial") + dataToInsert.createOrReplaceTempView("toInsert") + + spark.sql("INSERT OVERWRITE initial TABLE toInsert") + + spark.sql("SELECT * FROM initial").collect() shouldBe dataToInsert.collect() + } + } + + it should "support INSERT OVERWRITE using a FROM statement" in withQbeastContextSparkAndTmpDir { + (spark, tmpDir) => + { + import spark.implicits._ + + val cubeSize = 10000 + val initialData = Seq(("1", 1), ("2", 2), ("3", 3), ("4", 4)).toDF("a", "b") + val dataToInsert = Seq(("5", 5), ("6", 6)).toDF("a", "b") + + initialData.write + .mode("append") + .format("qbeast") + .options(Map("columnsToIndex" -> "a,b", "cubeSize" -> cubeSize.toString)) + .save(tmpDir) + + val df = spark.read.format("qbeast").load(tmpDir) + df.createOrReplaceTempView("initial") + dataToInsert.createOrReplaceTempView("toInsert") + + spark.sql("INSERT OVERWRITE initial FROM toInsert SELECT *") + + spark.sql("SELECT * FROM initial").collect() shouldBe dataToInsert.collect() + } + } + + it should "support INSERT INTO using a TABLE statement on real data" in + withQbeastContextSparkAndTmpDir { (spark, tmpDir) => + { + import spark.implicits._ + + val sourcePath = "src/test/resources/ecommerce100k_2019_Oct.csv" + val targetColumns = Seq("product_id", "brand", "price", "user_id") + + val initialData = + spark.read + .option("header", "true") + .option("inferSchema", "true") + .csv(sourcePath) + .select(targetColumns.map(col): _*) + val dataToInsert = Seq((1, "qbeast", 9.99, 1)).toDF(targetColumns: _*) + + initialData.write + .format("qbeast") + .option("cubeSize", "5000") + .option("columnsToIndex", "user_id,product_id") + .save(tmpDir) + + val df = spark.read.format("qbeast").load(tmpDir) + + df.createOrReplaceTempView("initial") + dataToInsert.createOrReplaceTempView("toInsert") + + spark.sql("INSERT INTO initial TABLE toInsert") + + spark + .sql("""SELECT * FROM initial + |WHERE product_id == 1 and brand == 'qbeast' and + |price == 9.99 and user_id == 1""".stripMargin.replaceAll("\n", " ")) + .count shouldBe 1 + + spark.sql("SELECT * FROM initial").count shouldBe 100001 + } + } + + it should "support INSERT OVERWRITE using a TABLE statement on real data" in + withQbeastContextSparkAndTmpDir { (spark, tmpDir) => + { + import spark.implicits._ + + val sourcePath = "src/test/resources/ecommerce100k_2019_Oct.csv" + val targetColumns = Seq("product_id", "brand", "price", "user_id") + + val initialData = + spark.read + .option("header", "true") + .option("inferSchema", "true") + .csv(sourcePath) + .select(targetColumns.map(col): _*) + val dataToInsert = Seq((1, "qbeast", 9.99, 1)).toDF(targetColumns: _*) + + initialData.write + .format("qbeast") + .option("cubeSize", "5000") + .option("columnsToIndex", "user_id,product_id") + .save(tmpDir) + + val df = spark.read.format("qbeast").load(tmpDir) + + df.createOrReplaceTempView("initial") + dataToInsert.createOrReplaceTempView("toInsert") - // TODO there might be more types of insert statements, but the most important - // ones are the ones above + spark.sql("INSERT OVERWRITE initial TABLE toInsert") + spark.sql("SELECT * FROM initial").collect() shouldBe dataToInsert.collect() } } From de3242d15c9fd6284178419b4003555b2635ec69 Mon Sep 17 00:00:00 2001 From: Jiawei Date: Mon, 8 Aug 2022 17:50:08 +0200 Subject: [PATCH 21/49] Add comments --- .../scala/io/qbeast/spark/utils/QbeastInsertToTest.scala | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/src/test/scala/io/qbeast/spark/utils/QbeastInsertToTest.scala b/src/test/scala/io/qbeast/spark/utils/QbeastInsertToTest.scala index 4210317e9..c5b03d0bb 100644 --- a/src/test/scala/io/qbeast/spark/utils/QbeastInsertToTest.scala +++ b/src/test/scala/io/qbeast/spark/utils/QbeastInsertToTest.scala @@ -133,6 +133,7 @@ class QbeastInsertToTest extends QbeastIntegrationTestSpec { df.createOrReplaceTempView("initial") dataToInsert.createOrReplaceTempView("toInsert") + // Insert using a TABLE statement spark.sql("INSERT INTO initial TABLE toInsert") spark.sql("SELECT * FROM initial").collect() shouldBe initialData @@ -159,6 +160,7 @@ class QbeastInsertToTest extends QbeastIntegrationTestSpec { val df = spark.read.format("qbeast").load(tmpDir) df.createOrReplaceTempView("initial") + // Insert using a COLUMN LIST spark.sql("INSERT INTO initial (a, b) VALUES ('5', 5), ('6', 6)") spark.sql("SELECT * FROM initial").collect() shouldBe initialData @@ -185,6 +187,7 @@ class QbeastInsertToTest extends QbeastIntegrationTestSpec { val df = spark.read.format("qbeast").load(tmpDir) df.createOrReplaceTempView("initial") + // Overwrite using a VALUE clause spark.sql("INSERT OVERWRITE initial VALUES ('5', 5), ('6', 6)") spark.sql("SELECT * FROM initial").collect() shouldBe dataToInsert.collect() @@ -210,6 +213,7 @@ class QbeastInsertToTest extends QbeastIntegrationTestSpec { df.createOrReplaceTempView("initial") dataToInsert.createOrReplaceTempView("toInsert") + // Overwrite using a SELECT statement spark.sql("INSERT OVERWRITE initial SELECT a, b FROM toInsert") spark.sql("SELECT * FROM initial").collect() shouldBe dataToInsert.collect() @@ -235,6 +239,7 @@ class QbeastInsertToTest extends QbeastIntegrationTestSpec { df.createOrReplaceTempView("initial") dataToInsert.createOrReplaceTempView("toInsert") + // Overwrite using a TABLE statement spark.sql("INSERT OVERWRITE initial TABLE toInsert") spark.sql("SELECT * FROM initial").collect() shouldBe dataToInsert.collect() @@ -260,6 +265,7 @@ class QbeastInsertToTest extends QbeastIntegrationTestSpec { df.createOrReplaceTempView("initial") dataToInsert.createOrReplaceTempView("toInsert") + // Overwrite using a FROM statement spark.sql("INSERT OVERWRITE initial FROM toInsert SELECT *") spark.sql("SELECT * FROM initial").collect() shouldBe dataToInsert.collect() @@ -293,6 +299,7 @@ class QbeastInsertToTest extends QbeastIntegrationTestSpec { df.createOrReplaceTempView("initial") dataToInsert.createOrReplaceTempView("toInsert") + // Insert using a TABLE statement on real data spark.sql("INSERT INTO initial TABLE toInsert") spark @@ -332,7 +339,9 @@ class QbeastInsertToTest extends QbeastIntegrationTestSpec { df.createOrReplaceTempView("initial") dataToInsert.createOrReplaceTempView("toInsert") + // Overwrite using a TABLE statement on real data spark.sql("INSERT OVERWRITE initial TABLE toInsert") + spark.sql("SELECT * FROM initial").collect() shouldBe dataToInsert.collect() } } From 54679866281aaa81724da98e631b498dadcfdadb Mon Sep 17 00:00:00 2001 From: Jiawei Date: Tue, 9 Aug 2022 16:24:52 +0200 Subject: [PATCH 22/49] Use existing method to read csv data --- .../spark/utils/QbeastInsertToTest.scala | 18 +++--------------- 1 file changed, 3 insertions(+), 15 deletions(-) diff --git a/src/test/scala/io/qbeast/spark/utils/QbeastInsertToTest.scala b/src/test/scala/io/qbeast/spark/utils/QbeastInsertToTest.scala index c5b03d0bb..ae1817338 100644 --- a/src/test/scala/io/qbeast/spark/utils/QbeastInsertToTest.scala +++ b/src/test/scala/io/qbeast/spark/utils/QbeastInsertToTest.scala @@ -277,15 +277,9 @@ class QbeastInsertToTest extends QbeastIntegrationTestSpec { { import spark.implicits._ - val sourcePath = "src/test/resources/ecommerce100k_2019_Oct.csv" val targetColumns = Seq("product_id", "brand", "price", "user_id") - val initialData = - spark.read - .option("header", "true") - .option("inferSchema", "true") - .csv(sourcePath) - .select(targetColumns.map(col): _*) + val initialData = loadTestData(spark).select(targetColumns.map(col): _*) val dataToInsert = Seq((1, "qbeast", 9.99, 1)).toDF(targetColumns: _*) initialData.write @@ -308,7 +302,7 @@ class QbeastInsertToTest extends QbeastIntegrationTestSpec { |price == 9.99 and user_id == 1""".stripMargin.replaceAll("\n", " ")) .count shouldBe 1 - spark.sql("SELECT * FROM initial").count shouldBe 100001 + spark.sql("SELECT * FROM initial").count shouldBe initialData.count + 1 } } @@ -317,15 +311,9 @@ class QbeastInsertToTest extends QbeastIntegrationTestSpec { { import spark.implicits._ - val sourcePath = "src/test/resources/ecommerce100k_2019_Oct.csv" val targetColumns = Seq("product_id", "brand", "price", "user_id") - val initialData = - spark.read - .option("header", "true") - .option("inferSchema", "true") - .csv(sourcePath) - .select(targetColumns.map(col): _*) + val initialData = loadTestData(spark).select(targetColumns.map(col): _*) val dataToInsert = Seq((1, "qbeast", 9.99, 1)).toDF(targetColumns: _*) initialData.write From f23dbdd4ed5505104e2a9304298294aa049c6bd4 Mon Sep 17 00:00:00 2001 From: Jiawei Date: Wed, 10 Aug 2022 19:59:30 +0200 Subject: [PATCH 23/49] Add INSERT INTO test for managed table --- .../utils/QbeastSparkIntegrationTest.scala | 26 +++++++++++++++++++ 1 file changed, 26 insertions(+) diff --git a/src/test/scala/io/qbeast/spark/utils/QbeastSparkIntegrationTest.scala b/src/test/scala/io/qbeast/spark/utils/QbeastSparkIntegrationTest.scala index 4b72b8432..4a9c83b74 100644 --- a/src/test/scala/io/qbeast/spark/utils/QbeastSparkIntegrationTest.scala +++ b/src/test/scala/io/qbeast/spark/utils/QbeastSparkIntegrationTest.scala @@ -1,6 +1,7 @@ package io.qbeast.spark.utils import io.qbeast.spark.QbeastIntegrationTestSpec +import org.apache.spark.sql.functions.col import org.apache.spark.sql.{DataFrame, SparkSession} class QbeastSparkIntegrationTest extends QbeastIntegrationTestSpec { @@ -47,4 +48,29 @@ class QbeastSparkIntegrationTest extends QbeastIntegrationTestSpec { } } + it should "support INSERT INTO on a managed Table" in + withQbeastContextSparkAndTmpWarehouse { (spark, _) => + { + import spark.implicits._ + + val targetColumns = Seq("product_id", "brand", "price", "user_id") + + val initialData = loadTestData(spark).select(targetColumns.map(col): _*) + val dataToInsert = Seq((1, "qbeast", 9.99, 1)).toDF(targetColumns: _*) + + initialData.write + .format("qbeast") + .option("cubeSize", "5000") + .option("columnsToIndex", "user_id,product_id") + .saveAsTable("ecommerce") + + dataToInsert.createOrReplaceTempView("toInsert") + + spark.sql("INSERT INTO ecommerce TABLE toInsert") + spark.sql("SELECT * FROM ecommerce").count shouldBe 1 + initialData.count + spark.sql("SELECT * FROM ecommerce WHERE brand == 'qbeast'").count shouldBe 1 + + } + } + } From 7b60c1b2561ca8068b6acb41b1e338ddae29ac45 Mon Sep 17 00:00:00 2001 From: Jiawei Date: Thu, 11 Aug 2022 19:05:56 +0200 Subject: [PATCH 24/49] Add 'INSERT OVERWRITE'-awareness to managed tables --- .../internal/sources/v2/QbeastWriteBuilder.scala | 13 +++++++++++-- 1 file changed, 11 insertions(+), 2 deletions(-) diff --git a/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastWriteBuilder.scala b/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastWriteBuilder.scala index f09361682..0375bb580 100644 --- a/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastWriteBuilder.scala +++ b/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastWriteBuilder.scala @@ -29,7 +29,14 @@ class QbeastWriteBuilder( with V1WriteBuilder with SupportsOverwrite { - override def overwrite(filters: Array[Filter]): WriteBuilder = this + private var forceOverwrite = false + + override def overwrite(filters: Array[Filter]): WriteBuilder = { + // TODO: User filters to select existing data to remove + // The remaining and the inserted data are then to be written + forceOverwrite = true + this + } /** * Build an InsertableRelation to be able to write the data in QbeastFormat @@ -39,10 +46,12 @@ class QbeastWriteBuilder( new InsertableRelation { def insert(data: DataFrame, overwrite: Boolean): Unit = { + val append = if (forceOverwrite) false else !overwrite + // Passing the options in the query plan plus the properties // because columnsToIndex needs to be included in the contract val writeOptions = info.options().toMap ++ properties - indexedTable.save(data, writeOptions, append = !overwrite) + indexedTable.save(data, writeOptions, append) } } } From 8894e91a94e6e595cd702d9d5194f094713d91ee Mon Sep 17 00:00:00 2001 From: Jiawei Date: Thu, 11 Aug 2022 19:06:21 +0200 Subject: [PATCH 25/49] Add test for INSERT OVERWRITE on managed tables --- .../utils/QbeastSparkIntegrationTest.scala | 24 +++++++++++++++++++ 1 file changed, 24 insertions(+) diff --git a/src/test/scala/io/qbeast/spark/utils/QbeastSparkIntegrationTest.scala b/src/test/scala/io/qbeast/spark/utils/QbeastSparkIntegrationTest.scala index 4a9c83b74..30c210afe 100644 --- a/src/test/scala/io/qbeast/spark/utils/QbeastSparkIntegrationTest.scala +++ b/src/test/scala/io/qbeast/spark/utils/QbeastSparkIntegrationTest.scala @@ -73,4 +73,28 @@ class QbeastSparkIntegrationTest extends QbeastIntegrationTestSpec { } } + it should "support INSERT OVERWRITE on a managed Table" in + withQbeastContextSparkAndTmpWarehouse { (spark, _) => + { + import spark.implicits._ + + val targetColumns = Seq("product_id", "brand", "price", "user_id") + + val initialData = loadTestData(spark).select(targetColumns.map(col): _*) + val dataToInsert = Seq((1, "qbeast", 9.99, 1)).toDF(targetColumns: _*) + + initialData.write + .format("qbeast") + .option("cubeSize", "5000") + .option("columnsToIndex", "user_id,product_id") + .saveAsTable("ecommerce") + + dataToInsert.createOrReplaceTempView("toInsert") + + spark.sql("INSERT OVERWRITE ecommerce TABLE toInsert") + spark.sql("SELECT * FROM ecommerce").count shouldBe 1 + spark.sql("SELECT * FROM ecommerce WHERE brand == 'qbeast'").count shouldBe 1 + + } + } } From 10f1f415227cc054bd32f92f86ae11fc774b944b Mon Sep 17 00:00:00 2001 From: osopardo1 Date: Mon, 5 Sep 2022 13:31:23 +0200 Subject: [PATCH 26/49] Use same test data --- .../spark/QbeastIntegrationTestSpec.scala | 1 - .../utils/QbeastSQLIntegrationTest.scala | 92 ++++++++++++++++--- .../utils/QbeastSparkIntegrationTest.scala | 84 ++++++++--------- 3 files changed, 118 insertions(+), 59 deletions(-) diff --git a/src/test/scala/io/qbeast/spark/QbeastIntegrationTestSpec.scala b/src/test/scala/io/qbeast/spark/QbeastIntegrationTestSpec.scala index e04766779..1289f61ba 100644 --- a/src/test/scala/io/qbeast/spark/QbeastIntegrationTestSpec.scala +++ b/src/test/scala/io/qbeast/spark/QbeastIntegrationTestSpec.scala @@ -10,7 +10,6 @@ import io.qbeast.core.model.IndexManager import io.qbeast.spark.delta.SparkDeltaMetadataManager import io.qbeast.spark.delta.writer.{SparkDeltaDataWriter} import io.qbeast.spark.index.{SparkOTreeManager, SparkRevisionFactory} -import io.qbeast.spark.internal.QbeastSparkSessionExtension import io.qbeast.spark.table.IndexedTableFactoryImpl import org.apache.log4j.{Level, Logger} import org.apache.spark.SparkConf diff --git a/src/test/scala/io/qbeast/spark/utils/QbeastSQLIntegrationTest.scala b/src/test/scala/io/qbeast/spark/utils/QbeastSQLIntegrationTest.scala index 78ec0baab..a9cbc4f70 100644 --- a/src/test/scala/io/qbeast/spark/utils/QbeastSQLIntegrationTest.scala +++ b/src/test/scala/io/qbeast/spark/utils/QbeastSQLIntegrationTest.scala @@ -2,6 +2,9 @@ package io.qbeast.spark.utils import io.qbeast.TestClasses.Student import io.qbeast.spark.QbeastIntegrationTestSpec +import org.apache.spark.sql.{DataFrame, SparkSession} +import org.apache.spark.sql.functions.col +import org.apache.spark.sql.types.{IntegerType, StringType, StructField, StructType} import scala.util.Random @@ -9,10 +12,21 @@ class QbeastSQLIntegrationTest extends QbeastIntegrationTestSpec { private val students = 1.to(10).map(i => Student(i, i.toString, Random.nextInt())) - "QbeastSpark SQL" should "work with CREATE TABLE" in withQbeastContextSparkAndTmpWarehouse( + private val schema = StructType( + Seq( + StructField("id", IntegerType, true), + StructField("name", StringType, true), + StructField("age", IntegerType, true))) + + private def createTestData(spark: SparkSession): DataFrame = { + import spark.implicits._ + val data = students.toDF() + spark.createDataFrame(data.rdd, schema) + } + + "QbeastSpark SQL" should "support CREATE TABLE" in withQbeastContextSparkAndTmpWarehouse( (spark, tmpWarehouse) => { - import spark.implicits._ - val data = students.toDF() + val data = createTestData(spark) data.createOrReplaceTempView("data") spark.sql( @@ -39,9 +53,8 @@ class QbeastSQLIntegrationTest extends QbeastIntegrationTestSpec { }) - it should "work with INSERT INTO" in withQbeastContextSparkAndTmpWarehouse((spark, _) => { - import spark.implicits._ - val data = students.toDF() + it should "support INSERT INTO" in withQbeastContextSparkAndTmpWarehouse((spark, _) => { + val data = createTestData(spark) data.createOrReplaceTempView("data") spark.sql( @@ -55,13 +68,15 @@ class QbeastSQLIntegrationTest extends QbeastIntegrationTestSpec { indexed.count() shouldBe data.count() indexed.columns.toSet shouldBe data.columns.toSet + + assertSmallDatasetEquality(indexed, data, orderedComparison = false) + }) - it should "work with CREATE TABLE AS SELECT statement" in withQbeastContextSparkAndTmpWarehouse( + it should "support CREATE TABLE AS SELECT statement" in withQbeastContextSparkAndTmpWarehouse( (spark, _) => { - import spark.implicits._ - val data = students.toDF() + val data = createTestData(spark) data.createOrReplaceTempView("data") spark.sql( @@ -74,12 +89,14 @@ class QbeastSQLIntegrationTest extends QbeastIntegrationTestSpec { indexed.count() shouldBe data.count() indexed.columns.toSet shouldBe data.columns.toSet + + assertSmallDatasetEquality(indexed, data, orderedComparison = false) + }) it should "work with LOCATION" in withQbeastContextSparkAndTmpDir((spark, tmpDir) => { - import spark.implicits._ - val data = students.toDF() + val data = createTestData(spark) data.createOrReplaceTempView("data") spark.sql( @@ -93,6 +110,59 @@ class QbeastSQLIntegrationTest extends QbeastIntegrationTestSpec { indexed.count() shouldBe data.count() indexed.columns.toSet shouldBe data.columns.toSet + + assertSmallDatasetEquality(indexed, data, orderedComparison = false) + }) + it should "support INSERT INTO on a managed Table" in + withQbeastContextSparkAndTmpWarehouse { (spark, _) => + { + import spark.implicits._ + + val targetColumns = Seq("product_id", "brand", "price", "user_id") + + val initialData = loadTestData(spark).select(targetColumns.map(col): _*) + val dataToInsert = Seq((1, "qbeast", 9.99, 1)).toDF(targetColumns: _*) + + initialData.write + .format("qbeast") + .option("cubeSize", "5000") + .option("columnsToIndex", "user_id,product_id") + .saveAsTable("ecommerce") + + dataToInsert.createOrReplaceTempView("toInsert") + + spark.sql("INSERT INTO ecommerce TABLE toInsert") + spark.sql("SELECT * FROM ecommerce").count shouldBe 1 + initialData.count + spark.sql("SELECT * FROM ecommerce WHERE brand == 'qbeast'").count shouldBe 1 + + } + } + + it should "support INSERT OVERWRITE on a managed Table" in + withQbeastContextSparkAndTmpWarehouse { (spark, _) => + { + import spark.implicits._ + + val targetColumns = Seq("product_id", "brand", "price", "user_id") + + val initialData = loadTestData(spark).select(targetColumns.map(col): _*) + val dataToInsert = Seq((1, "qbeast", 9.99, 1)).toDF(targetColumns: _*) + + initialData.write + .format("qbeast") + .option("cubeSize", "5000") + .option("columnsToIndex", "user_id,product_id") + .saveAsTable("ecommerce") + + dataToInsert.createOrReplaceTempView("toInsert") + + spark.sql("INSERT OVERWRITE ecommerce TABLE toInsert") + spark.sql("SELECT * FROM ecommerce").count shouldBe 1 + spark.sql("SELECT * FROM ecommerce WHERE brand == 'qbeast'").count shouldBe 1 + + } + } + } diff --git a/src/test/scala/io/qbeast/spark/utils/QbeastSparkIntegrationTest.scala b/src/test/scala/io/qbeast/spark/utils/QbeastSparkIntegrationTest.scala index 30c210afe..774ce153e 100644 --- a/src/test/scala/io/qbeast/spark/utils/QbeastSparkIntegrationTest.scala +++ b/src/test/scala/io/qbeast/spark/utils/QbeastSparkIntegrationTest.scala @@ -1,14 +1,26 @@ package io.qbeast.spark.utils +import io.qbeast.TestClasses.Student import io.qbeast.spark.QbeastIntegrationTestSpec -import org.apache.spark.sql.functions.col +import org.apache.spark.sql.types.{IntegerType, StringType, StructField, StructType} import org.apache.spark.sql.{DataFrame, SparkSession} +import scala.util.Random + class QbeastSparkIntegrationTest extends QbeastIntegrationTestSpec { - def createTestData(spark: SparkSession): DataFrame = { + private val students = 1.to(10).map(i => Student(i, i.toString, Random.nextInt())) + + private val schema = StructType( + Seq( + StructField("id", IntegerType, true), + StructField("name", StringType, true), + StructField("age", IntegerType, true))) + + private def createTestData(spark: SparkSession): DataFrame = { import spark.implicits._ - Seq(1, 2, 3, 4).toDF("id") + val data = students.toDF() + spark.createDataFrame(data.rdd, schema) } "The QbeastDataSource" should @@ -19,11 +31,11 @@ class QbeastSparkIntegrationTest extends QbeastIntegrationTestSpec { val indexed = spark.read.format("qbeast").load(tmpDir) - indexed.count() shouldBe 4 + indexed.count() shouldBe data.count() - indexed.columns shouldBe Seq("id") + indexed.columns.toSet shouldBe data.columns.toSet - indexed.orderBy("id").collect() shouldBe data.orderBy("id").collect() + assertSmallDatasetEquality(indexed, data, orderedComparison = false) } } @@ -45,56 +57,34 @@ class QbeastSparkIntegrationTest extends QbeastIntegrationTestSpec { indexed.columns.toSet shouldBe data.columns.toSet + assertSmallDatasetEquality(indexed, data, orderedComparison = false) + } } - it should "support INSERT INTO on a managed Table" in - withQbeastContextSparkAndTmpWarehouse { (spark, _) => - { - import spark.implicits._ + it should "work with InsertInto" in withQbeastContextSparkAndTmpWarehouse { (spark, tmpDir) => + { - val targetColumns = Seq("product_id", "brand", "price", "user_id") + val data = createTestData(spark) + val location = tmpDir + "/external" + data.write + .format("qbeast") + .option("columnsToIndex", "id") + .option("location", location) + .saveAsTable("qbeast") - val initialData = loadTestData(spark).select(targetColumns.map(col): _*) - val dataToInsert = Seq((1, "qbeast", 9.99, 1)).toDF(targetColumns: _*) + val newData = data + newData.write.insertInto("qbeast") - initialData.write - .format("qbeast") - .option("cubeSize", "5000") - .option("columnsToIndex", "user_id,product_id") - .saveAsTable("ecommerce") + val indexed = spark.read.table("qbeast") + val allData = data.union(data) - dataToInsert.createOrReplaceTempView("toInsert") + indexed.count() shouldBe allData.count() - spark.sql("INSERT INTO ecommerce TABLE toInsert") - spark.sql("SELECT * FROM ecommerce").count shouldBe 1 + initialData.count - spark.sql("SELECT * FROM ecommerce WHERE brand == 'qbeast'").count shouldBe 1 + indexed.columns.toSet shouldBe allData.columns.toSet - } + assertSmallDatasetEquality(indexed, allData, orderedComparison = false) } + } - it should "support INSERT OVERWRITE on a managed Table" in - withQbeastContextSparkAndTmpWarehouse { (spark, _) => - { - import spark.implicits._ - - val targetColumns = Seq("product_id", "brand", "price", "user_id") - - val initialData = loadTestData(spark).select(targetColumns.map(col): _*) - val dataToInsert = Seq((1, "qbeast", 9.99, 1)).toDF(targetColumns: _*) - - initialData.write - .format("qbeast") - .option("cubeSize", "5000") - .option("columnsToIndex", "user_id,product_id") - .saveAsTable("ecommerce") - - dataToInsert.createOrReplaceTempView("toInsert") - - spark.sql("INSERT OVERWRITE ecommerce TABLE toInsert") - spark.sql("SELECT * FROM ecommerce").count shouldBe 1 - spark.sql("SELECT * FROM ecommerce WHERE brand == 'qbeast'").count shouldBe 1 - - } - } } From 88dd7f788e36e94c1715498b20d9c879f450a9cd Mon Sep 17 00:00:00 2001 From: osopardo1 Date: Tue, 6 Sep 2022 12:32:05 +0200 Subject: [PATCH 27/49] Adapting QbeastWriteBuilder to version 3.2.2 of Spark --- .../sources/v2/QbeastWriteBuilder.scala | 25 +++++++++++-------- 1 file changed, 14 insertions(+), 11 deletions(-) diff --git a/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastWriteBuilder.scala b/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastWriteBuilder.scala index 0375bb580..e61b19b48 100644 --- a/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastWriteBuilder.scala +++ b/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastWriteBuilder.scala @@ -4,11 +4,11 @@ package io.qbeast.spark.internal.sources.v2 import io.qbeast.spark.table.IndexedTable -import org.apache.spark.sql.{DataFrame} +import org.apache.spark.sql.DataFrame import org.apache.spark.sql.connector.write.{ LogicalWriteInfo, SupportsOverwrite, - V1WriteBuilder, + V1Write, WriteBuilder } import org.apache.spark.sql.sources.{Filter, InsertableRelation} @@ -26,7 +26,6 @@ class QbeastWriteBuilder( properties: Map[String, String], indexedTable: IndexedTable) extends WriteBuilder - with V1WriteBuilder with SupportsOverwrite { private var forceOverwrite = false @@ -42,18 +41,22 @@ class QbeastWriteBuilder( * Build an InsertableRelation to be able to write the data in QbeastFormat * @return the InsertableRelation with the corresponding method */ - override def buildForV1Write(): InsertableRelation = { + override def build(): V1Write = new V1Write { - new InsertableRelation { - def insert(data: DataFrame, overwrite: Boolean): Unit = { - val append = if (forceOverwrite) false else !overwrite + override def toInsertableRelation: InsertableRelation = { - // Passing the options in the query plan plus the properties - // because columnsToIndex needs to be included in the contract - val writeOptions = info.options().toMap ++ properties - indexedTable.save(data, writeOptions, append) + new InsertableRelation { + def insert(data: DataFrame, overwrite: Boolean): Unit = { + val append = if (forceOverwrite) false else !overwrite + + // Passing the options in the query plan plus the properties + // because columnsToIndex needs to be included in the contract + val writeOptions = info.options().toMap ++ properties + indexedTable.save(data, writeOptions, append) + } } } + } } From 13f86b9e995a66e2fff0cf8695f6c928df53ae54 Mon Sep 17 00:00:00 2001 From: osopardo1 Date: Tue, 6 Sep 2022 14:14:50 +0200 Subject: [PATCH 28/49] Armonize tests --- .../utils/QbeastSQLIntegrationTest.scala | 56 +++++++------------ .../utils/QbeastSparkIntegrationTest.scala | 24 ++++---- 2 files changed, 32 insertions(+), 48 deletions(-) diff --git a/src/test/scala/io/qbeast/spark/utils/QbeastSQLIntegrationTest.scala b/src/test/scala/io/qbeast/spark/utils/QbeastSQLIntegrationTest.scala index a9cbc4f70..1c2e90d08 100644 --- a/src/test/scala/io/qbeast/spark/utils/QbeastSQLIntegrationTest.scala +++ b/src/test/scala/io/qbeast/spark/utils/QbeastSQLIntegrationTest.scala @@ -3,8 +3,6 @@ package io.qbeast.spark.utils import io.qbeast.TestClasses.Student import io.qbeast.spark.QbeastIntegrationTestSpec import org.apache.spark.sql.{DataFrame, SparkSession} -import org.apache.spark.sql.functions.col -import org.apache.spark.sql.types.{IntegerType, StringType, StructField, StructType} import scala.util.Random @@ -12,16 +10,9 @@ class QbeastSQLIntegrationTest extends QbeastIntegrationTestSpec { private val students = 1.to(10).map(i => Student(i, i.toString, Random.nextInt())) - private val schema = StructType( - Seq( - StructField("id", IntegerType, true), - StructField("name", StringType, true), - StructField("age", IntegerType, true))) - private def createTestData(spark: SparkSession): DataFrame = { import spark.implicits._ - val data = students.toDF() - spark.createDataFrame(data.rdd, schema) + students.toDF() } "QbeastSpark SQL" should "support CREATE TABLE" in withQbeastContextSparkAndTmpWarehouse( @@ -69,7 +60,7 @@ class QbeastSQLIntegrationTest extends QbeastIntegrationTestSpec { indexed.columns.toSet shouldBe data.columns.toSet - assertSmallDatasetEquality(indexed, data, orderedComparison = false) + assertSmallDatasetEquality(indexed, data, orderedComparison = false, ignoreNullable = true) }) @@ -90,7 +81,7 @@ class QbeastSQLIntegrationTest extends QbeastIntegrationTestSpec { indexed.columns.toSet shouldBe data.columns.toSet - assertSmallDatasetEquality(indexed, data, orderedComparison = false) + assertSmallDatasetEquality(indexed, data, orderedComparison = false, ignoreNullable = true) }) @@ -111,7 +102,7 @@ class QbeastSQLIntegrationTest extends QbeastIntegrationTestSpec { indexed.columns.toSet shouldBe data.columns.toSet - assertSmallDatasetEquality(indexed, data, orderedComparison = false) + assertSmallDatasetEquality(indexed, data, orderedComparison = false, ignoreNullable = true) }) @@ -120,22 +111,19 @@ class QbeastSQLIntegrationTest extends QbeastIntegrationTestSpec { { import spark.implicits._ - val targetColumns = Seq("product_id", "brand", "price", "user_id") - - val initialData = loadTestData(spark).select(targetColumns.map(col): _*) - val dataToInsert = Seq((1, "qbeast", 9.99, 1)).toDF(targetColumns: _*) + val data = createTestData(spark) + val dataToInsert = Seq(Student(90, "qbeast", 2)).toDF() - initialData.write + data.write .format("qbeast") - .option("cubeSize", "5000") - .option("columnsToIndex", "user_id,product_id") - .saveAsTable("ecommerce") + .option("columnsToIndex", "id,name") + .saveAsTable("students") dataToInsert.createOrReplaceTempView("toInsert") - spark.sql("INSERT INTO ecommerce TABLE toInsert") - spark.sql("SELECT * FROM ecommerce").count shouldBe 1 + initialData.count - spark.sql("SELECT * FROM ecommerce WHERE brand == 'qbeast'").count shouldBe 1 + spark.sql("INSERT INTO students TABLE toInsert") + spark.sql("SELECT * FROM students").count shouldBe 1 + data.count + spark.sql("SELECT * FROM students WHERE name == 'qbeast'").count shouldBe 1 } } @@ -145,23 +133,19 @@ class QbeastSQLIntegrationTest extends QbeastIntegrationTestSpec { { import spark.implicits._ - val targetColumns = Seq("product_id", "brand", "price", "user_id") + val data = createTestData(spark) + val dataToInsert = Seq(Student(90, "qbeast", 2)).toDF() - val initialData = loadTestData(spark).select(targetColumns.map(col): _*) - val dataToInsert = Seq((1, "qbeast", 9.99, 1)).toDF(targetColumns: _*) - - initialData.write + data.write .format("qbeast") - .option("cubeSize", "5000") - .option("columnsToIndex", "user_id,product_id") - .saveAsTable("ecommerce") + .option("columnsToIndex", "id,name") + .saveAsTable("students") dataToInsert.createOrReplaceTempView("toInsert") - spark.sql("INSERT OVERWRITE ecommerce TABLE toInsert") - spark.sql("SELECT * FROM ecommerce").count shouldBe 1 - spark.sql("SELECT * FROM ecommerce WHERE brand == 'qbeast'").count shouldBe 1 - + spark.sql("INSERT OVERWRITE students TABLE toInsert") + spark.sql("SELECT * FROM students").count shouldBe 1 + spark.sql("SELECT * FROM students WHERE name == 'qbeast'").count shouldBe 1 } } diff --git a/src/test/scala/io/qbeast/spark/utils/QbeastSparkIntegrationTest.scala b/src/test/scala/io/qbeast/spark/utils/QbeastSparkIntegrationTest.scala index 774ce153e..2cbccab13 100644 --- a/src/test/scala/io/qbeast/spark/utils/QbeastSparkIntegrationTest.scala +++ b/src/test/scala/io/qbeast/spark/utils/QbeastSparkIntegrationTest.scala @@ -2,7 +2,6 @@ package io.qbeast.spark.utils import io.qbeast.TestClasses.Student import io.qbeast.spark.QbeastIntegrationTestSpec -import org.apache.spark.sql.types.{IntegerType, StringType, StructField, StructType} import org.apache.spark.sql.{DataFrame, SparkSession} import scala.util.Random @@ -11,16 +10,9 @@ class QbeastSparkIntegrationTest extends QbeastIntegrationTestSpec { private val students = 1.to(10).map(i => Student(i, i.toString, Random.nextInt())) - private val schema = StructType( - Seq( - StructField("id", IntegerType, true), - StructField("name", StringType, true), - StructField("age", IntegerType, true))) - private def createTestData(spark: SparkSession): DataFrame = { import spark.implicits._ - val data = students.toDF() - spark.createDataFrame(data.rdd, schema) + students.toDF() } "The QbeastDataSource" should @@ -35,7 +27,11 @@ class QbeastSparkIntegrationTest extends QbeastIntegrationTestSpec { indexed.columns.toSet shouldBe data.columns.toSet - assertSmallDatasetEquality(indexed, data, orderedComparison = false) + assertSmallDatasetEquality( + indexed, + data, + orderedComparison = false, + ignoreNullable = true) } } @@ -57,7 +53,7 @@ class QbeastSparkIntegrationTest extends QbeastIntegrationTestSpec { indexed.columns.toSet shouldBe data.columns.toSet - assertSmallDatasetEquality(indexed, data, orderedComparison = false) + assertSmallDatasetEquality(indexed, data, orderedComparison = false, ignoreNullable = true) } } @@ -83,7 +79,11 @@ class QbeastSparkIntegrationTest extends QbeastIntegrationTestSpec { indexed.columns.toSet shouldBe allData.columns.toSet - assertSmallDatasetEquality(indexed, allData, orderedComparison = false) + assertSmallDatasetEquality( + indexed, + allData, + orderedComparison = false, + ignoreNullable = true) } } From 88087facb4e9bae139367409173db3c4ca78363b Mon Sep 17 00:00:00 2001 From: osopardo1 Date: Tue, 13 Sep 2022 11:40:25 +0200 Subject: [PATCH 29/49] Add staging properties to the table and change QbeastCatalog to QbeastDeltaCatalog To allow DeltaCatalog to act simultaneously with Qbeast, one solution is to extend directly that catalog. --- .../internal/sources/QbeastDataSource.scala | 9 +- .../sources/catalog/DefaultStagedTable.scala | 47 +++ .../sources/catalog/QbeastCatalog.scala | 281 ++++++++++++++---- .../sources/catalog/QbeastDeltaCatalog.scala | 102 +++++++ .../sources/v2/QbeastStagedTableImpl.scala | 113 +++++++ .../internal/sources/v2/QbeastTableImpl.scala | 3 +- .../spark/sql/SparkTransformUtils.scala | 84 ++++++ .../io/qbeast/context/QbeastConfigTest.scala | 5 +- .../io/qbeast/docs/DocumentationTests.scala | 2 +- .../spark/QbeastIntegrationTestSpec.scala | 67 +++-- .../sources/QbeastDeltaCatalogTest.scala | 44 +++ .../QbeastCompactionIntegrationTest.scala | 58 ++-- 12 files changed, 696 insertions(+), 119 deletions(-) create mode 100644 src/main/scala/io/qbeast/spark/internal/sources/catalog/DefaultStagedTable.scala create mode 100644 src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastDeltaCatalog.scala create mode 100644 src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastStagedTableImpl.scala create mode 100644 src/main/scala/org/apache/spark/sql/SparkTransformUtils.scala create mode 100644 src/test/scala/io/qbeast/spark/internal/sources/QbeastDeltaCatalogTest.scala diff --git a/src/main/scala/io/qbeast/spark/internal/sources/QbeastDataSource.scala b/src/main/scala/io/qbeast/spark/internal/sources/QbeastDataSource.scala index 77d588995..873b2d80d 100644 --- a/src/main/scala/io/qbeast/spark/internal/sources/QbeastDataSource.scala +++ b/src/main/scala/io/qbeast/spark/internal/sources/QbeastDataSource.scala @@ -62,9 +62,16 @@ class QbeastDataSource private[sources] (private val tableFactory: IndexedTableF "columnsToIndex" -> currentRevision.columnTransformers.map(_.columnName).mkString(","), "cubeSize" -> currentRevision.desiredCubeSize.toString) val tableProperties = properties.asScala.toMap ++ indexProperties - new QbeastTableImpl(new Path(tableId.id), tableProperties, None, None, tableFactory) + new QbeastTableImpl( + tableId.id, + new Path(tableId.id), + tableProperties, + None, + None, + tableFactory) } else { new QbeastTableImpl( + tableId.id, new Path(tableId.id), properties.asScala.toMap, Some(schema), diff --git a/src/main/scala/io/qbeast/spark/internal/sources/catalog/DefaultStagedTable.scala b/src/main/scala/io/qbeast/spark/internal/sources/catalog/DefaultStagedTable.scala new file mode 100644 index 000000000..32d81ea21 --- /dev/null +++ b/src/main/scala/io/qbeast/spark/internal/sources/catalog/DefaultStagedTable.scala @@ -0,0 +1,47 @@ +/* + * Copyright 2021 Qbeast Analytics, S.L. + */ +package io.qbeast.spark.internal.sources.catalog + +import org.apache.spark.sql.AnalysisExceptionFactory +import org.apache.spark.sql.connector.catalog.{ + Identifier, + StagedTable, + SupportsWrite, + Table, + TableCapability, + TableCatalog +} +import org.apache.spark.sql.connector.expressions.Transform +import org.apache.spark.sql.connector.write.{LogicalWriteInfo, WriteBuilder} +import org.apache.spark.sql.types.StructType + +private[catalog] case class DefaultStagedTable( + ident: Identifier, + table: Table, + catalog: TableCatalog) + extends SupportsWrite + with StagedTable { + + override def newWriteBuilder(info: LogicalWriteInfo): WriteBuilder = { + info match { + case supportsWrite: SupportsWrite => supportsWrite.newWriteBuilder(info) + case _ => + throw AnalysisExceptionFactory.create(s"Table `${ident.name}` does not support writes.") + } + } + + override def abortStagedChanges(): Unit = catalog.dropTable(ident) + + override def commitStagedChanges(): Unit = {} + + override def name(): String = ident.name() + + override def schema(): StructType = table.schema() + + override def partitioning(): Array[Transform] = table.partitioning() + + override def capabilities(): java.util.Set[TableCapability] = table.capabilities() + + override def properties(): java.util.Map[String, String] = table.properties() +} diff --git a/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalog.scala b/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalog.scala index 57fa01306..2ada7d13c 100644 --- a/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalog.scala +++ b/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalog.scala @@ -3,86 +3,255 @@ */ package io.qbeast.spark.internal.sources.catalog -import io.qbeast.context.QbeastContext -import io.qbeast.spark.internal.QbeastOptions.checkQbeastProperties +import io.qbeast.core.model.QTableID import io.qbeast.spark.internal.sources.v2.QbeastTableImpl +import io.qbeast.spark.table.IndexedTableFactory import org.apache.hadoop.fs.Path -import org.apache.spark.sql.{AnalysisExceptionFactory, V1TableQbeast} -import org.apache.spark.sql.catalyst.catalog.{CatalogTableType} -import org.apache.spark.sql.connector.catalog.{DelegatingCatalogExtension, Identifier, Table} +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.analysis.CannotReplaceMissingTableException +import org.apache.spark.sql.catalyst.catalog._ +import org.apache.spark.sql.connector.catalog.{Identifier, Table, TableCatalog} import org.apache.spark.sql.connector.expressions.Transform +import org.apache.spark.sql.delta.catalog.DeltaTableV2 +import org.apache.spark.sql.delta.commands.TableCreationModes +import org.apache.spark.sql.execution.datasources.DataSource import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.{ + AnalysisExceptionFactory, + DataFrame, + SaveMode, + SparkSession, + SparkTransformUtils, + V1TableQbeast +} import java.util -import scala.collection.JavaConverters.mapAsScalaMapConverter +import java.util.Locale +import scala.collection.JavaConverters._ + +object QbeastCatalog { + + lazy val spark: SparkSession = SparkSession.active + + // TODO move definition of string to a generic object + def isQbeastTable(provider: Option[String]): Boolean = { + provider.isDefined && provider.get == "qbeast" + } + + def isPathTable(ident: Identifier): Boolean = { + try { + spark.sessionState.conf.runSQLonFile && hasQbeastNamespace(ident) && new Path( + ident.name()).isAbsolute + } catch { + case _: IllegalArgumentException => false + } + } + + def isPathTable(identifier: TableIdentifier): Boolean = { + isPathTable(Identifier.of(identifier.database.toArray, identifier.table)) + } + + /** Checks if a table already exists for the provided identifier. */ + def getExistingTableIfExists( + table: TableIdentifier, + existingSessionCatalog: SessionCatalog): Option[CatalogTable] = { + // If this is a path identifier, we cannot return an existing CatalogTable. The Create command + // will check the file system itself + if (isPathTable(table)) return None + val tableExists = existingSessionCatalog.tableExists(table) + if (tableExists) { + val oldTable = existingSessionCatalog.getTableMetadata(table) + if (oldTable.tableType == CatalogTableType.VIEW) { + throw AnalysisExceptionFactory.create( + s"$table is a view. You may not write data into a view.") + } + if (!isQbeastTable(oldTable.provider)) { + throw AnalysisExceptionFactory.create(s"$table is not a Qbeast table.") + } + Some(oldTable) + } else { + None + } + } + + /** + * Checks if the identifier has namespace of Qbeast + * @param ident + * @return + */ + def hasQbeastNamespace(ident: Identifier): Boolean = { + ident.namespace().length == 1 && ident.name.toLowerCase(Locale.ROOT) == "qbeast" + } + + /** + * Creates a Table with Qbeast + * @param ident the Identifier of the table + * @param schema the schema of the table + * @param partitions the partitions of the table, if any + * @param allTableProperties all the table properties + * @param writeOptions the write properties of the table + * @param dataFrame the dataframe to write, if any + * @param tableCreationMode the creation mode (could be CREATE, REPLACE or CREATE OR REPLACE) + * @param tableFactory the indexed table factory + * @param existingSessionCatalog the existing session catalog + */ -class QbeastCatalog extends DelegatingCatalogExtension { + def createQbeastTable( + ident: Identifier, + schema: StructType, + partitions: Array[Transform], + allTableProperties: util.Map[String, String], + writeOptions: Map[String, String], + dataFrame: Option[DataFrame], + tableCreationMode: TableCreationModes.CreationMode, + tableFactory: IndexedTableFactory, + existingSessionCatalog: SessionCatalog): Unit = { + // These two keys are tableProperties in data source v2 but not in v1, so we have to filter + // them out. Otherwise property consistency checks will fail. + val tableProperties = allTableProperties.asScala.filterKeys { + case TableCatalog.PROP_LOCATION => false + case TableCatalog.PROP_PROVIDER => false + case TableCatalog.PROP_COMMENT => false + case TableCatalog.PROP_OWNER => false + case TableCatalog.PROP_EXTERNAL => false + case "path" => false + case _ => true + } + + val isPathTable = QbeastCatalog.isPathTable(ident) + + if (isPathTable + && allTableProperties.containsKey("location") + // The location property can be qualified and different from the path in the identifier, so + // we check `endsWith` here. + && Option(allTableProperties.get("location")).exists(!_.endsWith(ident.name()))) { + throw AnalysisExceptionFactory.create( + s"CREATE TABLE contains two different locations: ${ident.name()} " + + s"and ${allTableProperties.get("location")}.") + } + + val location = if (isPathTable) { + Option(ident.name()) + } else { + Option(allTableProperties.get("location")) + } + val id = TableIdentifier(ident.name(), ident.namespace().lastOption) + val locUriOpt = location.map(CatalogUtils.stringToURI) + val existingTableOpt = QbeastCatalog.getExistingTableIfExists(id, existingSessionCatalog) + val loc = locUriOpt + .orElse(existingTableOpt.flatMap(_.storage.locationUri)) + .getOrElse(existingSessionCatalog.defaultTablePath(id)) + + val storage = DataSource + .buildStorageFormatFromOptions(writeOptions) + .copy(locationUri = Option(loc)) + val tableType = + if (location.isDefined) CatalogTableType.EXTERNAL else CatalogTableType.MANAGED + val commentOpt = Option(allTableProperties.get("comment")) + val (partitionColumns, bucketSpec) = SparkTransformUtils.convertTransforms(partitions) + + val table = new CatalogTable( + identifier = id, + tableType = tableType, + storage = storage, + schema = schema, + provider = Some("qbeast"), + partitionColumnNames = partitionColumns, + bucketSpec = bucketSpec, + properties = tableProperties.toMap, + comment = commentOpt) - private val tableFactory = QbeastContext.indexedTableFactory + val append = tableCreationMode.mode == SaveMode.Append + dataFrame.map { df => + tableFactory + .getIndexedTable(QTableID(loc.toString)) + .save(df, allTableProperties.asScala.toMap, append) + } + + updateCatalog(tableCreationMode, table, isPathTable, existingTableOpt, existingSessionCatalog) + } /** - * Checks if the Table is created with Qbeast Format - * @param properties the map of properties of the table - * @return a boolean set to true in the case it's a Qbeast formatted table + * Based on DeltaCatalog updateCatalog private method, + * it maintains the consistency of creating a table + * calling the spark session catalog. + * @param operation + * @param table + * @param isPathTable + * @param existingTableOpt + * @param existingSessionCatalog */ - protected def isQbeastTableProvider(properties: Map[String, String]): Boolean = { - properties.get("provider") match { - case Some("qbeast") => true - case _ => false + def updateCatalog( + operation: TableCreationModes.CreationMode, + table: CatalogTable, + isPathTable: Boolean, + existingTableOpt: Option[CatalogTable], + existingSessionCatalog: SessionCatalog): Unit = { + + operation match { + case _ if isPathTable => // do nothing + case TableCreationModes.Create => + existingSessionCatalog.createTable( + table, + ignoreIfExists = existingTableOpt.isDefined, + validateLocation = false) + case TableCreationModes.Replace | TableCreationModes.CreateOrReplace + if existingTableOpt.isDefined => + existingSessionCatalog.alterTable(table) + case TableCreationModes.Replace => + val ident = Identifier.of(table.identifier.database.toArray, table.identifier.table) + throw new CannotReplaceMissingTableException(ident) + case TableCreationModes.CreateOrReplace => + existingSessionCatalog.createTable( + table, + ignoreIfExists = false, + validateLocation = false) } } /** - * Creates a qbeast table based on the underlying table + * Loads a qbeast table based on the underlying table * @param table the underlying table * @return a Table with Qbeast information and implementations */ - protected def qbeastTable(table: Table): Table = { + def loadQbeastTable(table: Table, tableFactory: IndexedTableFactory): Table = { - val prop = table.properties().asScala.toMap + val prop = table.properties() val schema = table.schema() - if (isQbeastTableProvider(prop)) { - table match { - case V1TableQbeast(t) => - checkQbeastProperties(prop) - val catalogTable = t.v1Table - - val path: String = if (catalogTable.tableType == CatalogTableType.EXTERNAL) { - // If it's an EXTERNAL TABLE, we can find the path through the Storage Properties - catalogTable.storage.locationUri.get.toString - } else if (catalogTable.tableType == CatalogTableType.MANAGED) { - // If it's a MANAGED TABLE, the location is set in the former catalogTable - catalogTable.location.toString - } else { - // Otherwise, TODO - throw AnalysisExceptionFactory.create("No path found for table " + table.name()) - } - new QbeastTableImpl( - new Path(path), - prop, - Some(schema), - Some(catalogTable), - tableFactory) - - case _ => table - } - } else table - } + table match { + case V1TableQbeast(t) => + val catalogTable = t.v1Table - override def loadTable(ident: Identifier): Table = { - val superTable = super.loadTable(ident) - qbeastTable(superTable) - } + val path: String = if (catalogTable.tableType == CatalogTableType.EXTERNAL) { + // If it's an EXTERNAL TABLE, we can find the path through the Storage Properties + catalogTable.storage.locationUri.get.toString + } else if (catalogTable.tableType == CatalogTableType.MANAGED) { + // If it's a MANAGED TABLE, the location is set in the former catalogTable + catalogTable.location.toString + } else { + // Otherwise, TODO + throw AnalysisExceptionFactory.create("No path found for table " + table.name()) + } + new QbeastTableImpl( + catalogTable.identifier.identifier, + new Path(path), + prop.asScala.toMap, + Some(schema), + Some(catalogTable), + tableFactory) - override def createTable( - ident: Identifier, - schema: StructType, - partitions: Array[Transform], - properties: util.Map[String, String]): Table = { + case DeltaTableV2(_, path, catalogTable, tableIdentifier, _, options, _) => + new QbeastTableImpl( + tableIdentifier.get, + path, + options, + Some(schema), + catalogTable, + tableFactory) - val superTable = super.createTable(ident, schema, partitions, properties) - qbeastTable(superTable) + case _ => table + } } } diff --git a/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastDeltaCatalog.scala b/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastDeltaCatalog.scala new file mode 100644 index 000000000..9f3a49e58 --- /dev/null +++ b/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastDeltaCatalog.scala @@ -0,0 +1,102 @@ +/* + * Copyright 2021 Qbeast Analytics, S.L. + */ +package io.qbeast.spark.internal.sources.catalog + +import io.qbeast.context.QbeastContext +import io.qbeast.spark.internal.QbeastOptions.checkQbeastProperties +import io.qbeast.spark.internal.sources.v2.QbeastStagedTableImpl +import org.apache.spark.sql.connector.catalog.{Identifier, StagedTable, Table} +import org.apache.spark.sql.connector.expressions.Transform +import org.apache.spark.sql.delta.catalog.DeltaCatalog +import org.apache.spark.sql.delta.commands.TableCreationModes +import org.apache.spark.sql.types.StructType + +import java.util +import scala.collection.JavaConverters._ + +class QbeastDeltaCatalog extends DeltaCatalog { + + private val tableFactory = QbeastContext.indexedTableFactory + + override def loadTable(ident: Identifier): Table = { + val superTable = super.loadTable(ident) + if (QbeastCatalog.isQbeastTable(superTable.properties().asScala.get("provider"))) { + QbeastCatalog.loadQbeastTable(superTable, tableFactory) + } else { + superTable + } + } + + override def createTable( + ident: Identifier, + schema: StructType, + partitions: Array[Transform], + properties: util.Map[String, String]): Table = { + + val superTable = super.createTable(ident, schema, partitions, properties) + if (QbeastCatalog.isQbeastTable(superTable.properties().asScala.get("provider"))) { + checkQbeastProperties(properties.asScala.toMap) + QbeastCatalog.loadQbeastTable(superTable, tableFactory) + } else { + super.createTable(ident, schema, partitions, properties) + } + + } + + override def stageReplace( + ident: Identifier, + schema: StructType, + partitions: Array[Transform], + properties: util.Map[String, String]): StagedTable = { + if (QbeastCatalog.isQbeastTable(properties.asScala.get("provider"))) { + new QbeastStagedTableImpl( + ident, + schema, + partitions, + TableCreationModes.Replace, + properties, + tableFactory) + } else { + super.stageReplace(ident, schema, partitions, properties) + } + } + + override def stageCreateOrReplace( + ident: Identifier, + schema: StructType, + partitions: Array[Transform], + properties: util.Map[String, String]): StagedTable = { + if (QbeastCatalog.isQbeastTable(properties.asScala.get("provider"))) { + new QbeastStagedTableImpl( + ident, + schema, + partitions, + TableCreationModes.CreateOrReplace, + properties, + tableFactory) + } else { + super.stageCreateOrReplace(ident, schema, partitions, properties) + + } + } + + override def stageCreate( + ident: Identifier, + schema: StructType, + partitions: Array[Transform], + properties: util.Map[String, String]): StagedTable = { + if (QbeastCatalog.isQbeastTable(properties.asScala.get("provider"))) { + new QbeastStagedTableImpl( + ident, + schema, + partitions, + TableCreationModes.Create, + properties, + tableFactory) + } else { + super.stageCreate(ident, schema, partitions, properties) + } + } + +} diff --git a/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastStagedTableImpl.scala b/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastStagedTableImpl.scala new file mode 100644 index 000000000..b049dee47 --- /dev/null +++ b/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastStagedTableImpl.scala @@ -0,0 +1,113 @@ +/* + * Copyright 2021 Qbeast Analytics, S.L. + */ +package io.qbeast.spark.internal.sources.v2 + +import io.qbeast.spark.internal.QbeastOptions.checkQbeastProperties +import io.qbeast.spark.internal.sources.catalog.QbeastCatalog +import io.qbeast.spark.table.IndexedTableFactory +import org.apache.spark.sql.catalyst.catalog.{SessionCatalog} +import org.apache.spark.sql.connector.catalog.TableCapability.V1_BATCH_WRITE +import org.apache.spark.sql.connector.catalog.{ + Identifier, + StagedTable, + SupportsWrite, + TableCapability +} +import org.apache.spark.sql.connector.expressions.Transform +import org.apache.spark.sql.connector.write.{LogicalWriteInfo, V1Write, Write, WriteBuilder} +import org.apache.spark.sql.delta.commands.TableCreationModes +import org.apache.spark.sql.sources.InsertableRelation +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.{DataFrame, SparkSession} + +import java.util +import scala.collection.JavaConverters._ + +/** + * An StagedTable allows Atomic CREATE TABLE AS SELECT / REPLACE TABLE AS SELECT + * This table should implement SupportsWrite + */ +private[sources] class QbeastStagedTableImpl( + ident: Identifier, + schema: StructType, + partitions: Array[Transform], + tableCreationMode: TableCreationModes.CreationMode, + override val properties: util.Map[String, String], + private val tableFactory: IndexedTableFactory) + extends StagedTable + with SupportsWrite { + + lazy val spark: SparkSession = SparkSession.active + lazy val catalog: SessionCatalog = spark.sessionState.catalog + + // Variables for the creation of the writeBuilder + var writeOptions: Map[String, String] = Map.empty + var dataFrame: Option[DataFrame] = None + + override def commitStagedChanges(): Unit = { + + val props = new util.HashMap[String, String]() + + // Options passed in through the SQL API will show up both with an "option." prefix and + // without in Spark 3.1, so we need to remove those from the properties + val optionsThroughProperties = properties.asScala.collect { + case (k, _) if k.startsWith("option.") => k.stripPrefix("option.") + }.toSet + val sqlWriteOptions = new util.HashMap[String, String]() + properties.asScala.foreach { case (k, v) => + if (!k.startsWith("option.") && !optionsThroughProperties.contains(k)) { + // Do not add to properties + props.put(k, v) + } else if (optionsThroughProperties.contains(k)) { + sqlWriteOptions.put(k, v) + } + } + if (writeOptions.isEmpty && !sqlWriteOptions.isEmpty) { + writeOptions = sqlWriteOptions.asScala.toMap + } + // Drop the delta configuration check, + // we pass all the writeOptions to the properties as well + writeOptions.foreach { case (k, v) => props.put(k, v) } + + checkQbeastProperties(props.asScala.toMap) + + QbeastCatalog.createQbeastTable( + ident, + schema, + partitions, + props, + writeOptions, + dataFrame, + tableCreationMode, + tableFactory, + catalog) + + } + + override def abortStagedChanges(): Unit = {} + + override def name(): String = ident.name() + + override def schema(): StructType = schema + + override def capabilities(): util.Set[TableCapability] = Set(V1_BATCH_WRITE).asJava + + override def newWriteBuilder(info: LogicalWriteInfo): WriteBuilder = + new WriteBuilder { + + override def build(): Write = new V1Write { + + override def toInsertableRelation: InsertableRelation = { + new InsertableRelation { + def insert(data: DataFrame, overwrite: Boolean): Unit = { + dataFrame = Some(data) + } + } + } + + } + + } + +} diff --git a/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastTableImpl.scala b/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastTableImpl.scala index dfd13d30a..2e2cb23ef 100644 --- a/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastTableImpl.scala +++ b/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastTableImpl.scala @@ -28,6 +28,7 @@ import scala.collection.JavaConverters._ * @param tableFactory the IndexedTable Factory */ class QbeastTableImpl private[sources] ( + identifier: String, path: Path, options: Map[String, String], schema: Option[StructType] = None, @@ -43,7 +44,7 @@ class QbeastTableImpl private[sources] ( private val indexedTable = tableFactory.getIndexedTable(tableId) - override def name(): String = tableId.id + override def name(): String = identifier override def schema(): StructType = { if (schema.isDefined) schema.get diff --git a/src/main/scala/org/apache/spark/sql/SparkTransformUtils.scala b/src/main/scala/org/apache/spark/sql/SparkTransformUtils.scala new file mode 100644 index 000000000..64f96e40b --- /dev/null +++ b/src/main/scala/org/apache/spark/sql/SparkTransformUtils.scala @@ -0,0 +1,84 @@ +/* + * Copyright 2021 Qbeast Analytics, S.L. + */ +package org.apache.spark.sql + +import org.apache.spark.sql.catalyst.catalog.BucketSpec +import org.apache.spark.sql.connector.expressions.{ + FieldReference, + IdentityTransform, + Literal, + NamedReference, + Transform +} +import org.apache.spark.sql.types.IntegerType + +import scala.collection.mutable + +object SparkTransformUtils { + + /** + * Copy of V2 convertTransforms + * @param partitions + * @return + */ + def convertTransforms(partitions: Seq[Transform]): (Seq[String], Option[BucketSpec]) = { + val identityCols = new mutable.ArrayBuffer[String] + var bucketSpec = Option.empty[BucketSpec] + + partitions.map { + case IdentityTransform(FieldReference(Seq(col))) => + identityCols += col + + case BucketTransform(numBuckets, bucketCols, sortCols) => + bucketSpec = Some( + BucketSpec( + numBuckets, + bucketCols.map(_.fieldNames.head), + sortCols.map(_.fieldNames.head))) + + case _ => + throw AnalysisExceptionFactory.create( + "Operation not supported, non partition based transformation") + } + + (identityCols, bucketSpec) + } + +} + +object BucketTransform { + + def unapply(transform: Transform): Option[(Int, Seq[NamedReference], Seq[NamedReference])] = { + val arguments = transform.arguments() + if (transform.name() == "sorted_bucket") { + var posOfLit: Int = -1 + var numOfBucket: Int = -1 + arguments.zipWithIndex.foreach { + case (literal: Literal[_], i) if literal.dataType() == IntegerType => + numOfBucket = literal.value().asInstanceOf[Integer] + posOfLit = i + case _ => + } + Some( + ( + numOfBucket, + arguments.take(posOfLit).map(_.asInstanceOf[NamedReference]), + arguments.drop(posOfLit + 1).map(_.asInstanceOf[NamedReference]))) + } else if (transform.name() == "bucket") { + val numOfBucket = arguments(0) match { + case literal: Literal[_] if literal.dataType() == IntegerType => + literal.value().asInstanceOf[Integer] + case _ => throw new IllegalStateException("invalid bucket transform") + } + Some( + ( + numOfBucket, + arguments.drop(1).map(_.asInstanceOf[NamedReference]), + Seq.empty[FieldReference])) + } else { + None + } + } + +} diff --git a/src/test/scala/io/qbeast/context/QbeastConfigTest.scala b/src/test/scala/io/qbeast/context/QbeastConfigTest.scala index 400fcb060..048972072 100644 --- a/src/test/scala/io/qbeast/context/QbeastConfigTest.scala +++ b/src/test/scala/io/qbeast/context/QbeastConfigTest.scala @@ -1,7 +1,6 @@ package io.qbeast.context import io.qbeast.spark.QbeastIntegrationTestSpec -import org.apache.spark.SparkConf import org.apache.spark.qbeast.config import org.apache.spark.sql.SparkSession import org.scalatest.flatspec.AnyFlatSpec @@ -15,7 +14,7 @@ class QbeastConfigTest extends AnyFlatSpec with Matchers with QbeastIntegrationT } it should "change configurations accordingly" in withExtendedSpark( - new SparkConf() + sparkConfWithSqlAndCatalog .set("spark.qbeast.index.defaultCubeSize", "1000") .set("spark.qbeast.index.cubeWeightsBufferCapacity", "1000") .set("spark.qbeast.index.numberOfRetries", "10")) { _ => @@ -32,7 +31,7 @@ class QbeastConfigTest extends AnyFlatSpec with Matchers with QbeastIntegrationT } it should "be defined" in withExtendedSpark( - new SparkConf() + sparkConfWithSqlAndCatalog .set("spark.qbeast.keeper", "myKeeper")) { spark => val keeperString = spark.sparkContext.getConf.getOption("spark.qbeast.keeper") keeperString.get shouldBe "myKeeper" diff --git a/src/test/scala/io/qbeast/docs/DocumentationTests.scala b/src/test/scala/io/qbeast/docs/DocumentationTests.scala index a47567003..2ac2e0b43 100644 --- a/src/test/scala/io/qbeast/docs/DocumentationTests.scala +++ b/src/test/scala/io/qbeast/docs/DocumentationTests.scala @@ -8,7 +8,7 @@ import org.scalatest.AppendedClues.convertToClueful class DocumentationTests extends QbeastIntegrationTestSpec { - val config: SparkConf = new SparkConf().set( + val config: SparkConf = sparkConfWithSqlAndCatalog.set( "spark.hadoop.fs.s3a.aws.credentials.provider", "org.apache.hadoop.fs.s3a.AnonymousAWSCredentialsProvider") diff --git a/src/test/scala/io/qbeast/spark/QbeastIntegrationTestSpec.scala b/src/test/scala/io/qbeast/spark/QbeastIntegrationTestSpec.scala index 1289f61ba..9b985f8eb 100644 --- a/src/test/scala/io/qbeast/spark/QbeastIntegrationTestSpec.scala +++ b/src/test/scala/io/qbeast/spark/QbeastIntegrationTestSpec.scala @@ -35,6 +35,15 @@ trait QbeastIntegrationTestSpec extends AnyFlatSpec with Matchers with DatasetCo // This reduce the verbosity of Spark Logger.getLogger("org.apache").setLevel(Level.WARN) + // Spark Configuration + // Including Session Extensions and Catalog + def sparkConfWithSqlAndCatalog: SparkConf = new SparkConf() + .setMaster("local[8]") + .set("spark.sql.extensions", "io.qbeast.spark.internal.QbeastSparkSessionExtension") + .set( + SQLConf.V2_SESSION_CATALOG_IMPLEMENTATION.key, + "io.qbeast.spark.internal.sources.catalog.QbeastDeltaCatalog") + def loadTestData(spark: SparkSession): DataFrame = spark.read .format("csv") .option("header", "true") @@ -66,16 +75,10 @@ trait QbeastIntegrationTestSpec extends AnyFlatSpec with Matchers with DatasetCo */ def withExtendedSpark[T](sparkConf: SparkConf = new SparkConf())( testCode: SparkSession => T): T = { - val conf = sparkConf - .setMaster("local[8]") - .set("spark.sql.extensions", "io.qbeast.spark.internal.QbeastSparkSessionExtension") - .set( - SQLConf.V2_SESSION_CATALOG_IMPLEMENTATION.key, - "io.qbeast.spark.internal.sources.catalog.QbeastCatalog") val spark = SparkSession .builder() .appName("QbeastDataSource") - .config(conf) + .config(sparkConf) .getOrCreate() try { testCode(spark) @@ -96,27 +99,15 @@ trait QbeastIntegrationTestSpec extends AnyFlatSpec with Matchers with DatasetCo * @return */ def withSpark[T](testCode: SparkSession => T): T = { - // Spark Configuration - // Including Session Extensions and Catalog - val conf = new SparkConf() - .setMaster("local[8]") - .set("spark.sql.extensions", "io.qbeast.spark.internal.QbeastSparkSessionExtension") - .set( - SQLConf.V2_SESSION_CATALOG_IMPLEMENTATION.key, - "io.qbeast.spark.internal.sources.catalog.QbeastCatalog") - - val spark = SparkSession - .builder() - .config(conf) - .appName("QbeastDataSource") - .getOrCreate() - try { - testCode(spark) - } finally { - spark.close() - } + withExtendedSpark(sparkConfWithSqlAndCatalog)(testCode) } + /** + * Runs code with a Temporary Directory. After execution, the content of the directory is deleted. + * @param testCode + * @tparam T + * @return + */ def withTmpDir[T](testCode: String => T): T = { val directory = Files.createTempDirectory("qb-testing") try { @@ -159,14 +150,34 @@ trait QbeastIntegrationTestSpec extends AnyFlatSpec with Matchers with DatasetCo } } + /** + * Runs code with QbeastContext and a Temporary Directory. + * @param testCode + * @tparam T + * @return + */ + def withQbeastContextSparkAndTmpDir[T](testCode: (SparkSession, String) => T): T = withTmpDir(tmpDir => withQbeastAndSparkContext()(spark => testCode(spark, tmpDir))) + /** + * Runs code with Warehouse/Catalog extensions + * @param testCode the code to reproduce + * @tparam T + * @return + */ def withQbeastContextSparkAndTmpWarehouse[T](testCode: (SparkSession, String) => T): T = withTmpDir(tmpDir => - withExtendedSpark(new SparkConf().set("spark.sql.warehouse.dir", tmpDir))(spark => - testCode(spark, tmpDir))) + withExtendedSpark( + sparkConfWithSqlAndCatalog + .set("spark.sql.warehouse.dir", tmpDir))(spark => testCode(spark, tmpDir))) + /** + * Runs code with OTreeAlgorithm configuration + * @param code + * @tparam T + * @return + */ def withOTreeAlgorithm[T](code: IndexManager[DataFrame] => T): T = { code(SparkOTreeManager) } diff --git a/src/test/scala/io/qbeast/spark/internal/sources/QbeastDeltaCatalogTest.scala b/src/test/scala/io/qbeast/spark/internal/sources/QbeastDeltaCatalogTest.scala new file mode 100644 index 000000000..a2890e942 --- /dev/null +++ b/src/test/scala/io/qbeast/spark/internal/sources/QbeastDeltaCatalogTest.scala @@ -0,0 +1,44 @@ +package io.qbeast.spark.internal.sources + +import io.qbeast.TestClasses.Student +import io.qbeast.spark.QbeastIntegrationTestSpec +import org.apache.spark.sql.{DataFrame, SparkSession} + +import scala.util.Random + +class QbeastDeltaCatalogTest extends QbeastIntegrationTestSpec { + + private val students = 1.to(10).map(i => Student(i, i.toString, Random.nextInt())) + + private def createTestData(spark: SparkSession): DataFrame = { + import spark.implicits._ + students.toDF() + } + + "QbeastDeltaCatalogTest" should + "coexist with Delta tables" in withQbeastContextSparkAndTmpWarehouse((spark, tmpDir) => { + + val data = createTestData(spark) + + data.write.format("delta").saveAsTable("delta_table") // delta catalog + + data.write + .format("qbeast") + .option("columnsToIndex", "id") + .saveAsTable("qbeast_table") // qbeast catalog + + val tables = spark.sessionState.catalog.listTables("default") + tables.size shouldBe 2 + + val deltaTable = spark.read.table("delta_table") + val qbeastTable = spark.read.table("qbeast_table") + + assertSmallDatasetEquality( + deltaTable, + qbeastTable, + orderedComparison = false, + ignoreNullable = true) + + }) + +} diff --git a/src/test/scala/io/qbeast/spark/utils/QbeastCompactionIntegrationTest.scala b/src/test/scala/io/qbeast/spark/utils/QbeastCompactionIntegrationTest.scala index 5aee254b3..23f851c76 100644 --- a/src/test/scala/io/qbeast/spark/utils/QbeastCompactionIntegrationTest.scala +++ b/src/test/scala/io/qbeast/spark/utils/QbeastCompactionIntegrationTest.scala @@ -3,7 +3,6 @@ package io.qbeast.spark.utils import io.qbeast.core.model.QTableID import io.qbeast.spark.delta.{DeltaQbeastSnapshot, SparkDeltaMetadataManager} import io.qbeast.spark.{QbeastIntegrationTestSpec, QbeastTable} -import org.apache.spark.SparkConf import org.apache.spark.sql.{AnalysisException, DataFrame} import org.apache.spark.sql.delta.DeltaLog import org.apache.spark.sql.functions._ @@ -18,43 +17,44 @@ class QbeastCompactionIntegrationTest extends QbeastIntegrationTestSpec { "Compaction command" should "reduce the number of files" in withExtendedSparkAndTmpDir( - new SparkConf().set("spark.qbeast.compact.minFileSize", "1")) { (spark, tmpDir) => - { + sparkConfWithSqlAndCatalog.set("spark.qbeast.compact.minFileSize", "1")) { + (spark, tmpDir) => + { - val data = loadTestData(spark) + val data = loadTestData(spark) - // Creating four batches of 20000 elements each one - // So they all go to the root cube - // and we can compact them later - val limit = 20000 - val numBatches = 4 - val batch = data.limit(limit) + // Creating four batches of 20000 elements each one + // So they all go to the root cube + // and we can compact them later + val limit = 20000 + val numBatches = 4 + val batch = data.limit(limit) - // Write four batches - writeTestDataInBatches(batch, tmpDir, numBatches) + // Write four batches + writeTestDataInBatches(batch, tmpDir, numBatches) - val indexed = spark.read.format("qbeast").load(tmpDir) - val originalNumOfFiles = indexed.select(input_file_name()).distinct().count() + val indexed = spark.read.format("qbeast").load(tmpDir) + val originalNumOfFiles = indexed.select(input_file_name()).distinct().count() - val qbeastTable = QbeastTable.forPath(spark, tmpDir) - qbeastTable.compact() + val qbeastTable = QbeastTable.forPath(spark, tmpDir) + qbeastTable.compact() - val finalNumOfFiles = indexed.select(input_file_name()).distinct().count() - finalNumOfFiles shouldBe <(originalNumOfFiles) - finalNumOfFiles shouldBe 1L + val finalNumOfFiles = indexed.select(input_file_name()).distinct().count() + finalNumOfFiles shouldBe <(originalNumOfFiles) + finalNumOfFiles shouldBe 1L - // Test if the dataframe is correctly loaded - val deltaData = spark.read.format("delta").load(tmpDir) - indexed.count() shouldBe (limit * numBatches) - assertLargeDatasetEquality(indexed, deltaData, orderedComparison = false) + // Test if the dataframe is correctly loaded + val deltaData = spark.read.format("delta").load(tmpDir) + indexed.count() shouldBe (limit * numBatches) + assertLargeDatasetEquality(indexed, deltaData, orderedComparison = false) - } + } } it should "compact in more than one file if MAX_FILE_SIZE_COMPACTION " + "is exceeded" in withExtendedSparkAndTmpDir( - new SparkConf() + sparkConfWithSqlAndCatalog .set("spark.qbeast.compact.minFileSize", "1") .set("spark.qbeast.compact.maxFileSize", "2000000")) { (spark, tmpDir) => { @@ -111,7 +111,7 @@ class QbeastCompactionIntegrationTest extends QbeastIntegrationTestSpec { } it should "respect cube information" in withExtendedSparkAndTmpDir( - new SparkConf().set("spark.qbeast.compact.minFileSize", "1"))((spark, tmpDir) => { + sparkConfWithSqlAndCatalog.set("spark.qbeast.compact.minFileSize", "1"))((spark, tmpDir) => { val data = loadTestData(spark) @@ -139,7 +139,7 @@ class QbeastCompactionIntegrationTest extends QbeastIntegrationTestSpec { }) it should "compact the latest revision available" in withExtendedSparkAndTmpDir( - new SparkConf().set("spark.qbeast.compact.minFileSize", "1"))((spark, tmpDir) => { + sparkConfWithSqlAndCatalog.set("spark.qbeast.compact.minFileSize", "1"))((spark, tmpDir) => { val data = loadTestData(spark) @@ -178,7 +178,7 @@ class QbeastCompactionIntegrationTest extends QbeastIntegrationTestSpec { }) it should "compact the specified revision" in withExtendedSparkAndTmpDir( - new SparkConf().set("spark.qbeast.compact.minFileSize", "1"))((spark, tmpDir) => { + sparkConfWithSqlAndCatalog.set("spark.qbeast.compact.minFileSize", "1"))((spark, tmpDir) => { val data = loadTestData(spark) @@ -217,7 +217,7 @@ class QbeastCompactionIntegrationTest extends QbeastIntegrationTestSpec { }) it should "not compact if the revision does not exists" in withExtendedSparkAndTmpDir( - new SparkConf().set("spark.qbeast.compact.minFileSize", "1"))((spark, tmpDir) => { + sparkConfWithSqlAndCatalog.set("spark.qbeast.compact.minFileSize", "1"))((spark, tmpDir) => { val data = loadTestData(spark) From 7f03e96b32a23c7a9bd42c7feea1cfb365b00927 Mon Sep 17 00:00:00 2001 From: osopardo1 Date: Tue, 13 Sep 2022 15:44:56 +0200 Subject: [PATCH 30/49] Renamed QbeastCatalog to QbeastCatalogUtils --- ...beastCatalog.scala => QbeastCatalogUtils.scala} | 6 +++--- .../sources/catalog/QbeastDeltaCatalog.scala | 14 +++++++------- .../sources/v2/QbeastStagedTableImpl.scala | 6 +++--- 3 files changed, 13 insertions(+), 13 deletions(-) rename src/main/scala/io/qbeast/spark/internal/sources/catalog/{QbeastCatalog.scala => QbeastCatalogUtils.scala} (97%) diff --git a/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalog.scala b/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalogUtils.scala similarity index 97% rename from src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalog.scala rename to src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalogUtils.scala index 2ada7d13c..1fe03f65e 100644 --- a/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalog.scala +++ b/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalogUtils.scala @@ -29,7 +29,7 @@ import java.util import java.util.Locale import scala.collection.JavaConverters._ -object QbeastCatalog { +object QbeastCatalogUtils { lazy val spark: SparkSession = SparkSession.active @@ -118,7 +118,7 @@ object QbeastCatalog { case _ => true } - val isPathTable = QbeastCatalog.isPathTable(ident) + val isPathTable = QbeastCatalogUtils.isPathTable(ident) if (isPathTable && allTableProperties.containsKey("location") @@ -137,7 +137,7 @@ object QbeastCatalog { } val id = TableIdentifier(ident.name(), ident.namespace().lastOption) val locUriOpt = location.map(CatalogUtils.stringToURI) - val existingTableOpt = QbeastCatalog.getExistingTableIfExists(id, existingSessionCatalog) + val existingTableOpt = QbeastCatalogUtils.getExistingTableIfExists(id, existingSessionCatalog) val loc = locUriOpt .orElse(existingTableOpt.flatMap(_.storage.locationUri)) .getOrElse(existingSessionCatalog.defaultTablePath(id)) diff --git a/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastDeltaCatalog.scala b/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastDeltaCatalog.scala index 9f3a49e58..1281c507e 100644 --- a/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastDeltaCatalog.scala +++ b/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastDeltaCatalog.scala @@ -21,8 +21,8 @@ class QbeastDeltaCatalog extends DeltaCatalog { override def loadTable(ident: Identifier): Table = { val superTable = super.loadTable(ident) - if (QbeastCatalog.isQbeastTable(superTable.properties().asScala.get("provider"))) { - QbeastCatalog.loadQbeastTable(superTable, tableFactory) + if (QbeastCatalogUtils.isQbeastTable(superTable.properties().asScala.get("provider"))) { + QbeastCatalogUtils.loadQbeastTable(superTable, tableFactory) } else { superTable } @@ -35,9 +35,9 @@ class QbeastDeltaCatalog extends DeltaCatalog { properties: util.Map[String, String]): Table = { val superTable = super.createTable(ident, schema, partitions, properties) - if (QbeastCatalog.isQbeastTable(superTable.properties().asScala.get("provider"))) { + if (QbeastCatalogUtils.isQbeastTable(superTable.properties().asScala.get("provider"))) { checkQbeastProperties(properties.asScala.toMap) - QbeastCatalog.loadQbeastTable(superTable, tableFactory) + QbeastCatalogUtils.loadQbeastTable(superTable, tableFactory) } else { super.createTable(ident, schema, partitions, properties) } @@ -49,7 +49,7 @@ class QbeastDeltaCatalog extends DeltaCatalog { schema: StructType, partitions: Array[Transform], properties: util.Map[String, String]): StagedTable = { - if (QbeastCatalog.isQbeastTable(properties.asScala.get("provider"))) { + if (QbeastCatalogUtils.isQbeastTable(properties.asScala.get("provider"))) { new QbeastStagedTableImpl( ident, schema, @@ -67,7 +67,7 @@ class QbeastDeltaCatalog extends DeltaCatalog { schema: StructType, partitions: Array[Transform], properties: util.Map[String, String]): StagedTable = { - if (QbeastCatalog.isQbeastTable(properties.asScala.get("provider"))) { + if (QbeastCatalogUtils.isQbeastTable(properties.asScala.get("provider"))) { new QbeastStagedTableImpl( ident, schema, @@ -86,7 +86,7 @@ class QbeastDeltaCatalog extends DeltaCatalog { schema: StructType, partitions: Array[Transform], properties: util.Map[String, String]): StagedTable = { - if (QbeastCatalog.isQbeastTable(properties.asScala.get("provider"))) { + if (QbeastCatalogUtils.isQbeastTable(properties.asScala.get("provider"))) { new QbeastStagedTableImpl( ident, schema, diff --git a/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastStagedTableImpl.scala b/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastStagedTableImpl.scala index b049dee47..eec549000 100644 --- a/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastStagedTableImpl.scala +++ b/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastStagedTableImpl.scala @@ -4,9 +4,9 @@ package io.qbeast.spark.internal.sources.v2 import io.qbeast.spark.internal.QbeastOptions.checkQbeastProperties -import io.qbeast.spark.internal.sources.catalog.QbeastCatalog +import io.qbeast.spark.internal.sources.catalog.QbeastCatalogUtils import io.qbeast.spark.table.IndexedTableFactory -import org.apache.spark.sql.catalyst.catalog.{SessionCatalog} +import org.apache.spark.sql.catalyst.catalog.SessionCatalog import org.apache.spark.sql.connector.catalog.TableCapability.V1_BATCH_WRITE import org.apache.spark.sql.connector.catalog.{ Identifier, @@ -72,7 +72,7 @@ private[sources] class QbeastStagedTableImpl( checkQbeastProperties(props.asScala.toMap) - QbeastCatalog.createQbeastTable( + QbeastCatalogUtils.createQbeastTable( ident, schema, partitions, From 4923773bd5ab18326594e16eafe146b37b32f5d8 Mon Sep 17 00:00:00 2001 From: osopardo1 Date: Tue, 13 Sep 2022 16:16:42 +0200 Subject: [PATCH 31/49] Add scaladocs and comments --- .../sources/catalog/DefaultStagedTable.scala | 6 +++++ .../sources/catalog/QbeastCatalogUtils.scala | 27 ++++++++++++++----- .../sources/catalog/QbeastDeltaCatalog.scala | 24 +++++++++++++---- .../sources/v2/QbeastStagedTableImpl.scala | 2 ++ 4 files changed, 48 insertions(+), 11 deletions(-) diff --git a/src/main/scala/io/qbeast/spark/internal/sources/catalog/DefaultStagedTable.scala b/src/main/scala/io/qbeast/spark/internal/sources/catalog/DefaultStagedTable.scala index 32d81ea21..0690583e0 100644 --- a/src/main/scala/io/qbeast/spark/internal/sources/catalog/DefaultStagedTable.scala +++ b/src/main/scala/io/qbeast/spark/internal/sources/catalog/DefaultStagedTable.scala @@ -16,6 +16,12 @@ import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.connector.write.{LogicalWriteInfo, WriteBuilder} import org.apache.spark.sql.types.StructType +/** + * A default StagedTable + * @param ident the identifier + * @param table the Table + * @param catalog the Catalog + */ private[catalog] case class DefaultStagedTable( ident: Identifier, table: Table, diff --git a/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalogUtils.scala b/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalogUtils.scala index 1fe03f65e..b95be78a5 100644 --- a/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalogUtils.scala +++ b/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalogUtils.scala @@ -29,15 +29,30 @@ import java.util import java.util.Locale import scala.collection.JavaConverters._ +/** + * Object containing all the method utilities for creating and loading + * a Qbeast formatted Table into the Catalog + */ object QbeastCatalogUtils { + val QBEAST_PROVIDER_NAME: String = "qbeast" + lazy val spark: SparkSession = SparkSession.active - // TODO move definition of string to a generic object - def isQbeastTable(provider: Option[String]): Boolean = { - provider.isDefined && provider.get == "qbeast" + /** + * Checks if the provider is Qbeast + * @param provider the provider, if any + * @return + */ + def isQbeastProvider(provider: Option[String]): Boolean = { + provider.isDefined && provider.get == QBEAST_PROVIDER_NAME } + /** + * Checks if an Identifier is set with a path + * @param ident the Identifier + * @return + */ def isPathTable(ident: Identifier): Boolean = { try { spark.sessionState.conf.runSQLonFile && hasQbeastNamespace(ident) && new Path( @@ -65,7 +80,7 @@ object QbeastCatalogUtils { throw AnalysisExceptionFactory.create( s"$table is a view. You may not write data into a view.") } - if (!isQbeastTable(oldTable.provider)) { + if (!isQbeastProvider(oldTable.provider)) { throw AnalysisExceptionFactory.create(s"$table is not a Qbeast table.") } Some(oldTable) @@ -80,7 +95,7 @@ object QbeastCatalogUtils { * @return */ def hasQbeastNamespace(ident: Identifier): Boolean = { - ident.namespace().length == 1 && ident.name.toLowerCase(Locale.ROOT) == "qbeast" + ident.namespace().length == 1 && ident.name.toLowerCase(Locale.ROOT) == QBEAST_PROVIDER_NAME } /** @@ -181,7 +196,7 @@ object QbeastCatalogUtils { * @param existingTableOpt * @param existingSessionCatalog */ - def updateCatalog( + private def updateCatalog( operation: TableCreationModes.CreationMode, table: CatalogTable, isPathTable: Boolean, diff --git a/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastDeltaCatalog.scala b/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastDeltaCatalog.scala index 1281c507e..dc8ba1827 100644 --- a/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastDeltaCatalog.scala +++ b/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastDeltaCatalog.scala @@ -15,13 +15,19 @@ import org.apache.spark.sql.types.StructType import java.util import scala.collection.JavaConverters._ +/** + * QbeastDeltaCatalog is a DelegatingCatalogExtension with StagingTableCatalog + * that extends the current implementation of DeltaCatalog. + * This would allow to populate Delta Tables with this implementation, + * along with the creation of Qbeast tables + */ class QbeastDeltaCatalog extends DeltaCatalog { private val tableFactory = QbeastContext.indexedTableFactory override def loadTable(ident: Identifier): Table = { val superTable = super.loadTable(ident) - if (QbeastCatalogUtils.isQbeastTable(superTable.properties().asScala.get("provider"))) { + if (QbeastCatalogUtils.isQbeastProvider(superTable.properties().asScala.get("provider"))) { QbeastCatalogUtils.loadQbeastTable(superTable, tableFactory) } else { superTable @@ -35,7 +41,7 @@ class QbeastDeltaCatalog extends DeltaCatalog { properties: util.Map[String, String]): Table = { val superTable = super.createTable(ident, schema, partitions, properties) - if (QbeastCatalogUtils.isQbeastTable(superTable.properties().asScala.get("provider"))) { + if (QbeastCatalogUtils.isQbeastProvider(superTable.properties().asScala.get("provider"))) { checkQbeastProperties(properties.asScala.toMap) QbeastCatalogUtils.loadQbeastTable(superTable, tableFactory) } else { @@ -44,12 +50,20 @@ class QbeastDeltaCatalog extends DeltaCatalog { } + /** + * For StageReplace, StageReplaceOrCreate and StageCreate, the following pipeline is executed + * 1. Check if it's Qbeast Provider + * 2. Create a QbeastStagedTable. + * This type of table allows to commit the changes atomically to the Catalog. + * 3. If it was not a QbeastProvider, it delegates the creation/replacement to the DeltaCatalog + */ + override def stageReplace( ident: Identifier, schema: StructType, partitions: Array[Transform], properties: util.Map[String, String]): StagedTable = { - if (QbeastCatalogUtils.isQbeastTable(properties.asScala.get("provider"))) { + if (QbeastCatalogUtils.isQbeastProvider(properties.asScala.get("provider"))) { new QbeastStagedTableImpl( ident, schema, @@ -67,7 +81,7 @@ class QbeastDeltaCatalog extends DeltaCatalog { schema: StructType, partitions: Array[Transform], properties: util.Map[String, String]): StagedTable = { - if (QbeastCatalogUtils.isQbeastTable(properties.asScala.get("provider"))) { + if (QbeastCatalogUtils.isQbeastProvider(properties.asScala.get("provider"))) { new QbeastStagedTableImpl( ident, schema, @@ -86,7 +100,7 @@ class QbeastDeltaCatalog extends DeltaCatalog { schema: StructType, partitions: Array[Transform], properties: util.Map[String, String]): StagedTable = { - if (QbeastCatalogUtils.isQbeastTable(properties.asScala.get("provider"))) { + if (QbeastCatalogUtils.isQbeastProvider(properties.asScala.get("provider"))) { new QbeastStagedTableImpl( ident, schema, diff --git a/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastStagedTableImpl.scala b/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastStagedTableImpl.scala index eec549000..17f1e9db1 100644 --- a/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastStagedTableImpl.scala +++ b/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastStagedTableImpl.scala @@ -72,6 +72,8 @@ private[sources] class QbeastStagedTableImpl( checkQbeastProperties(props.asScala.toMap) + // Creates the corresponding table on the Catalog and executes + // the writing of the dataFrame if any QbeastCatalogUtils.createQbeastTable( ident, schema, From e2448ed8965c1fca2a6036ba88dacb131f278568 Mon Sep 17 00:00:00 2001 From: osopardo1 Date: Wed, 14 Sep 2022 10:15:26 +0200 Subject: [PATCH 32/49] Rename Catalog --- ...DeltaCatalog.scala => QbeastCatalog.scala} | 4 +- .../internal/sources/v2/QbeastScan.scala | 54 ------------------- .../spark/QbeastIntegrationTestSpec.scala | 2 +- ...alogTest.scala => QbeastCatalogTest.scala} | 6 ++- 4 files changed, 7 insertions(+), 59 deletions(-) rename src/main/scala/io/qbeast/spark/internal/sources/catalog/{QbeastDeltaCatalog.scala => QbeastCatalog.scala} (96%) delete mode 100644 src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastScan.scala rename src/test/scala/io/qbeast/spark/internal/sources/{QbeastDeltaCatalogTest.scala => QbeastCatalogTest.scala} (83%) diff --git a/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastDeltaCatalog.scala b/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalog.scala similarity index 96% rename from src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastDeltaCatalog.scala rename to src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalog.scala index dc8ba1827..3547bbee9 100644 --- a/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastDeltaCatalog.scala +++ b/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalog.scala @@ -16,12 +16,12 @@ import java.util import scala.collection.JavaConverters._ /** - * QbeastDeltaCatalog is a DelegatingCatalogExtension with StagingTableCatalog + * QbeastCatalog is a DelegatingCatalogExtension with StagingTableCatalog * that extends the current implementation of DeltaCatalog. * This would allow to populate Delta Tables with this implementation, * along with the creation of Qbeast tables */ -class QbeastDeltaCatalog extends DeltaCatalog { +class QbeastCatalog extends DeltaCatalog { private val tableFactory = QbeastContext.indexedTableFactory diff --git a/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastScan.scala b/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastScan.scala deleted file mode 100644 index d536a1d8e..000000000 --- a/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastScan.scala +++ /dev/null @@ -1,54 +0,0 @@ -/* - * Copyright 2021 Qbeast Analytics, S.L. - */ -package io.qbeast.spark.internal.sources.v2 - -import io.qbeast.spark.internal.sources.QbeastBaseRelation -import io.qbeast.spark.table.IndexedTable -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{Row, SQLContext} -import org.apache.spark.sql.connector.read.V1Scan -import org.apache.spark.sql.sources.{BaseRelation, TableScan} -import org.apache.spark.sql.types.StructType - -/** - * Extends Scan for V1 DataSource - * It allows Spark to read from a Qbeast Formatted table - * @param indexedTable the IndexedTable - */ - -// TODO Disclaimer: This class is not used yet. -// QbeastScan returns an Scan object that could allow QbeastTableImpl to SupportRead -// This can be the object where we can include logic -// to get rid of the QbeastHash while reading the records -class QbeastScan(indexedTable: IndexedTable) extends V1Scan { - - private lazy val qbeastBaseRelation = - QbeastBaseRelation.forQbeastTable(indexedTable) - - override def toV1TableScan[T <: BaseRelation with TableScan](context: SQLContext): T = { - - // TODO add PrunedFilteredScan (column pruning + filter pushdown) - // as an extension and implement the methods - new BaseRelation with TableScan { - override def sqlContext: SQLContext = context - - override def schema: StructType = qbeastBaseRelation.schema - - override def buildScan(): RDD[Row] = { - - // We output the rdd from the DataFrame scan - // This is a hack, and the implementation should add more Qbeast logic - val df = context.sparkSession.baseRelationToDataFrame(qbeastBaseRelation) - - // Return the RDD - df.rdd - } - - }.asInstanceOf[T] - - } - - override def readSchema(): StructType = qbeastBaseRelation.schema - -} diff --git a/src/test/scala/io/qbeast/spark/QbeastIntegrationTestSpec.scala b/src/test/scala/io/qbeast/spark/QbeastIntegrationTestSpec.scala index 9b985f8eb..8c6bece82 100644 --- a/src/test/scala/io/qbeast/spark/QbeastIntegrationTestSpec.scala +++ b/src/test/scala/io/qbeast/spark/QbeastIntegrationTestSpec.scala @@ -42,7 +42,7 @@ trait QbeastIntegrationTestSpec extends AnyFlatSpec with Matchers with DatasetCo .set("spark.sql.extensions", "io.qbeast.spark.internal.QbeastSparkSessionExtension") .set( SQLConf.V2_SESSION_CATALOG_IMPLEMENTATION.key, - "io.qbeast.spark.internal.sources.catalog.QbeastDeltaCatalog") + "io.qbeast.spark.internal.sources.catalog.QbeastCatalog") def loadTestData(spark: SparkSession): DataFrame = spark.read .format("csv") diff --git a/src/test/scala/io/qbeast/spark/internal/sources/QbeastDeltaCatalogTest.scala b/src/test/scala/io/qbeast/spark/internal/sources/QbeastCatalogTest.scala similarity index 83% rename from src/test/scala/io/qbeast/spark/internal/sources/QbeastDeltaCatalogTest.scala rename to src/test/scala/io/qbeast/spark/internal/sources/QbeastCatalogTest.scala index a2890e942..f4234e9c2 100644 --- a/src/test/scala/io/qbeast/spark/internal/sources/QbeastDeltaCatalogTest.scala +++ b/src/test/scala/io/qbeast/spark/internal/sources/QbeastCatalogTest.scala @@ -2,11 +2,13 @@ package io.qbeast.spark.internal.sources import io.qbeast.TestClasses.Student import io.qbeast.spark.QbeastIntegrationTestSpec +import io.qbeast.spark.internal.sources.v2.QbeastTableImpl +import org.apache.spark.sql.types.{IntegerType, StructField, StructType} import org.apache.spark.sql.{DataFrame, SparkSession} import scala.util.Random -class QbeastDeltaCatalogTest extends QbeastIntegrationTestSpec { +class QbeastCatalogTest extends QbeastIntegrationTestSpec { private val students = 1.to(10).map(i => Student(i, i.toString, Random.nextInt())) @@ -15,7 +17,7 @@ class QbeastDeltaCatalogTest extends QbeastIntegrationTestSpec { students.toDF() } - "QbeastDeltaCatalogTest" should + "QbeastCatalog" should "coexist with Delta tables" in withQbeastContextSparkAndTmpWarehouse((spark, tmpDir) => { val data = createTestData(spark) From 8f9218b096ea66addd09d02a9e69dc157db3a353 Mon Sep 17 00:00:00 2001 From: osopardo1 Date: Wed, 14 Sep 2022 10:25:10 +0200 Subject: [PATCH 33/49] Remove duplication of CreateTable --- .../spark/internal/sources/catalog/QbeastCatalog.scala | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalog.scala b/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalog.scala index 3547bbee9..211275747 100644 --- a/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalog.scala +++ b/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalog.scala @@ -40,10 +40,11 @@ class QbeastCatalog extends DeltaCatalog { partitions: Array[Transform], properties: util.Map[String, String]): Table = { - val superTable = super.createTable(ident, schema, partitions, properties) - if (QbeastCatalogUtils.isQbeastProvider(superTable.properties().asScala.get("provider"))) { + gitif (QbeastCatalogUtils.isQbeastProvider(properties.asScala.get("provider"))) { checkQbeastProperties(properties.asScala.toMap) - QbeastCatalogUtils.loadQbeastTable(superTable, tableFactory) + QbeastCatalogUtils.loadQbeastTable( + super.createTable(ident, schema, partitions, properties), + tableFactory) } else { super.createTable(ident, schema, partitions, properties) } From 9276b598ae56e1d4ebe2e7a8463bb3b2f0046569 Mon Sep 17 00:00:00 2001 From: osopardo1 Date: Wed, 14 Sep 2022 15:14:23 +0200 Subject: [PATCH 34/49] Resolving build errors --- .../qbeast/spark/internal/sources/catalog/QbeastCatalog.scala | 2 +- .../io/qbeast/spark/internal/sources/QbeastCatalogTest.scala | 2 -- 2 files changed, 1 insertion(+), 3 deletions(-) diff --git a/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalog.scala b/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalog.scala index 211275747..249d248f6 100644 --- a/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalog.scala +++ b/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalog.scala @@ -40,7 +40,7 @@ class QbeastCatalog extends DeltaCatalog { partitions: Array[Transform], properties: util.Map[String, String]): Table = { - gitif (QbeastCatalogUtils.isQbeastProvider(properties.asScala.get("provider"))) { + if (QbeastCatalogUtils.isQbeastProvider(properties.asScala.get("provider"))) { checkQbeastProperties(properties.asScala.toMap) QbeastCatalogUtils.loadQbeastTable( super.createTable(ident, schema, partitions, properties), diff --git a/src/test/scala/io/qbeast/spark/internal/sources/QbeastCatalogTest.scala b/src/test/scala/io/qbeast/spark/internal/sources/QbeastCatalogTest.scala index f4234e9c2..2d1b4a30c 100644 --- a/src/test/scala/io/qbeast/spark/internal/sources/QbeastCatalogTest.scala +++ b/src/test/scala/io/qbeast/spark/internal/sources/QbeastCatalogTest.scala @@ -2,8 +2,6 @@ package io.qbeast.spark.internal.sources import io.qbeast.TestClasses.Student import io.qbeast.spark.QbeastIntegrationTestSpec -import io.qbeast.spark.internal.sources.v2.QbeastTableImpl -import org.apache.spark.sql.types.{IntegerType, StructField, StructType} import org.apache.spark.sql.{DataFrame, SparkSession} import scala.util.Random From 1b5bebf5ac710bbdba59801e2f4aac2030723d06 Mon Sep 17 00:00:00 2001 From: osopardo1 Date: Mon, 19 Sep 2022 16:39:29 +0200 Subject: [PATCH 35/49] Extend CatalogExtenssion instead of DelegatingCatalogExtenssion or DeltaCatalog We need the QbeastCatalog to be independent from other existing Catalog solutions. --- .../sources/catalog/QbeastCatalog.scala | 106 +++++++++++++++--- .../sources/catalog/QbeastCatalogUtils.scala | 8 +- .../apache/spark/sql/SparkCatalogUtils.scala | 15 +++ .../internal/sources/QbeastCatalogTest.scala | 58 ++++++---- 4 files changed, 148 insertions(+), 39 deletions(-) create mode 100644 src/main/scala/org/apache/spark/sql/SparkCatalogUtils.scala diff --git a/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalog.scala b/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalog.scala index 249d248f6..9252c5e34 100644 --- a/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalog.scala +++ b/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalog.scala @@ -6,27 +6,54 @@ package io.qbeast.spark.internal.sources.catalog import io.qbeast.context.QbeastContext import io.qbeast.spark.internal.QbeastOptions.checkQbeastProperties import io.qbeast.spark.internal.sources.v2.QbeastStagedTableImpl -import org.apache.spark.sql.connector.catalog.{Identifier, StagedTable, Table} +import org.apache.spark.sql.{SparkCatalogUtils, SparkSession} +import org.apache.spark.sql.connector.catalog.{ + CatalogExtension, + CatalogPlugin, + Identifier, + NamespaceChange, + StagedTable, + StagingTableCatalog, + SupportsNamespaces, + Table, + TableCatalog, + TableChange +} import org.apache.spark.sql.connector.expressions.Transform -import org.apache.spark.sql.delta.catalog.DeltaCatalog import org.apache.spark.sql.delta.commands.TableCreationModes import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.CaseInsensitiveStringMap import java.util import scala.collection.JavaConverters._ /** - * QbeastCatalog is a DelegatingCatalogExtension with StagingTableCatalog - * that extends the current implementation of DeltaCatalog. - * This would allow to populate Delta Tables with this implementation, - * along with the creation of Qbeast tables + * QbeastCatalog is a CatalogExtenssion with StagingTableCatalog */ -class QbeastCatalog extends DeltaCatalog { +class QbeastCatalog[T <: TableCatalog with SupportsNamespaces] + extends CatalogExtension + with SupportsNamespaces + with StagingTableCatalog { private val tableFactory = QbeastContext.indexedTableFactory + private var delegatedCatalog: CatalogPlugin = null + + private var catalogName: String = null + + private def getSessionCatalog(): T = { + val sessionCatalog = delegatedCatalog match { + case null => + // In this case, any catalog has been delegated, so we need to search for the default + SparkCatalogUtils.getV2SessionCatalog(SparkSession.active) + case o => o + } + + sessionCatalog.asInstanceOf[T] + } + override def loadTable(ident: Identifier): Table = { - val superTable = super.loadTable(ident) + val superTable = getSessionCatalog().loadTable(ident) if (QbeastCatalogUtils.isQbeastProvider(superTable.properties().asScala.get("provider"))) { QbeastCatalogUtils.loadQbeastTable(superTable, tableFactory) } else { @@ -43,10 +70,10 @@ class QbeastCatalog extends DeltaCatalog { if (QbeastCatalogUtils.isQbeastProvider(properties.asScala.get("provider"))) { checkQbeastProperties(properties.asScala.toMap) QbeastCatalogUtils.loadQbeastTable( - super.createTable(ident, schema, partitions, properties), + getSessionCatalog().createTable(ident, schema, partitions, properties), tableFactory) } else { - super.createTable(ident, schema, partitions, properties) + getSessionCatalog().createTable(ident, schema, partitions, properties) } } @@ -73,7 +100,10 @@ class QbeastCatalog extends DeltaCatalog { properties, tableFactory) } else { - super.stageReplace(ident, schema, partitions, properties) + DefaultStagedTable( + ident, + getSessionCatalog().createTable(ident, schema, partitions, properties), + this) } } @@ -91,7 +121,10 @@ class QbeastCatalog extends DeltaCatalog { properties, tableFactory) } else { - super.stageCreateOrReplace(ident, schema, partitions, properties) + DefaultStagedTable( + ident, + getSessionCatalog().createTable(ident, schema, partitions, properties), + this) } } @@ -110,8 +143,55 @@ class QbeastCatalog extends DeltaCatalog { properties, tableFactory) } else { - super.stageCreate(ident, schema, partitions, properties) + DefaultStagedTable( + ident, + getSessionCatalog().createTable(ident, schema, partitions, properties), + this) } } + override def listTables(namespace: Array[String]): Array[Identifier] = + getSessionCatalog().listTables(namespace) + + override def alterTable(ident: Identifier, changes: TableChange*): Table = + getSessionCatalog().alterTable(ident, changes.head) + + override def dropTable(ident: Identifier): Boolean = getSessionCatalog().dropTable(ident) + + override def renameTable(oldIdent: Identifier, newIdent: Identifier): Unit = + getSessionCatalog().renameTable(oldIdent, newIdent) + + override def listNamespaces(): Array[Array[String]] = getSessionCatalog().listNamespaces() + + override def listNamespaces(namespace: Array[String]): Array[Array[String]] = + getSessionCatalog().listNamespaces(namespace) + + override def loadNamespaceMetadata(namespace: Array[String]): util.Map[String, String] = + getSessionCatalog().loadNamespaceMetadata(namespace) + + override def createNamespace( + namespace: Array[String], + metadata: util.Map[String, String]): Unit = + getSessionCatalog().createNamespace(namespace, metadata) + + override def alterNamespace(namespace: Array[String], changes: NamespaceChange*): Unit = + getSessionCatalog().alterNamespace(namespace, changes.head) + + override def dropNamespace(namespace: Array[String]): Boolean = + getSessionCatalog().dropNamespace(namespace) + + override def initialize(name: String, options: CaseInsensitiveStringMap): Unit = { + // Initialize the catalog with the corresponding name + this.catalogName = name + } + + override def name(): String = catalogName + + override def setDelegateCatalog(delegate: CatalogPlugin): Unit = { + // Check if the delegating catalog has Table and SupportsNamespace properties + if (delegate.isInstanceOf[TableCatalog] && delegate.isInstanceOf[SupportsNamespaces]) { + this.delegatedCatalog = delegate + } else throw new IllegalArgumentException("Invalid session catalog: " + delegate) + } + } diff --git a/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalogUtils.scala b/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalogUtils.scala index b95be78a5..04265785c 100644 --- a/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalogUtils.scala +++ b/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalogUtils.scala @@ -10,7 +10,7 @@ import org.apache.hadoop.fs.Path import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.CannotReplaceMissingTableException import org.apache.spark.sql.catalyst.catalog._ -import org.apache.spark.sql.connector.catalog.{Identifier, Table, TableCatalog} +import org.apache.spark.sql.connector.catalog.{Identifier, Table} import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.delta.catalog.DeltaTableV2 import org.apache.spark.sql.delta.commands.TableCreationModes @@ -121,6 +121,8 @@ object QbeastCatalogUtils { tableCreationMode: TableCreationModes.CreationMode, tableFactory: IndexedTableFactory, existingSessionCatalog: SessionCatalog): Unit = { + + /* // These two keys are tableProperties in data source v2 but not in v1, so we have to filter // them out. Otherwise property consistency checks will fail. val tableProperties = allTableProperties.asScala.filterKeys { @@ -133,6 +135,8 @@ object QbeastCatalogUtils { case _ => true } + */ + val isPathTable = QbeastCatalogUtils.isPathTable(ident) if (isPathTable @@ -173,7 +177,7 @@ object QbeastCatalogUtils { provider = Some("qbeast"), partitionColumnNames = partitionColumns, bucketSpec = bucketSpec, - properties = tableProperties.toMap, + properties = allTableProperties.asScala.toMap, comment = commentOpt) val append = tableCreationMode.mode == SaveMode.Append diff --git a/src/main/scala/org/apache/spark/sql/SparkCatalogUtils.scala b/src/main/scala/org/apache/spark/sql/SparkCatalogUtils.scala new file mode 100644 index 000000000..09b4392ca --- /dev/null +++ b/src/main/scala/org/apache/spark/sql/SparkCatalogUtils.scala @@ -0,0 +1,15 @@ +/* + * Copyright 2021 Qbeast Analytics, S.L. + */ +package org.apache.spark.sql + +import org.apache.spark.sql.connector.catalog.CatalogPlugin + +object SparkCatalogUtils { + + def getV2SessionCatalog(spark: SparkSession): CatalogPlugin = { + val catalogManager = spark.sessionState.catalogManager + catalogManager.v2SessionCatalog + } + +} diff --git a/src/test/scala/io/qbeast/spark/internal/sources/QbeastCatalogTest.scala b/src/test/scala/io/qbeast/spark/internal/sources/QbeastCatalogTest.scala index 2d1b4a30c..4e2ccad9b 100644 --- a/src/test/scala/io/qbeast/spark/internal/sources/QbeastCatalogTest.scala +++ b/src/test/scala/io/qbeast/spark/internal/sources/QbeastCatalogTest.scala @@ -2,6 +2,7 @@ package io.qbeast.spark.internal.sources import io.qbeast.TestClasses.Student import io.qbeast.spark.QbeastIntegrationTestSpec +import org.apache.spark.SparkConf import org.apache.spark.sql.{DataFrame, SparkSession} import scala.util.Random @@ -16,29 +17,38 @@ class QbeastCatalogTest extends QbeastIntegrationTestSpec { } "QbeastCatalog" should - "coexist with Delta tables" in withQbeastContextSparkAndTmpWarehouse((spark, tmpDir) => { - - val data = createTestData(spark) - - data.write.format("delta").saveAsTable("delta_table") // delta catalog - - data.write - .format("qbeast") - .option("columnsToIndex", "id") - .saveAsTable("qbeast_table") // qbeast catalog - - val tables = spark.sessionState.catalog.listTables("default") - tables.size shouldBe 2 - - val deltaTable = spark.read.table("delta_table") - val qbeastTable = spark.read.table("qbeast_table") - - assertSmallDatasetEquality( - deltaTable, - qbeastTable, - orderedComparison = false, - ignoreNullable = true) - - }) + "coexist with Delta tables" in withTmpDir(tmpDir => + withExtendedSpark(sparkConf = new SparkConf() + .setMaster("local[8]") + .set("spark.sql.extensions", "io.qbeast.spark.internal.QbeastSparkSessionExtension") + .set("spark.sql.warehouse.dir", tmpDir) + .set("spark.sql.catalog.spark_catalog", "org.apache.spark.sql.delta.catalog.DeltaCatalog") + .set( + "spark.sql.catalog.qbeast_catalog", + "io.qbeast.spark.internal.sources.catalog.QbeastCatalog"))(spark => { + + val data = createTestData(spark) + + data.write.format("delta").saveAsTable("delta_table") // delta catalog + + // spark.sql("USE CATALOG qbeast_catalog") + data.write + .format("qbeast") + .option("columnsToIndex", "id") + .saveAsTable("qbeast_catalog.default.qbeast_table") // qbeast catalog + + val tables = spark.sessionState.catalog.listTables("default") + tables.size shouldBe 2 + + val deltaTable = spark.read.table("delta_table") + val qbeastTable = spark.read.table("qbeast_catalog.default.qbeast_table") + + assertSmallDatasetEquality( + deltaTable, + qbeastTable, + orderedComparison = false, + ignoreNullable = true) + + })) } From b355478e0d5733186cf2e3c5d29fca33ae0e0c1d Mon Sep 17 00:00:00 2001 From: osopardo1 Date: Mon, 19 Sep 2022 17:00:45 +0200 Subject: [PATCH 36/49] Update readme --- README.md | 36 +++++++++++++++++++++--------------- 1 file changed, 21 insertions(+), 15 deletions(-) diff --git a/README.md b/README.md index b84efacec..ffb1f6aeb 100644 --- a/README.md +++ b/README.md @@ -92,6 +92,7 @@ export SPARK_HOME=$PWD/spark-3.1.1-bin-hadoop3.2 ```bash $SPARK_HOME/bin/spark-shell \ --conf spark.sql.extensions=io.qbeast.spark.internal.QbeastSparkSessionExtension \ +--conf spark.sql.catalog.spark_catalog=io.qbeast.spark.internal.sources.catalog.QbeastCatalog \ --packages io.qbeast:qbeast-spark_2.12:0.2.0,io.delta:delta-core_2.12:1.0.0 ``` @@ -118,6 +119,26 @@ csv_df.write .save(tmp_dir) ``` +#### SQL Syntax. +You can create a table with Qbeast with the help of `QbeastCatalog`. + +```scala +spark.sql( + "CREATE TABLE student (id INT, name STRING, age INT) " + + "USING qbeast OPTIONS ('columnsToIndex'='id')") + +``` + +Use **`INSERT INTO`** to add records to the new table. It will update the index in a **dynamic** fashion when new data is inserted. + +```scala + +spark.sql("INSERT INTO table student SELECT * from oldStudentsTable") + +spark.sql("INSERT INTO table student (value) values ((4, 'Joan', 20))") + +``` + ### 3. Load the dataset Load the newly indexed dataset. @@ -151,21 +172,6 @@ qbeastTable.getIndexMetrics() qbeastTable.analyze() ``` -The format supports **Spark SQL** syntax. -It also updates the index in a **dynamic** fashion when new data is inserted. - -```scala -val newData = Seq(1, 2, 3, 4).toDF("value") - -newData.createOrReplaceTempView("newTable") - -spark.sql("insert into table myTable select * from newTable") - -spark.sql("insert into table myTable (value) values (4)") - - -``` - Go to [QbeastTable documentation](./docs/QbeastTable.md) for more detailed information. # Dependencies and Version Compatibility From 3b10deebff7c590e401c2b50b329006eec631048 Mon Sep 17 00:00:00 2001 From: osopardo1 Date: Tue, 20 Sep 2022 11:43:42 +0200 Subject: [PATCH 37/49] Fix creation of empty tables and add more test This commit also includes removal of Delta dependencies on catalog/sources classes --- README.md | 2 +- .../qbeast/core/model/MetadataManager.scala | 13 +++ .../delta/SparkDeltaMetadataManager.scala | 19 ++++ .../internal/sources/QbeastBaseRelation.scala | 62 ++++++++----- .../sources/catalog/QbeastCatalog.scala | 17 ++-- .../sources/catalog/QbeastCatalogUtils.scala | 45 ++++++---- .../sources/catalog/TableCreationMode.scala | 27 ++++++ .../sources/v2/QbeastStagedTableImpl.scala | 5 +- .../internal/sources/QbeastCatalogTest.scala | 89 +++++++++++++++++++ 9 files changed, 225 insertions(+), 54 deletions(-) create mode 100644 src/main/scala/io/qbeast/spark/internal/sources/catalog/TableCreationMode.scala diff --git a/README.md b/README.md index ffb1f6aeb..1c58f63e3 100644 --- a/README.md +++ b/README.md @@ -135,7 +135,7 @@ Use **`INSERT INTO`** to add records to the new table. It will update the index spark.sql("INSERT INTO table student SELECT * from oldStudentsTable") -spark.sql("INSERT INTO table student (value) values ((4, 'Joan', 20))") +spark.sql("INSERT INTO table student (id, name, age) values ((4, 'Joan', 20))") ``` 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 3fcf5abf8..4dc394925 100644 --- a/core/src/main/scala/io/qbeast/core/model/MetadataManager.scala +++ b/core/src/main/scala/io/qbeast/core/model/MetadataManager.scala @@ -63,4 +63,17 @@ trait MetadataManager[DataSchema, FileDescriptor] { knownAnnounced: Set[CubeId], oldReplicatedSet: ReplicatedSet): Boolean + /** + * Checks if there's an existing log directory for the table + * @param tableID the table ID + * @return + */ + def existsLog(tableID: QTableID): Boolean + + /** + * Creates an initial log directory + * @param tableID + */ + def createLog(tableID: QTableID): Unit + } diff --git a/src/main/scala/io/qbeast/spark/delta/SparkDeltaMetadataManager.scala b/src/main/scala/io/qbeast/spark/delta/SparkDeltaMetadataManager.scala index 458fdc95e..2c70c3395 100644 --- a/src/main/scala/io/qbeast/spark/delta/SparkDeltaMetadataManager.scala +++ b/src/main/scala/io/qbeast/spark/delta/SparkDeltaMetadataManager.scala @@ -62,4 +62,23 @@ object SparkDeltaMetadataManager extends MetadataManager[StructType, FileAction] diff.nonEmpty } + /** + * Checks if there's an existing log directory for the table + * + * @param tableID the table ID + * @return + */ + override def existsLog(tableID: QTableID): Boolean = { + loadDeltaQbeastLog(tableID).deltaLog.tableExists + } + + /** + * Creates an initial log directory + * + * @param tableID + */ + override def createLog(tableID: QTableID): Unit = { + loadDeltaQbeastLog(tableID).deltaLog.createLogDirectory() + } + } diff --git a/src/main/scala/io/qbeast/spark/internal/sources/QbeastBaseRelation.scala b/src/main/scala/io/qbeast/spark/internal/sources/QbeastBaseRelation.scala index 633a4e0a8..cdb14c029 100644 --- a/src/main/scala/io/qbeast/spark/internal/sources/QbeastBaseRelation.scala +++ b/src/main/scala/io/qbeast/spark/internal/sources/QbeastBaseRelation.scala @@ -5,17 +5,16 @@ package io.qbeast.spark.internal.sources import org.apache.spark.sql.sources.BaseRelation import org.apache.spark.sql.sources.InsertableRelation - -import org.apache.spark.sql.{SQLContext} -import org.apache.spark.sql.types.{StructType, StructField} +import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.types.{StructField, StructType} import org.apache.spark.sql.DataFrame import org.apache.spark.sql.SparkSession import io.qbeast.spark.delta.OTreeIndex import org.apache.spark.sql.execution.datasources.HadoopFsRelation import io.qbeast.spark.table.IndexedTable import io.qbeast.context.QbeastContext -import org.apache.hadoop.fs.{Path} -import org.apache.spark.sql.catalyst.catalog.{BucketSpec} +import org.apache.hadoop.fs.Path +import org.apache.spark.sql.catalyst.catalog.BucketSpec import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat /** @@ -38,26 +37,43 @@ object QbeastBaseRelation { val tableID = table.tableID val snapshot = QbeastContext.metadataManager.loadSnapshot(tableID) val schema = QbeastContext.metadataManager.loadCurrentSchema(tableID) - val revision = snapshot.loadLatestRevision - val columnsToIndex = revision.columnTransformers.map(row => row.columnName).mkString(",") - val cubeSize = revision.desiredCubeSize - val parameters = - Map[String, String]("columnsToIndex" -> columnsToIndex, "cubeSize" -> cubeSize.toString()) + if (snapshot.isInitial) { + // If the Table is initial, read empty relation + // This could happen if we CREATE/REPLACE TABLE without inserting data + new HadoopFsRelation( + OTreeIndex(spark, new Path(tableID.id)), + partitionSchema = StructType(Seq.empty[StructField]), + dataSchema = schema, + bucketSpec = None, + new ParquetFileFormat(), + Map.empty)(spark) with InsertableRelation { + def insert(data: DataFrame, overwrite: Boolean): Unit = { + table.save(data, Map.empty, append = !overwrite) + } + } + } else { + // If the table contains data, initialize it + val revision = snapshot.loadLatestRevision + val columnsToIndex = revision.columnTransformers.map(row => row.columnName).mkString(",") + val cubeSize = revision.desiredCubeSize + val parameters = + Map[String, String]("columnsToIndex" -> columnsToIndex, "cubeSize" -> cubeSize.toString()) - val path = new Path(tableID.id) - val fileIndex = OTreeIndex(spark, path) - val bucketSpec: Option[BucketSpec] = None - val file = new ParquetFileFormat() + val path = new Path(tableID.id) + val fileIndex = OTreeIndex(spark, path) + val bucketSpec: Option[BucketSpec] = None + val file = new ParquetFileFormat() - new HadoopFsRelation( - fileIndex, - partitionSchema = StructType(Seq.empty[StructField]), - dataSchema = schema, - bucketSpec = bucketSpec, - file, - parameters)(spark) with InsertableRelation { - def insert(data: DataFrame, overwrite: Boolean): Unit = { - table.save(data, parameters, append = !overwrite) + new HadoopFsRelation( + fileIndex, + partitionSchema = StructType(Seq.empty[StructField]), + dataSchema = schema, + bucketSpec = bucketSpec, + file, + parameters)(spark) with InsertableRelation { + def insert(data: DataFrame, overwrite: Boolean): Unit = { + table.save(data, parameters, append = !overwrite) + } } } } diff --git a/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalog.scala b/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalog.scala index 9252c5e34..9727ad457 100644 --- a/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalog.scala +++ b/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalog.scala @@ -20,7 +20,6 @@ import org.apache.spark.sql.connector.catalog.{ TableChange } import org.apache.spark.sql.connector.expressions.Transform -import org.apache.spark.sql.delta.commands.TableCreationModes import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -28,7 +27,10 @@ import java.util import scala.collection.JavaConverters._ /** - * QbeastCatalog is a CatalogExtenssion with StagingTableCatalog + * QbeastCatalog is a CatalogExtenssion that supports Namespaces + * and the CREATION and/or REPLACEMENT of tables + * QbeastCatalog uses a session catalog of type T + * to delegate high-level operations */ class QbeastCatalog[T <: TableCatalog with SupportsNamespaces] extends CatalogExtension @@ -81,9 +83,8 @@ class QbeastCatalog[T <: TableCatalog with SupportsNamespaces] /** * For StageReplace, StageReplaceOrCreate and StageCreate, the following pipeline is executed * 1. Check if it's Qbeast Provider - * 2. Create a QbeastStagedTable. - * This type of table allows to commit the changes atomically to the Catalog. - * 3. If it was not a QbeastProvider, it delegates the creation/replacement to the DeltaCatalog + * 2. Create a QbeastStagedTable. This type of table allows to commit the changes atomically to the Catalog. + * 3. If it was not a QbeastProvider, it outputs a DefaultStagedTable */ override def stageReplace( @@ -96,7 +97,7 @@ class QbeastCatalog[T <: TableCatalog with SupportsNamespaces] ident, schema, partitions, - TableCreationModes.Replace, + TableCreationMode.REPLACE_TABLE, properties, tableFactory) } else { @@ -117,7 +118,7 @@ class QbeastCatalog[T <: TableCatalog with SupportsNamespaces] ident, schema, partitions, - TableCreationModes.CreateOrReplace, + TableCreationMode.CREATE_OR_REPLACE, properties, tableFactory) } else { @@ -139,7 +140,7 @@ class QbeastCatalog[T <: TableCatalog with SupportsNamespaces] ident, schema, partitions, - TableCreationModes.Create, + TableCreationMode.CREATE_TABLE, properties, tableFactory) } else { diff --git a/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalogUtils.scala b/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalogUtils.scala index 04265785c..c64d4ae5c 100644 --- a/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalogUtils.scala +++ b/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalogUtils.scala @@ -3,6 +3,7 @@ */ package io.qbeast.spark.internal.sources.catalog +import io.qbeast.context.QbeastContext.metadataManager import io.qbeast.core.model.QTableID import io.qbeast.spark.internal.sources.v2.QbeastTableImpl import io.qbeast.spark.table.IndexedTableFactory @@ -12,8 +13,6 @@ import org.apache.spark.sql.catalyst.analysis.CannotReplaceMissingTableException import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.connector.catalog.{Identifier, Table} import org.apache.spark.sql.connector.expressions.Transform -import org.apache.spark.sql.delta.catalog.DeltaTableV2 -import org.apache.spark.sql.delta.commands.TableCreationModes import org.apache.spark.sql.execution.datasources.DataSource import org.apache.spark.sql.types.StructType import org.apache.spark.sql.{ @@ -118,7 +117,7 @@ object QbeastCatalogUtils { allTableProperties: util.Map[String, String], writeOptions: Map[String, String], dataFrame: Option[DataFrame], - tableCreationMode: TableCreationModes.CreationMode, + tableCreationMode: CreationMode, tableFactory: IndexedTableFactory, existingSessionCatalog: SessionCatalog): Unit = { @@ -180,14 +179,28 @@ object QbeastCatalogUtils { properties = allTableProperties.asScala.toMap, comment = commentOpt) - val append = tableCreationMode.mode == SaveMode.Append + val append = tableCreationMode.saveMode == SaveMode.Append dataFrame.map { df => tableFactory .getIndexedTable(QTableID(loc.toString)) .save(df, allTableProperties.asScala.toMap, append) } - updateCatalog(tableCreationMode, table, isPathTable, existingTableOpt, existingSessionCatalog) + updateCatalog( + QTableID(loc.toString), + tableCreationMode, + table, + isPathTable, + existingTableOpt, + existingSessionCatalog) + } + + private def checkLogCreation(tableID: QTableID): Unit = { + // If the Log is not created + // We make sure we create the table physically + // So new data can be inserted + val isLogCreated = metadataManager.existsLog(tableID) + if (!isLogCreated) metadataManager.createLog(tableID) } /** @@ -201,7 +214,8 @@ object QbeastCatalogUtils { * @param existingSessionCatalog */ private def updateCatalog( - operation: TableCreationModes.CreationMode, + tableID: QTableID, + operation: CreationMode, table: CatalogTable, isPathTable: Boolean, existingTableOpt: Option[CatalogTable], @@ -209,18 +223,20 @@ object QbeastCatalogUtils { operation match { case _ if isPathTable => // do nothing - case TableCreationModes.Create => + case TableCreationMode.CREATE_TABLE => + checkLogCreation(tableID) existingSessionCatalog.createTable( table, ignoreIfExists = existingTableOpt.isDefined, validateLocation = false) - case TableCreationModes.Replace | TableCreationModes.CreateOrReplace + case TableCreationMode.REPLACE_TABLE | TableCreationMode.CREATE_OR_REPLACE if existingTableOpt.isDefined => existingSessionCatalog.alterTable(table) - case TableCreationModes.Replace => + case TableCreationMode.REPLACE_TABLE => val ident = Identifier.of(table.identifier.database.toArray, table.identifier.table) throw new CannotReplaceMissingTableException(ident) - case TableCreationModes.CreateOrReplace => + case TableCreationMode.CREATE_OR_REPLACE => + checkLogCreation(tableID) existingSessionCatalog.createTable( table, ignoreIfExists = false, @@ -260,15 +276,6 @@ object QbeastCatalogUtils { Some(catalogTable), tableFactory) - case DeltaTableV2(_, path, catalogTable, tableIdentifier, _, options, _) => - new QbeastTableImpl( - tableIdentifier.get, - path, - options, - Some(schema), - catalogTable, - tableFactory) - case _ => table } } diff --git a/src/main/scala/io/qbeast/spark/internal/sources/catalog/TableCreationMode.scala b/src/main/scala/io/qbeast/spark/internal/sources/catalog/TableCreationMode.scala new file mode 100644 index 000000000..6ccbf32e4 --- /dev/null +++ b/src/main/scala/io/qbeast/spark/internal/sources/catalog/TableCreationMode.scala @@ -0,0 +1,27 @@ +package io.qbeast.spark.internal.sources.catalog + +import org.apache.spark.sql.SaveMode + +trait CreationMode { + val saveMode: SaveMode + val name: String +} + +object TableCreationMode { + + val CREATE_TABLE: CreationMode = new CreationMode { + override val saveMode: SaveMode = SaveMode.ErrorIfExists + override val name: String = "create" + } + + val CREATE_OR_REPLACE: CreationMode = new CreationMode { + override val saveMode: SaveMode = SaveMode.Overwrite + override val name: String = "createOrReplace" + } + + val REPLACE_TABLE: CreationMode = new CreationMode { + override val saveMode: SaveMode = SaveMode.Overwrite + override val name: String = "replace" + } + +} diff --git a/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastStagedTableImpl.scala b/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastStagedTableImpl.scala index 17f1e9db1..a4a1c3365 100644 --- a/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastStagedTableImpl.scala +++ b/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastStagedTableImpl.scala @@ -4,7 +4,7 @@ package io.qbeast.spark.internal.sources.v2 import io.qbeast.spark.internal.QbeastOptions.checkQbeastProperties -import io.qbeast.spark.internal.sources.catalog.QbeastCatalogUtils +import io.qbeast.spark.internal.sources.catalog.{CreationMode, QbeastCatalogUtils} import io.qbeast.spark.table.IndexedTableFactory import org.apache.spark.sql.catalyst.catalog.SessionCatalog import org.apache.spark.sql.connector.catalog.TableCapability.V1_BATCH_WRITE @@ -16,7 +16,6 @@ import org.apache.spark.sql.connector.catalog.{ } import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.connector.write.{LogicalWriteInfo, V1Write, Write, WriteBuilder} -import org.apache.spark.sql.delta.commands.TableCreationModes import org.apache.spark.sql.sources.InsertableRelation import org.apache.spark.sql.types.StructType import org.apache.spark.sql.{DataFrame, SparkSession} @@ -32,7 +31,7 @@ private[sources] class QbeastStagedTableImpl( ident: Identifier, schema: StructType, partitions: Array[Transform], - tableCreationMode: TableCreationModes.CreationMode, + tableCreationMode: CreationMode, override val properties: util.Map[String, String], private val tableFactory: IndexedTableFactory) extends StagedTable diff --git a/src/test/scala/io/qbeast/spark/internal/sources/QbeastCatalogTest.scala b/src/test/scala/io/qbeast/spark/internal/sources/QbeastCatalogTest.scala index 4e2ccad9b..b44a8eea0 100644 --- a/src/test/scala/io/qbeast/spark/internal/sources/QbeastCatalogTest.scala +++ b/src/test/scala/io/qbeast/spark/internal/sources/QbeastCatalogTest.scala @@ -3,6 +3,7 @@ package io.qbeast.spark.internal.sources import io.qbeast.TestClasses.Student import io.qbeast.spark.QbeastIntegrationTestSpec import org.apache.spark.SparkConf +import org.apache.spark.sql.types.{IntegerType, StringType, StructField, StructType} import org.apache.spark.sql.{DataFrame, SparkSession} import scala.util.Random @@ -16,6 +17,12 @@ class QbeastCatalogTest extends QbeastIntegrationTestSpec { students.toDF() } + private val schema = StructType( + Seq( + StructField("id", IntegerType, true), + StructField("name", StringType, true), + StructField("age", IntegerType, true))) + "QbeastCatalog" should "coexist with Delta tables" in withTmpDir(tmpDir => withExtendedSpark(sparkConf = new SparkConf() @@ -51,4 +58,86 @@ class QbeastCatalogTest extends QbeastIntegrationTestSpec { })) + it should "crate table" in withQbeastContextSparkAndTmpWarehouse((spark, _) => { + + spark.sql( + "CREATE TABLE student (id INT, name STRING, age INT) " + + "USING qbeast OPTIONS ('columnsToIndex'='id')") + + val table = spark.table("student") + table.schema shouldBe schema + table.count() shouldBe 0 + + }) + + it should "replace table" in withQbeastContextSparkAndTmpWarehouse((spark, tmpDir) => { + + // Create table first (must be in qbeast format) + spark.sql( + "CREATE TABLE student (id INT, name STRING, age INT) " + + "USING qbeast OPTIONS ('columnsToIndex'='age')") + + spark.sql("SHOW TABLES").count() shouldBe 1 + + // Try to replace it + spark.sql( + "REPLACE TABLE student (id INT, name STRING, age INT) " + + "USING qbeast OPTIONS ('columnsToIndex'='age')") + + spark.sql("SHOW TABLES").count() shouldBe 1 + + val table = spark.read.table("student") + table.schema shouldBe schema + table.count() shouldBe 0 + + }) + + it should "create or replace table" in withQbeastContextSparkAndTmpWarehouse( + (spark, tmpDir) => { + + spark.sql( + "CREATE OR REPLACE TABLE student (id INT, name STRING, age INT)" + + " USING qbeast OPTIONS ('columnsToIndex'='id')") + + val table = spark.read.table("student") + table.schema shouldBe schema + table.count() shouldBe 0 + + }) + + // TODO this test fails because the data has schema (col1, col2, col3) Check how to solve it +// it should "create table and insert data" in withQbeastContextSparkAndTmpWarehouse( +// (spark, tmpDir) => { +// +// spark.sql( +// "CREATE OR REPLACE TABLE student (id INT, name STRING, age INT)" + +// " USING qbeast OPTIONS ('columnsToIndex'='id')") +// +// // Insert one single element +// spark.sql("INSERT INTO table student values (4, 'Joan', 20)") +// val table = spark.sql("SELECT * FROM student") +// table.schema shouldBe schema +// table.count() shouldBe 1 +// +// }) + + it should "create table and insert data as select" in withQbeastContextSparkAndTmpWarehouse( + (spark, tmpDir) => { + + spark.sql( + "CREATE OR REPLACE TABLE student (id INT, name STRING, age INT)" + + " USING qbeast OPTIONS ('columnsToIndex'='id')") + + import spark.implicits._ + // Create temp view with data to try SELECT AS statement + students.toDF.createOrReplaceTempView("bronze_student") + + spark.sql("INSERT INTO table student SELECT * FROM bronze_student") + spark.sql("SELECT * FROM student").count() shouldBe students.size + + spark.sql("INSERT INTO table student TABLE bronze_student") + spark.sql("SELECT * FROM student").count() shouldBe students.size * 2 + + }) + } From 737efdadd7c9df40aa28b4eab2206ed20f625f14 Mon Sep 17 00:00:00 2001 From: osopardo1 Date: Tue, 20 Sep 2022 18:02:07 +0200 Subject: [PATCH 38/49] Update readme and add SupportsTruncate --- README.md | 4 +--- .../sources/catalog/TableCreationMode.scala | 3 +++ .../internal/sources/v2/QbeastWriteBuilder.scala | 9 ++++++++- .../internal/sources/QbeastCatalogTest.scala | 16 ---------------- 4 files changed, 12 insertions(+), 20 deletions(-) diff --git a/README.md b/README.md index 1c58f63e3..545ddf3c8 100644 --- a/README.md +++ b/README.md @@ -133,9 +133,7 @@ Use **`INSERT INTO`** to add records to the new table. It will update the index ```scala -spark.sql("INSERT INTO table student SELECT * from oldStudentsTable") - -spark.sql("INSERT INTO table student (id, name, age) values ((4, 'Joan', 20))") +spark.sql("INSERT INTO table student SELECT * FROM visitor_students") ``` diff --git a/src/main/scala/io/qbeast/spark/internal/sources/catalog/TableCreationMode.scala b/src/main/scala/io/qbeast/spark/internal/sources/catalog/TableCreationMode.scala index 6ccbf32e4..bfdba4f50 100644 --- a/src/main/scala/io/qbeast/spark/internal/sources/catalog/TableCreationMode.scala +++ b/src/main/scala/io/qbeast/spark/internal/sources/catalog/TableCreationMode.scala @@ -1,3 +1,6 @@ +/* + * Copyright 2021 Qbeast Analytics, S.L. + */ package io.qbeast.spark.internal.sources.catalog import org.apache.spark.sql.SaveMode diff --git a/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastWriteBuilder.scala b/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastWriteBuilder.scala index e61b19b48..b8aff40ef 100644 --- a/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastWriteBuilder.scala +++ b/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastWriteBuilder.scala @@ -8,6 +8,7 @@ import org.apache.spark.sql.DataFrame import org.apache.spark.sql.connector.write.{ LogicalWriteInfo, SupportsOverwrite, + SupportsTruncate, V1Write, WriteBuilder } @@ -26,7 +27,8 @@ class QbeastWriteBuilder( properties: Map[String, String], indexedTable: IndexedTable) extends WriteBuilder - with SupportsOverwrite { + with SupportsOverwrite + with SupportsTruncate { private var forceOverwrite = false @@ -37,6 +39,11 @@ class QbeastWriteBuilder( this } + override def truncate(): WriteBuilder = { + forceOverwrite = true + this + } + /** * Build an InsertableRelation to be able to write the data in QbeastFormat * @return the InsertableRelation with the corresponding method diff --git a/src/test/scala/io/qbeast/spark/internal/sources/QbeastCatalogTest.scala b/src/test/scala/io/qbeast/spark/internal/sources/QbeastCatalogTest.scala index b44a8eea0..d1518ecf2 100644 --- a/src/test/scala/io/qbeast/spark/internal/sources/QbeastCatalogTest.scala +++ b/src/test/scala/io/qbeast/spark/internal/sources/QbeastCatalogTest.scala @@ -105,22 +105,6 @@ class QbeastCatalogTest extends QbeastIntegrationTestSpec { }) - // TODO this test fails because the data has schema (col1, col2, col3) Check how to solve it -// it should "create table and insert data" in withQbeastContextSparkAndTmpWarehouse( -// (spark, tmpDir) => { -// -// spark.sql( -// "CREATE OR REPLACE TABLE student (id INT, name STRING, age INT)" + -// " USING qbeast OPTIONS ('columnsToIndex'='id')") -// -// // Insert one single element -// spark.sql("INSERT INTO table student values (4, 'Joan', 20)") -// val table = spark.sql("SELECT * FROM student") -// table.schema shouldBe schema -// table.count() shouldBe 1 -// -// }) - it should "create table and insert data as select" in withQbeastContextSparkAndTmpWarehouse( (spark, tmpDir) => { From 93d4e7b4ca244c1b2f1d11d58b725b4bcf65ffe7 Mon Sep 17 00:00:00 2001 From: Jiawei Date: Wed, 28 Sep 2022 15:21:17 +0200 Subject: [PATCH 39/49] Shorten code comment --- .../spark/internal/sources/catalog/QbeastCatalog.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalog.scala b/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalog.scala index 9727ad457..b45122e8f 100644 --- a/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalog.scala +++ b/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalog.scala @@ -81,10 +81,10 @@ class QbeastCatalog[T <: TableCatalog with SupportsNamespaces] } /** - * For StageReplace, StageReplaceOrCreate and StageCreate, the following pipeline is executed - * 1. Check if it's Qbeast Provider - * 2. Create a QbeastStagedTable. This type of table allows to commit the changes atomically to the Catalog. - * 3. If it was not a QbeastProvider, it outputs a DefaultStagedTable + * For StageReplace, StageReplaceOrCreate and StageCreate, the following pipeline is executed: + * 1. Check if it's a Qbeast Provider + * 2. If true, it creates a QbeastStagedTable, which allows atomizing the changes to the Catalog. + * 3. Otherwise, output a DefaultStagedTable */ override def stageReplace( From d0f84cb8cf8b515b08e55237895710814a227eaf Mon Sep 17 00:00:00 2001 From: osopardo1 Date: Wed, 5 Oct 2022 16:01:04 +0200 Subject: [PATCH 40/49] Add basic test for Catalog and DefaultStagedTable We delegate a lot of methods to the Session Catalog. We need to verify they work properly with unit tests. Add some minor fixes as well --- .../internal/sources/QbeastBaseRelation.scala | 18 +- .../sources/catalog/QbeastCatalog.scala | 19 +- .../internal/sources/v2/QbeastTableImpl.scala | 2 +- .../sources/QbeastBaseRelationTest.scala | 67 ++++++ .../sources/catalog/CatalogTestSuite.scala | 55 +++++ .../catalog/DefaultStagedTableTest.scala | 139 ++++++++++++ .../QbeastCatalogIntegrationTest.scala} | 30 +-- .../sources/catalog/QbeastCatalogTest.scala | 198 ++++++++++++++++++ 8 files changed, 490 insertions(+), 38 deletions(-) create mode 100644 src/test/scala/io/qbeast/spark/internal/sources/QbeastBaseRelationTest.scala create mode 100644 src/test/scala/io/qbeast/spark/internal/sources/catalog/CatalogTestSuite.scala create mode 100644 src/test/scala/io/qbeast/spark/internal/sources/catalog/DefaultStagedTableTest.scala rename src/test/scala/io/qbeast/spark/internal/sources/{QbeastCatalogTest.scala => catalog/QbeastCatalogIntegrationTest.scala} (83%) create mode 100644 src/test/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalogTest.scala diff --git a/src/main/scala/io/qbeast/spark/internal/sources/QbeastBaseRelation.scala b/src/main/scala/io/qbeast/spark/internal/sources/QbeastBaseRelation.scala index cdb14c029..b3672d5b9 100644 --- a/src/main/scala/io/qbeast/spark/internal/sources/QbeastBaseRelation.scala +++ b/src/main/scala/io/qbeast/spark/internal/sources/QbeastBaseRelation.scala @@ -31,7 +31,10 @@ object QbeastBaseRelation { * @param sqlContext the SQLContext * @return the HadoopFsRelation */ - def createRelation(sqlContext: SQLContext, table: IndexedTable): BaseRelation = { + def createRelation( + sqlContext: SQLContext, + table: IndexedTable, + options: Map[String, String]): BaseRelation = { val spark = SparkSession.active val tableID = table.tableID @@ -40,15 +43,16 @@ object QbeastBaseRelation { if (snapshot.isInitial) { // If the Table is initial, read empty relation // This could happen if we CREATE/REPLACE TABLE without inserting data + // In this case, we use the options variable new HadoopFsRelation( OTreeIndex(spark, new Path(tableID.id)), partitionSchema = StructType(Seq.empty[StructField]), dataSchema = schema, bucketSpec = None, new ParquetFileFormat(), - Map.empty)(spark) with InsertableRelation { + options)(spark) with InsertableRelation { def insert(data: DataFrame, overwrite: Boolean): Unit = { - table.save(data, Map.empty, append = !overwrite) + table.save(data, options, append = !overwrite) } } } else { @@ -85,10 +89,14 @@ object QbeastBaseRelation { * @return BaseRelation for the new table in Qbeast format */ def forQbeastTable(indexedTable: IndexedTable): BaseRelation = { + forQbeastTableWithOptions(indexedTable, Map.empty) + } + def forQbeastTableWithOptions( + indexedTable: IndexedTable, + withOptions: Map[String, String]): BaseRelation = { val spark = SparkSession.active - createRelation(spark.sqlContext, indexedTable) - + createRelation(spark.sqlContext, indexedTable, withOptions) } } diff --git a/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalog.scala b/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalog.scala index b45122e8f..b4fe4a04b 100644 --- a/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalog.scala +++ b/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalog.scala @@ -7,18 +7,7 @@ import io.qbeast.context.QbeastContext import io.qbeast.spark.internal.QbeastOptions.checkQbeastProperties import io.qbeast.spark.internal.sources.v2.QbeastStagedTableImpl import org.apache.spark.sql.{SparkCatalogUtils, SparkSession} -import org.apache.spark.sql.connector.catalog.{ - CatalogExtension, - CatalogPlugin, - Identifier, - NamespaceChange, - StagedTable, - StagingTableCatalog, - SupportsNamespaces, - Table, - TableCatalog, - TableChange -} +import org.apache.spark.sql.connector.catalog._ import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -101,6 +90,9 @@ class QbeastCatalog[T <: TableCatalog with SupportsNamespaces] properties, tableFactory) } else { + if (getSessionCatalog().tableExists(ident)) { + getSessionCatalog().dropTable(ident) + } DefaultStagedTable( ident, getSessionCatalog().createTable(ident, schema, partitions, properties), @@ -122,6 +114,9 @@ class QbeastCatalog[T <: TableCatalog with SupportsNamespaces] properties, tableFactory) } else { + if (getSessionCatalog().tableExists(ident)) { + getSessionCatalog().dropTable(ident) + } DefaultStagedTable( ident, getSessionCatalog().createTable(ident, schema, partitions, properties), diff --git a/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastTableImpl.scala b/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastTableImpl.scala index 2e2cb23ef..c5c72f3fb 100644 --- a/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastTableImpl.scala +++ b/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastTableImpl.scala @@ -60,7 +60,7 @@ class QbeastTableImpl private[sources] ( } def toBaseRelation: BaseRelation = { - QbeastBaseRelation.forQbeastTable(indexedTable) + QbeastBaseRelation.forQbeastTableWithOptions(indexedTable, properties().asScala.toMap) } override def properties(): util.Map[String, String] = options.asJava diff --git a/src/test/scala/io/qbeast/spark/internal/sources/QbeastBaseRelationTest.scala b/src/test/scala/io/qbeast/spark/internal/sources/QbeastBaseRelationTest.scala new file mode 100644 index 000000000..c21d1cec6 --- /dev/null +++ b/src/test/scala/io/qbeast/spark/internal/sources/QbeastBaseRelationTest.scala @@ -0,0 +1,67 @@ +package io.qbeast.spark.internal.sources + +import io.qbeast.TestClasses.Student +import io.qbeast.context.QbeastContext +import io.qbeast.core.model.QTableID +import io.qbeast.spark.QbeastIntegrationTestSpec +import org.apache.spark.sql.execution.datasources.HadoopFsRelation +import org.apache.spark.sql.sources.InsertableRelation + +import scala.util.Random + +class QbeastBaseRelationTest extends QbeastIntegrationTestSpec { + + "QbeastBaseRelation" should "output a HadoopFsRelation with Insertable" in withSparkAndTmpDir( + (spark, tmpDir) => { + import spark.implicits._ + val df = + 1.to(10).map(i => Student(i, i.toString, Random.nextInt())).toDF("id", "name", "age") + + df.write.format("qbeast").option("columnsToIndex", "id").save(tmpDir) + + val indexedTable = QbeastContext.indexedTableFactory.getIndexedTable(QTableID(tmpDir)) + QbeastBaseRelation + .forQbeastTable(indexedTable) shouldBe a[HadoopFsRelation with InsertableRelation] + }) + + it should "save new data" in withSparkAndTmpDir((spark, tmpDir) => { + import spark.implicits._ + val df = + 1.to(10).map(i => Student(i, i.toString, Random.nextInt())).toDF("id", "name", "age") + + df.write.format("qbeast").option("columnsToIndex", "id").save(tmpDir) + + val indexedTable = QbeastContext.indexedTableFactory.getIndexedTable(QTableID(tmpDir)) + val qbeastBaseRelation = QbeastBaseRelation.forQbeastTable(indexedTable) + + qbeastBaseRelation.asInstanceOf[InsertableRelation].insert(df, false) + + val indexed = spark.read.format("qbeast").load(tmpDir) + indexed.count() shouldBe df.count() * 2 // we write two times the data + + }) + + it should "save new data on empty table" in withSparkAndTmpDir((spark, tmpDir) => { + import spark.implicits._ + + // Create Staged Table with Spark SQL + spark.sql( + s"CREATE TABLE student (id INT, name STRING, age INT) USING qbeast " + + "OPTIONS ('columnsToIndex'='id')") + + val indexedTable = QbeastContext.indexedTableFactory.getIndexedTable(QTableID(tmpDir)) + val qbeastBaseRelation = + QbeastBaseRelation.forQbeastTableWithOptions(indexedTable, Map("columnsToIndex" -> "id")) + + // Insert new data + val df = + 1.to(10).map(i => Student(i, i.toString, Random.nextInt())).toDF("id", "name", "age") + + qbeastBaseRelation.asInstanceOf[InsertableRelation].insert(df, false) + + val indexed = spark.read.format("qbeast").load(tmpDir) + indexed.count() shouldBe df.count() + + }) + +} diff --git a/src/test/scala/io/qbeast/spark/internal/sources/catalog/CatalogTestSuite.scala b/src/test/scala/io/qbeast/spark/internal/sources/catalog/CatalogTestSuite.scala new file mode 100644 index 000000000..2dfd5b3d5 --- /dev/null +++ b/src/test/scala/io/qbeast/spark/internal/sources/catalog/CatalogTestSuite.scala @@ -0,0 +1,55 @@ +package io.qbeast.spark.internal.sources.catalog + +import io.qbeast.TestClasses.Student +import org.apache.spark.sql.{DataFrame, SparkCatalogUtils, SparkSession} +import org.apache.spark.sql.connector.catalog.{ + StagingTableCatalog, + SupportsNamespaces, + TableCatalog +} +import org.apache.spark.sql.types.{IntegerType, StringType, StructField, StructType} + +import scala.collection.immutable +import scala.util.Random + +/** + * A test suite for Catalog tests. It includes: + * - Creation of Student's dataframe + * - Creation of QbeastCatalog with a delegated session catalog + * - Schema of the Student's dataframe + */ +trait CatalogTestSuite { + + val schema: StructType = StructType( + Seq( + StructField("id", IntegerType, true), + StructField("name", StringType, true), + StructField("age", IntegerType, true))) + + val defaultNamespace: Array[String] = Array("default") + + val students: immutable.Seq[Student] = { + 1.to(10).map(i => Student(i, i.toString, Random.nextInt())) + } + + def sessionCatalog(spark: SparkSession): TableCatalog = { + SparkCatalogUtils.getV2SessionCatalog(spark).asInstanceOf[TableCatalog] + + } + + def createTestData(spark: SparkSession): DataFrame = { + import spark.implicits._ + students.toDF() + } + + def createQbeastCatalog( + spark: SparkSession): TableCatalog with SupportsNamespaces with StagingTableCatalog = { + val qbeastCatalog = new QbeastCatalog + + // set default catalog + qbeastCatalog.setDelegateCatalog(sessionCatalog(spark)) + + qbeastCatalog + } + +} diff --git a/src/test/scala/io/qbeast/spark/internal/sources/catalog/DefaultStagedTableTest.scala b/src/test/scala/io/qbeast/spark/internal/sources/catalog/DefaultStagedTableTest.scala new file mode 100644 index 000000000..da64ca1d1 --- /dev/null +++ b/src/test/scala/io/qbeast/spark/internal/sources/catalog/DefaultStagedTableTest.scala @@ -0,0 +1,139 @@ +package io.qbeast.spark.internal.sources.catalog + +import io.qbeast.spark.QbeastIntegrationTestSpec +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.connector.catalog.Identifier +import org.apache.spark.sql.connector.write.LogicalWriteInfo +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +import scala.collection.JavaConverters._ + +class DefaultStagedTableTest extends QbeastIntegrationTestSpec with CatalogTestSuite { + + "A DefaultStagedTable" should "be returned by default" in withQbeastContextSparkAndTmpWarehouse( + (spark, _) => { + val qbeastCatalog = createQbeastCatalog(spark) + val tableIdentifier = Identifier.of(defaultNamespace, "student") + + qbeastCatalog.stageCreate( + tableIdentifier, + schema, + Array.empty, + Map.empty[String, String].asJava) shouldBe a[DefaultStagedTable] + + qbeastCatalog.stageReplace( + tableIdentifier, + schema, + Array.empty, + Map.empty[String, String].asJava) shouldBe a[DefaultStagedTable] + + qbeastCatalog.stageCreateOrReplace( + tableIdentifier, + schema, + Array.empty, + Map.empty[String, String].asJava) shouldBe a[DefaultStagedTable] + }) + + it should "output table properties" in withQbeastContextSparkAndTmpWarehouse((spark, _) => { + val tableIdentifier = Identifier.of(Array("default"), "students") + val catalog = sessionCatalog(spark) + val underlyingTable = + catalog.createTable(tableIdentifier, schema, Array.empty, Map.empty[String, String].asJava) + + val defaultStagedTable = DefaultStagedTable + .apply(tableIdentifier, underlyingTable, catalog) + + defaultStagedTable.properties() shouldBe underlyingTable.properties() + }) + + it should "output partitioning" in withQbeastContextSparkAndTmpWarehouse((spark, _) => { + val tableIdentifier = Identifier.of(Array("default"), "students") + val catalog = sessionCatalog(spark) + val underlyingTable = + catalog.createTable(tableIdentifier, schema, Array.empty, Map.empty[String, String].asJava) + + val defaultStagedTable = DefaultStagedTable + .apply(tableIdentifier, underlyingTable, catalog) + + defaultStagedTable.partitioning() shouldBe underlyingTable.partitioning() + }) + + it should "output capabilities" in withQbeastContextSparkAndTmpWarehouse((spark, _) => { + val tableIdentifier = Identifier.of(Array("default"), "students") + val catalog = sessionCatalog(spark) + val underlyingTable = + catalog.createTable(tableIdentifier, schema, Array.empty, Map.empty[String, String].asJava) + + val defaultStagedTable = DefaultStagedTable + .apply(tableIdentifier, underlyingTable, catalog) + + defaultStagedTable.capabilities() shouldBe underlyingTable.capabilities() + }) + + it should "output schema" in withQbeastContextSparkAndTmpWarehouse((spark, tmpWarehouse) => { + val tableIdentifier = Identifier.of(Array("default"), "students") + val catalog = sessionCatalog(spark) + val underlyingTable = + catalog.createTable(tableIdentifier, schema, Array.empty, Map.empty[String, String].asJava) + + val defaultStagedTable = DefaultStagedTable + .apply(tableIdentifier, underlyingTable, catalog) + + defaultStagedTable.schema() shouldBe schema + }) + + it should "output name" in withQbeastContextSparkAndTmpWarehouse((spark, tmpWarehouse) => { + val tableIdentifier = Identifier.of(Array("default"), "students") + val catalog = sessionCatalog(spark) + val underlyingTable = + catalog.createTable(tableIdentifier, schema, Array.empty, Map.empty[String, String].asJava) + + val defaultStagedTable = DefaultStagedTable + .apply(tableIdentifier, underlyingTable, catalog) + + defaultStagedTable.name() shouldBe "students" + }) + + it should "drop table on abort" in withQbeastContextSparkAndTmpWarehouse( + (spark, tmpWarehouse) => { + val tableIdentifier = Identifier.of(Array("default"), "students") + val catalog = sessionCatalog(spark) + val underlyingTable = catalog.createTable( + tableIdentifier, + schema, + Array.empty, + Map.empty[String, String].asJava) + + val defaultStagedTable = DefaultStagedTable + .apply(tableIdentifier, underlyingTable, catalog) + + defaultStagedTable.abortStagedChanges() + catalog.listTables(defaultNamespace) shouldBe Array() + }) + + it should "throw exception when creating default WriteBuilder" in + withQbeastContextSparkAndTmpWarehouse((spark, tmpWarehouse) => { + val tableIdentifier = Identifier.of(Array("default"), "students") + val catalog = sessionCatalog(spark) + val underlyingTable = catalog.createTable( + tableIdentifier, + schema, + Array.empty, + Map.empty[String, String].asJava) + + val defaultStagedTable = DefaultStagedTable + .apply(tableIdentifier, underlyingTable, catalog) + + val logicalWriteInfo = new LogicalWriteInfo { + override def options(): CaseInsensitiveStringMap = CaseInsensitiveStringMap.empty() + + override def queryId(): String = "1" + + override def schema(): StructType = schema + } + + an[AnalysisException] shouldBe thrownBy( + defaultStagedTable.newWriteBuilder(logicalWriteInfo)) + }) +} diff --git a/src/test/scala/io/qbeast/spark/internal/sources/QbeastCatalogTest.scala b/src/test/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalogIntegrationTest.scala similarity index 83% rename from src/test/scala/io/qbeast/spark/internal/sources/QbeastCatalogTest.scala rename to src/test/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalogIntegrationTest.scala index d1518ecf2..e884bb689 100644 --- a/src/test/scala/io/qbeast/spark/internal/sources/QbeastCatalogTest.scala +++ b/src/test/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalogIntegrationTest.scala @@ -1,27 +1,10 @@ -package io.qbeast.spark.internal.sources +package io.qbeast.spark.internal.sources.catalog -import io.qbeast.TestClasses.Student import io.qbeast.spark.QbeastIntegrationTestSpec import org.apache.spark.SparkConf -import org.apache.spark.sql.types.{IntegerType, StringType, StructField, StructType} -import org.apache.spark.sql.{DataFrame, SparkSession} +import org.apache.spark.sql.AnalysisException -import scala.util.Random - -class QbeastCatalogTest extends QbeastIntegrationTestSpec { - - private val students = 1.to(10).map(i => Student(i, i.toString, Random.nextInt())) - - private def createTestData(spark: SparkSession): DataFrame = { - import spark.implicits._ - students.toDF() - } - - private val schema = StructType( - Seq( - StructField("id", IntegerType, true), - StructField("name", StringType, true), - StructField("age", IntegerType, true))) +class QbeastCatalogIntegrationTest extends QbeastIntegrationTestSpec with CatalogTestSuite { "QbeastCatalog" should "coexist with Delta tables" in withTmpDir(tmpDir => @@ -124,4 +107,11 @@ class QbeastCatalogTest extends QbeastIntegrationTestSpec { }) + it should "throw an error when no columnsToIndex is specified" in + withQbeastContextSparkAndTmpWarehouse((spark, tmpDir) => { + an[AnalysisException] shouldBe thrownBy( + spark.sql("CREATE OR REPLACE TABLE student (id INT, name STRING, age INT)" + + " USING qbeast")) + }) + } diff --git a/src/test/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalogTest.scala b/src/test/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalogTest.scala new file mode 100644 index 000000000..5d6782643 --- /dev/null +++ b/src/test/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalogTest.scala @@ -0,0 +1,198 @@ +package io.qbeast.spark.internal.sources.catalog + +import io.qbeast.spark.QbeastIntegrationTestSpec +import org.apache.spark.sql.connector.catalog.{Identifier, NamespaceChange, TableChange} +import org.apache.spark.sql.types.{IntegerType, StructField, StructType} +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +import scala.collection.JavaConverters._ + +class QbeastCatalogTest extends QbeastIntegrationTestSpec with CatalogTestSuite { + + "Qbeast catalog" should "create table" in withQbeastContextSparkAndTmpWarehouse((spark, _) => { + + val qbeastCatalog = createQbeastCatalog(spark) + val tableIdentifier = Identifier.of(defaultNamespace, "student") + + qbeastCatalog.createTable( + tableIdentifier, + schema, + Array.empty, + Map.empty[String, String].asJava) + + qbeastCatalog.listTables(defaultNamespace) shouldBe Array(tableIdentifier) + }) + + it should "replace table" in withQbeastContextSparkAndTmpWarehouse((spark, _) => { + + val qbeastCatalog = createQbeastCatalog(spark) + val tableIdentifier = Identifier.of(defaultNamespace, "student") + + qbeastCatalog.createTable( + tableIdentifier, + schema, + Array.empty, + Map.empty[String, String].asJava) + + val newSchema = schema.add(StructField("newCol", IntegerType, false)) + qbeastCatalog.stageReplace( + tableIdentifier, + newSchema, + Array.empty, + Map.empty[String, String].asJava) + + qbeastCatalog.listTables(defaultNamespace) shouldBe Array(tableIdentifier) + qbeastCatalog.loadTable(tableIdentifier).schema() shouldBe newSchema + + }) + + it should "create or replace table" in withQbeastContextSparkAndTmpWarehouse((spark, _) => { + + val qbeastCatalog = createQbeastCatalog(spark) + val tableIdentifier = Identifier.of(defaultNamespace, "student") + + qbeastCatalog.stageCreateOrReplace( + tableIdentifier, + schema, + Array.empty, + Map.empty[String, String].asJava) + + qbeastCatalog.listTables(defaultNamespace) shouldBe Array(tableIdentifier) + + val newSchema = schema.add(StructField("newCol", IntegerType, false)) + qbeastCatalog.stageCreateOrReplace( + tableIdentifier, + newSchema, + Array.empty, + Map.empty[String, String].asJava) + + qbeastCatalog.listTables(defaultNamespace) shouldBe Array(tableIdentifier) + qbeastCatalog.loadTable(tableIdentifier).schema() shouldBe newSchema + }) + + it should "list tables" in withQbeastContextSparkAndTmpWarehouse((spark, _) => { + val qbeastCatalog = createQbeastCatalog(spark) + qbeastCatalog.listTables(defaultNamespace) shouldBe Array() + }) + + it should "alter table" in withQbeastContextSparkAndTmpWarehouse((spark, _) => { + val qbeastCatalog = createQbeastCatalog(spark) + val tableIdentifier = Identifier.of(defaultNamespace, "student") + qbeastCatalog.createTable( + tableIdentifier, + schema, + Array.empty, + Map.empty[String, String].asJava) + + // Alter table with new information + qbeastCatalog.alterTable( + tableIdentifier, + TableChange.addColumn(Array("x"), IntegerType, false)) + + val modifiedSchema = StructType(schema.fields ++ Seq(StructField("x", IntegerType, false))) + qbeastCatalog + .loadTable(Identifier.of(defaultNamespace, "student")) + .schema() shouldBe modifiedSchema + }) + + it should "drop table" in withQbeastContextSparkAndTmpWarehouse((spark, _) => { + val qbeastCatalog = createQbeastCatalog(spark) + val tableIdentifier = Identifier.of(defaultNamespace, "student") + qbeastCatalog.createTable( + tableIdentifier, + schema, + Array.empty, + Map.empty[String, String].asJava) + qbeastCatalog.listTables(defaultNamespace) shouldBe Array(tableIdentifier) + + // Drop table + qbeastCatalog.dropTable(tableIdentifier) + + qbeastCatalog.listTables(defaultNamespace) shouldBe Array() + + }) + it should "rename table" in withQbeastContextSparkAndTmpWarehouse((spark, _) => { + val qbeastCatalog = createQbeastCatalog(spark) + val tableIdentifier = Identifier.of(defaultNamespace, "student") + qbeastCatalog.createTable( + tableIdentifier, + schema, + Array.empty, + Map.empty[String, String].asJava) + qbeastCatalog.listTables(defaultNamespace) shouldBe Array(tableIdentifier) + + // Rename table + val newTableIdentifier = Identifier.of(defaultNamespace, "new_students") + qbeastCatalog.renameTable(tableIdentifier, newTableIdentifier) + + qbeastCatalog.listTables(defaultNamespace) shouldBe Array(newTableIdentifier) + }) + + it should "list all namespaces" in withQbeastContextSparkAndTmpWarehouse((spark, _) => { + val qbeastCatalog = createQbeastCatalog(spark) + qbeastCatalog.listNamespaces() shouldBe Array(defaultNamespace) + }) + + it should "create namespace" in withQbeastContextSparkAndTmpWarehouse((spark, _) => { + val qbeastCatalog = createQbeastCatalog(spark) + qbeastCatalog.createNamespace(Array("new_namespace"), Map.empty[String, String].asJava) + + qbeastCatalog.listNamespaces() shouldBe Array(defaultNamespace, Array("new_namespace")) + }) + + it should "list specific namespaces" in withQbeastContextSparkAndTmpWarehouse((spark, _) => { + + val qbeastCatalog = createQbeastCatalog(spark) + + qbeastCatalog.listNamespaces(defaultNamespace) shouldBe Array() + + }) + + it should "load namespace metadata" in withQbeastContextSparkAndTmpWarehouse( + (spark, tmpLocation) => { + val qbeastCatalog = createQbeastCatalog(spark) + qbeastCatalog.loadNamespaceMetadata(defaultNamespace) shouldBe Map( + "comment" -> "default database", + "location" -> ("file:" + tmpLocation)).asJava + }) + + it should "alter namespace" in withQbeastContextSparkAndTmpWarehouse((spark, tmpLocation) => { + val qbeastCatalog = createQbeastCatalog(spark) + val newNamespace = Array("new_namespace") + qbeastCatalog.createNamespace(newNamespace, Map.empty[String, String].asJava) + + // Alter namespace + qbeastCatalog.alterNamespace( + newNamespace, + NamespaceChange.setProperty("newPropertie", "newValue")) + + qbeastCatalog.loadNamespaceMetadata(newNamespace) shouldBe Map( + "comment" -> "", + "location" -> ("file:" + tmpLocation + "/new_namespace.db"), + "newPropertie" -> "newValue").asJava + + }) + + it should "drop namespace" in withQbeastContextSparkAndTmpWarehouse((spark, _) => { + val qbeastCatalog = createQbeastCatalog(spark) + val newNamespace = Array("new_namespace") + qbeastCatalog.createNamespace(newNamespace, Map.empty[String, String].asJava) + + qbeastCatalog.listNamespaces() shouldBe Array(defaultNamespace, Array("new_namespace")) + + // Drop Namespace + qbeastCatalog.dropNamespace(newNamespace) + + qbeastCatalog.listNamespaces() shouldBe Array(defaultNamespace) + + }) + + it should "output correct name" in withQbeastContextSparkAndTmpWarehouse((spark, _) => { + + val qbeastCatalog = createQbeastCatalog(spark) + qbeastCatalog.name() shouldBe null + qbeastCatalog.initialize("newName", CaseInsensitiveStringMap.empty()) + qbeastCatalog.name() shouldBe "newName" + }) + +} From de86bd8aef3147b572c6dc82e3a09bb249517ed8 Mon Sep 17 00:00:00 2001 From: osopardo1 Date: Thu, 6 Oct 2022 14:46:02 +0200 Subject: [PATCH 41/49] Test for QbeastStagedTableTest and other exceptions tests --- .../sources/catalog/QbeastCatalogUtils.scala | 30 ++----- .../sources/v2/QbeastStagedTableImpl.scala | 4 +- .../sources/QbeastStagedTableTest.scala | 90 +++++++++++++++++++ .../QbeastCatalogIntegrationTest.scala | 22 +++++ .../sources/catalog/QbeastCatalogTest.scala | 21 ++++- .../utils/QbeastSQLIntegrationTest.scala | 16 +++- 6 files changed, 156 insertions(+), 27 deletions(-) create mode 100644 src/test/scala/io/qbeast/spark/internal/sources/QbeastStagedTableTest.scala diff --git a/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalogUtils.scala b/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalogUtils.scala index c64d4ae5c..c99731f80 100644 --- a/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalogUtils.scala +++ b/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalogUtils.scala @@ -53,12 +53,7 @@ object QbeastCatalogUtils { * @return */ def isPathTable(ident: Identifier): Boolean = { - try { - spark.sessionState.conf.runSQLonFile && hasQbeastNamespace(ident) && new Path( - ident.name()).isAbsolute - } catch { - case _: IllegalArgumentException => false - } + new Path(ident.name()).isAbsolute } def isPathTable(identifier: TableIdentifier): Boolean = { @@ -121,21 +116,6 @@ object QbeastCatalogUtils { tableFactory: IndexedTableFactory, existingSessionCatalog: SessionCatalog): Unit = { - /* - // These two keys are tableProperties in data source v2 but not in v1, so we have to filter - // them out. Otherwise property consistency checks will fail. - val tableProperties = allTableProperties.asScala.filterKeys { - case TableCatalog.PROP_LOCATION => false - case TableCatalog.PROP_PROVIDER => false - case TableCatalog.PROP_COMMENT => false - case TableCatalog.PROP_OWNER => false - case TableCatalog.PROP_EXTERNAL => false - case "path" => false - case _ => true - } - - */ - val isPathTable = QbeastCatalogUtils.isPathTable(ident) if (isPathTable @@ -153,8 +133,11 @@ object QbeastCatalogUtils { } else { Option(allTableProperties.get("location")) } - val id = TableIdentifier(ident.name(), ident.namespace().lastOption) + val tableType = + if (location.isDefined) CatalogTableType.EXTERNAL else CatalogTableType.MANAGED val locUriOpt = location.map(CatalogUtils.stringToURI) + + val id = TableIdentifier(ident.name(), ident.namespace().lastOption) val existingTableOpt = QbeastCatalogUtils.getExistingTableIfExists(id, existingSessionCatalog) val loc = locUriOpt .orElse(existingTableOpt.flatMap(_.storage.locationUri)) @@ -163,8 +146,7 @@ object QbeastCatalogUtils { val storage = DataSource .buildStorageFormatFromOptions(writeOptions) .copy(locationUri = Option(loc)) - val tableType = - if (location.isDefined) CatalogTableType.EXTERNAL else CatalogTableType.MANAGED + val commentOpt = Option(allTableProperties.get("comment")) val (partitionColumns, bucketSpec) = SparkTransformUtils.convertTransforms(partitions) diff --git a/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastStagedTableImpl.scala b/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastStagedTableImpl.scala index a4a1c3365..9db5f25be 100644 --- a/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastStagedTableImpl.scala +++ b/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastStagedTableImpl.scala @@ -86,7 +86,9 @@ private[sources] class QbeastStagedTableImpl( } - override def abortStagedChanges(): Unit = {} + override def abortStagedChanges(): Unit = { + // Do nothing since any path is created until commitStagedChanges()) + } override def name(): String = ident.name() diff --git a/src/test/scala/io/qbeast/spark/internal/sources/QbeastStagedTableTest.scala b/src/test/scala/io/qbeast/spark/internal/sources/QbeastStagedTableTest.scala new file mode 100644 index 000000000..0ed92ff42 --- /dev/null +++ b/src/test/scala/io/qbeast/spark/internal/sources/QbeastStagedTableTest.scala @@ -0,0 +1,90 @@ +package io.qbeast.spark.internal.sources + +import io.qbeast.spark.QbeastIntegrationTestSpec +import io.qbeast.spark.internal.sources.catalog.CatalogTestSuite +import io.qbeast.spark.internal.sources.v2.QbeastStagedTableImpl +import org.apache.hadoop.fs.Path +import org.apache.spark.sql.connector.catalog.Identifier +import org.apache.spark.sql.connector.catalog.TableCapability.V1_BATCH_WRITE + +import scala.collection.JavaConverters._ + +class QbeastStagedTableTest extends QbeastIntegrationTestSpec with CatalogTestSuite { + + "QbeastCatalog" should "return a QbeastStagedTableImpl when provider = qbeast" in + withQbeastContextSparkAndTmpWarehouse((spark, _) => { + val qbeastCatalog = createQbeastCatalog(spark) + qbeastCatalog + .stageCreate( + Identifier.of(Array("default"), "students"), + schema, + Array.empty, + Map("provider" -> "qbeast", "columnsToIndex" -> "id").asJava) shouldBe a[ + QbeastStagedTableImpl] + + }) + "QbeastStagedTable" should "retrieve schema" in withQbeastContextSparkAndTmpWarehouse( + (spark, _) => { + val qbeastCatalog = createQbeastCatalog(spark) + val qbeastStagedTable = + qbeastCatalog + .stageCreate( + Identifier.of(Array("default"), "students"), + schema, + Array.empty, + Map("provider" -> "qbeast", "columnsToIndex" -> "id").asJava) + .asInstanceOf[QbeastStagedTableImpl] + + qbeastStagedTable.schema() shouldBe schema + }) + + it should "retrieve name" in withQbeastContextSparkAndTmpWarehouse((spark, _) => { + val qbeastCatalog = createQbeastCatalog(spark) + val qbeastStagedTable = + qbeastCatalog + .stageCreate( + Identifier.of(Array("default"), "students"), + schema, + Array.empty, + Map("provider" -> "qbeast", "columnsToIndex" -> "id").asJava) + .asInstanceOf[QbeastStagedTableImpl] + + qbeastStagedTable.name() shouldBe "students" + }) + + it should "retrieve capabilities" in withQbeastContextSparkAndTmpWarehouse((spark, _) => { + val qbeastCatalog = createQbeastCatalog(spark) + val qbeastStagedTable = + qbeastCatalog + .stageCreate( + Identifier.of(Array("default"), "students"), + schema, + Array.empty, + Map("provider" -> "qbeast", "columnsToIndex" -> "id").asJava) + .asInstanceOf[QbeastStagedTableImpl] + + qbeastStagedTable.capabilities() shouldBe Set(V1_BATCH_WRITE).asJava + }) + + it should "clean path on abort changes" in withQbeastContextSparkAndTmpWarehouse( + (spark, tmpWarehouse) => { + + val qbeastCatalog = createQbeastCatalog(spark) + val qbeastStagedTable = + qbeastCatalog + .stageCreate( + Identifier.of(Array("default"), "students"), + schema, + Array.empty, + Map("provider" -> "qbeast", "columnsToIndex" -> "id").asJava) + .asInstanceOf[QbeastStagedTableImpl] + + qbeastStagedTable.abortStagedChanges() + + val path = new Path(tmpWarehouse) + val fs = path.getFileSystem(spark.sessionState.newHadoopConf()) + fs.exists(path.suffix("students")) shouldBe false + + }) + +} diff --git a/src/test/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalogIntegrationTest.scala b/src/test/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalogIntegrationTest.scala index e884bb689..01d183074 100644 --- a/src/test/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalogIntegrationTest.scala +++ b/src/test/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalogIntegrationTest.scala @@ -114,4 +114,26 @@ class QbeastCatalogIntegrationTest extends QbeastIntegrationTestSpec with Catalo " USING qbeast")) }) + it should "throw an error when trying to replace a non-qbeast table" in + withQbeastContextSparkAndTmpWarehouse((spark, tmpDir) => { + + spark.sql( + "CREATE TABLE student (id INT, name STRING, age INT)" + + " USING parquet") + + an[AnalysisException] shouldBe thrownBy( + spark.sql("REPLACE TABLE student (id INT, name STRING, age INT)" + + " USING qbeast OPTIONS ('columnsToIndex'='id')")) + + }) + + it should "throw an error when replacing non-existing table" in + withQbeastContextSparkAndTmpWarehouse((spark, tmpDir) => { + + an[AnalysisException] shouldBe thrownBy( + spark.sql("REPLACE TABLE student (id INT, name STRING, age INT)" + + " USING qbeast OPTIONS ('columnsToIndex'='id')")) + + }) + } diff --git a/src/test/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalogTest.scala b/src/test/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalogTest.scala index 5d6782643..f71dbf6d5 100644 --- a/src/test/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalogTest.scala +++ b/src/test/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalogTest.scala @@ -1,7 +1,13 @@ package io.qbeast.spark.internal.sources.catalog import io.qbeast.spark.QbeastIntegrationTestSpec -import org.apache.spark.sql.connector.catalog.{Identifier, NamespaceChange, TableChange} +import org.apache.spark.sql.connector.catalog.{ + CatalogExtension, + CatalogPlugin, + Identifier, + NamespaceChange, + TableChange +} import org.apache.spark.sql.types.{IntegerType, StructField, StructType} import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -195,4 +201,17 @@ class QbeastCatalogTest extends QbeastIntegrationTestSpec with CatalogTestSuite qbeastCatalog.name() shouldBe "newName" }) + it should "throw error when delegating wrong catalog" in withQbeastContextSparkAndTmpWarehouse( + (spark, _) => { + + val qbeastCatalog = createQbeastCatalog(spark).asInstanceOf[CatalogExtension] + val fakeCatalog = new CatalogPlugin { + override def initialize(name: String, options: CaseInsensitiveStringMap): Unit = {} + + override def name(): String = "fake" + } + + an[IllegalArgumentException] shouldBe thrownBy( + qbeastCatalog.setDelegateCatalog(fakeCatalog)) + }) } diff --git a/src/test/scala/io/qbeast/spark/utils/QbeastSQLIntegrationTest.scala b/src/test/scala/io/qbeast/spark/utils/QbeastSQLIntegrationTest.scala index 1c2e90d08..c022cf746 100644 --- a/src/test/scala/io/qbeast/spark/utils/QbeastSQLIntegrationTest.scala +++ b/src/test/scala/io/qbeast/spark/utils/QbeastSQLIntegrationTest.scala @@ -2,7 +2,7 @@ package io.qbeast.spark.utils import io.qbeast.TestClasses.Student import io.qbeast.spark.QbeastIntegrationTestSpec -import org.apache.spark.sql.{DataFrame, SparkSession} +import org.apache.spark.sql.{AnalysisException, DataFrame, SparkSession} import scala.util.Random @@ -106,6 +106,20 @@ class QbeastSQLIntegrationTest extends QbeastIntegrationTestSpec { }) + it should "throw an error when using different path locations" in + withQbeastContextSparkAndTmpWarehouse((spark, tmpDir) => { + + val data = createTestData(spark) + data.createOrReplaceTempView("data") + + an[AnalysisException] shouldBe thrownBy( + spark.sql( + s"CREATE OR REPLACE TABLE student USING qbeast " + + s"OPTIONS ('columnsToIndex'='id','location'='$tmpDir/new') " + + s"LOCATION '$tmpDir' ")) + + }) + it should "support INSERT INTO on a managed Table" in withQbeastContextSparkAndTmpWarehouse { (spark, _) => { From c46ffbd3313a154ded5cf8fc98083a8f3a0ddc0b Mon Sep 17 00:00:00 2001 From: osopardo1 Date: Thu, 6 Oct 2022 15:17:15 +0200 Subject: [PATCH 42/49] Add test on commitStagedChanges --- .../sources/QbeastStagedTableTest.scala | 53 ++++++++++++++++++- 1 file changed, 52 insertions(+), 1 deletion(-) diff --git a/src/test/scala/io/qbeast/spark/internal/sources/QbeastStagedTableTest.scala b/src/test/scala/io/qbeast/spark/internal/sources/QbeastStagedTableTest.scala index 0ed92ff42..9ffcdc3ef 100644 --- a/src/test/scala/io/qbeast/spark/internal/sources/QbeastStagedTableTest.scala +++ b/src/test/scala/io/qbeast/spark/internal/sources/QbeastStagedTableTest.scala @@ -6,6 +6,9 @@ import io.qbeast.spark.internal.sources.v2.QbeastStagedTableImpl import org.apache.hadoop.fs.Path import org.apache.spark.sql.connector.catalog.Identifier import org.apache.spark.sql.connector.catalog.TableCapability.V1_BATCH_WRITE +import org.apache.spark.sql.connector.write.{LogicalWriteInfo, V1Write} +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.CaseInsensitiveStringMap import scala.collection.JavaConverters._ @@ -83,8 +86,56 @@ class QbeastStagedTableTest extends QbeastIntegrationTestSpec with CatalogTestSu val path = new Path(tmpWarehouse) val fs = path.getFileSystem(spark.sessionState.newHadoopConf()) - fs.exists(path.suffix("students")) shouldBe false + fs.exists(new Path(tmpWarehouse + "/students")) shouldBe false }) + it should "commit changes" in withQbeastContextSparkAndTmpWarehouse((spark, tmpWarehouse) => { + + val qbeastCatalog = createQbeastCatalog(spark) + val qbeastStagedTable = + qbeastCatalog + .stageCreate( + Identifier.of(Array("default"), "students"), + schema, + Array.empty, + Map("provider" -> "qbeast", "columnsToIndex" -> "id").asJava) + .asInstanceOf[QbeastStagedTableImpl] + + // Prepare data to the commit + val dataToCommit = createTestData(spark) + + // We use the write builder to add the data to the commit + val writeBuilder = qbeastStagedTable + .newWriteBuilder(new LogicalWriteInfo { + override def options(): CaseInsensitiveStringMap = CaseInsensitiveStringMap.empty() + + override def queryId(): String = "1" + + override def schema(): StructType = schema + }) + .build() + + writeBuilder shouldBe a[V1Write] + writeBuilder.asInstanceOf[V1Write].toInsertableRelation.insert(dataToCommit, false) + + // Commit the staged changes + // This should create the log and write the data + qbeastStagedTable.commitStagedChanges() + + // Check if the path exists + val path = new Path(tmpWarehouse) + val fs = path.getFileSystem(spark.sessionState.newHadoopConf()) + fs.exists(new Path(tmpWarehouse + "/students")) shouldBe true + + // Check if the content of the table is correct + spark.read.table("students").count() shouldBe dataToCommit.count() + assertSmallDatasetEquality( + spark.read.table("students"), + dataToCommit, + ignoreNullable = true, + orderedComparison = false) + + }) + } From 0aa8e59ef25be28705296bd8458788dfee1eed9f Mon Sep 17 00:00:00 2001 From: osopardo1 Date: Thu, 6 Oct 2022 16:41:58 +0200 Subject: [PATCH 43/49] Fixed bug on newWriteBuilder() creation and fixed tests --- .../sources/catalog/DefaultStagedTable.scala | 3 +- .../sources/catalog/QbeastCatalogUtils.scala | 29 +++++---- .../sources/v2/QbeastStagedTableImpl.scala | 5 +- .../sources/QbeastStagedTableTest.scala | 12 +--- .../sources/catalog/CatalogTestSuite.scala | 10 ++++ .../catalog/DefaultStagedTableTest.scala | 59 +++++++++++++++---- .../QbeastCatalogIntegrationTest.scala | 14 +++++ 7 files changed, 93 insertions(+), 39 deletions(-) diff --git a/src/main/scala/io/qbeast/spark/internal/sources/catalog/DefaultStagedTable.scala b/src/main/scala/io/qbeast/spark/internal/sources/catalog/DefaultStagedTable.scala index 0690583e0..1c6861da2 100644 --- a/src/main/scala/io/qbeast/spark/internal/sources/catalog/DefaultStagedTable.scala +++ b/src/main/scala/io/qbeast/spark/internal/sources/catalog/DefaultStagedTable.scala @@ -18,6 +18,7 @@ import org.apache.spark.sql.types.StructType /** * A default StagedTable + * This case class would delegate the methods to the underlying Catalog Table * @param ident the identifier * @param table the Table * @param catalog the Catalog @@ -30,7 +31,7 @@ private[catalog] case class DefaultStagedTable( with StagedTable { override def newWriteBuilder(info: LogicalWriteInfo): WriteBuilder = { - info match { + table match { case supportsWrite: SupportsWrite => supportsWrite.newWriteBuilder(info) case _ => throw AnalysisExceptionFactory.create(s"Table `${ident.name}` does not support writes.") diff --git a/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalogUtils.scala b/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalogUtils.scala index c99731f80..bee91f995 100644 --- a/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalogUtils.scala +++ b/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalogUtils.scala @@ -25,7 +25,6 @@ import org.apache.spark.sql.{ } import java.util -import java.util.Locale import scala.collection.JavaConverters._ /** @@ -61,7 +60,7 @@ object QbeastCatalogUtils { } /** Checks if a table already exists for the provided identifier. */ - def getExistingTableIfExists( + def getTableIfExists( table: TableIdentifier, existingSessionCatalog: SessionCatalog): Option[CatalogTable] = { // If this is a path identifier, we cannot return an existing CatalogTable. The Create command @@ -84,16 +83,7 @@ object QbeastCatalogUtils { } /** - * Checks if the identifier has namespace of Qbeast - * @param ident - * @return - */ - def hasQbeastNamespace(ident: Identifier): Boolean = { - ident.namespace().length == 1 && ident.name.toLowerCase(Locale.ROOT) == QBEAST_PROVIDER_NAME - } - - /** - * Creates a Table with Qbeast + * Creates a Table on the Catalog * @param ident the Identifier of the table * @param schema the schema of the table * @param partitions the partitions of the table, if any @@ -118,6 +108,7 @@ object QbeastCatalogUtils { val isPathTable = QbeastCatalogUtils.isPathTable(ident) + // Checks if the location properties are coherent if (isPathTable && allTableProperties.containsKey("location") // The location property can be qualified and different from the path in the identifier, so @@ -128,21 +119,26 @@ object QbeastCatalogUtils { s"and ${allTableProperties.get("location")}.") } + // Get table location val location = if (isPathTable) { Option(ident.name()) } else { Option(allTableProperties.get("location")) } + + // Define the table type. + // Either can be EXTERNAL (if the location is defined) or MANAGED val tableType = if (location.isDefined) CatalogTableType.EXTERNAL else CatalogTableType.MANAGED val locUriOpt = location.map(CatalogUtils.stringToURI) val id = TableIdentifier(ident.name(), ident.namespace().lastOption) - val existingTableOpt = QbeastCatalogUtils.getExistingTableIfExists(id, existingSessionCatalog) + val existingTableOpt = QbeastCatalogUtils.getTableIfExists(id, existingSessionCatalog) val loc = locUriOpt .orElse(existingTableOpt.flatMap(_.storage.locationUri)) .getOrElse(existingSessionCatalog.defaultTablePath(id)) + // Initialize the path option val storage = DataSource .buildStorageFormatFromOptions(writeOptions) .copy(locationUri = Option(loc)) @@ -150,6 +146,7 @@ object QbeastCatalogUtils { val commentOpt = Option(allTableProperties.get("comment")) val (partitionColumns, bucketSpec) = SparkTransformUtils.convertTransforms(partitions) + // Create the CatalogTable representation for updating the Catalog val table = new CatalogTable( identifier = id, tableType = tableType, @@ -161,6 +158,7 @@ object QbeastCatalogUtils { properties = allTableProperties.asScala.toMap, comment = commentOpt) + // Write data, if any val append = tableCreationMode.saveMode == SaveMode.Append dataFrame.map { df => tableFactory @@ -168,6 +166,7 @@ object QbeastCatalogUtils { .save(df, allTableProperties.asScala.toMap, append) } + // Update the existing session catalog with the Qbeast table information updateCatalog( QTableID(loc.toString), tableCreationMode, @@ -206,6 +205,8 @@ object QbeastCatalogUtils { operation match { case _ if isPathTable => // do nothing case TableCreationMode.CREATE_TABLE => + // To create the table, check if the log exists/create a new one + // create table in the SessionCatalog checkLogCreation(tableID) existingSessionCatalog.createTable( table, @@ -213,8 +214,10 @@ object QbeastCatalogUtils { validateLocation = false) case TableCreationMode.REPLACE_TABLE | TableCreationMode.CREATE_OR_REPLACE if existingTableOpt.isDefined => + // REPLACE the metadata of the table with the new one existingSessionCatalog.alterTable(table) case TableCreationMode.REPLACE_TABLE => + // Throw an exception if the table to replace does not exists val ident = Identifier.of(table.identifier.database.toArray, table.identifier.table) throw new CannotReplaceMissingTableException(ident) case TableCreationMode.CREATE_OR_REPLACE => diff --git a/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastStagedTableImpl.scala b/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastStagedTableImpl.scala index 9db5f25be..46070ddc0 100644 --- a/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastStagedTableImpl.scala +++ b/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastStagedTableImpl.scala @@ -24,8 +24,8 @@ import java.util import scala.collection.JavaConverters._ /** + * Qbeast Implementation of StagedTable * An StagedTable allows Atomic CREATE TABLE AS SELECT / REPLACE TABLE AS SELECT - * This table should implement SupportsWrite */ private[sources] class QbeastStagedTableImpl( ident: Identifier, @@ -69,10 +69,11 @@ private[sources] class QbeastStagedTableImpl( // we pass all the writeOptions to the properties as well writeOptions.foreach { case (k, v) => props.put(k, v) } + // Check all the Qbeast properties are correctly specified checkQbeastProperties(props.asScala.toMap) // Creates the corresponding table on the Catalog and executes - // the writing of the dataFrame if any + // the writing of the dataFrame (if any) QbeastCatalogUtils.createQbeastTable( ident, schema, diff --git a/src/test/scala/io/qbeast/spark/internal/sources/QbeastStagedTableTest.scala b/src/test/scala/io/qbeast/spark/internal/sources/QbeastStagedTableTest.scala index 9ffcdc3ef..ee6ebe88d 100644 --- a/src/test/scala/io/qbeast/spark/internal/sources/QbeastStagedTableTest.scala +++ b/src/test/scala/io/qbeast/spark/internal/sources/QbeastStagedTableTest.scala @@ -6,9 +6,7 @@ import io.qbeast.spark.internal.sources.v2.QbeastStagedTableImpl import org.apache.hadoop.fs.Path import org.apache.spark.sql.connector.catalog.Identifier import org.apache.spark.sql.connector.catalog.TableCapability.V1_BATCH_WRITE -import org.apache.spark.sql.connector.write.{LogicalWriteInfo, V1Write} -import org.apache.spark.sql.types.StructType -import org.apache.spark.sql.util.CaseInsensitiveStringMap +import org.apache.spark.sql.connector.write.V1Write import scala.collection.JavaConverters._ @@ -107,13 +105,7 @@ class QbeastStagedTableTest extends QbeastIntegrationTestSpec with CatalogTestSu // We use the write builder to add the data to the commit val writeBuilder = qbeastStagedTable - .newWriteBuilder(new LogicalWriteInfo { - override def options(): CaseInsensitiveStringMap = CaseInsensitiveStringMap.empty() - - override def queryId(): String = "1" - - override def schema(): StructType = schema - }) + .newWriteBuilder(fakeLogicalWriteInfo) .build() writeBuilder shouldBe a[V1Write] diff --git a/src/test/scala/io/qbeast/spark/internal/sources/catalog/CatalogTestSuite.scala b/src/test/scala/io/qbeast/spark/internal/sources/catalog/CatalogTestSuite.scala index 2dfd5b3d5..55397a998 100644 --- a/src/test/scala/io/qbeast/spark/internal/sources/catalog/CatalogTestSuite.scala +++ b/src/test/scala/io/qbeast/spark/internal/sources/catalog/CatalogTestSuite.scala @@ -7,7 +7,9 @@ import org.apache.spark.sql.connector.catalog.{ SupportsNamespaces, TableCatalog } +import org.apache.spark.sql.connector.write.LogicalWriteInfo import org.apache.spark.sql.types.{IntegerType, StringType, StructField, StructType} +import org.apache.spark.sql.util.CaseInsensitiveStringMap import scala.collection.immutable import scala.util.Random @@ -32,6 +34,14 @@ trait CatalogTestSuite { 1.to(10).map(i => Student(i, i.toString, Random.nextInt())) } + val fakeLogicalWriteInfo = new LogicalWriteInfo { + override def options(): CaseInsensitiveStringMap = CaseInsensitiveStringMap.empty() + + override def queryId(): String = "1" + + override def schema(): StructType = schema + } + def sessionCatalog(spark: SparkSession): TableCatalog = { SparkCatalogUtils.getV2SessionCatalog(spark).asInstanceOf[TableCatalog] diff --git a/src/test/scala/io/qbeast/spark/internal/sources/catalog/DefaultStagedTableTest.scala b/src/test/scala/io/qbeast/spark/internal/sources/catalog/DefaultStagedTableTest.scala index da64ca1d1..1d40fb21c 100644 --- a/src/test/scala/io/qbeast/spark/internal/sources/catalog/DefaultStagedTableTest.scala +++ b/src/test/scala/io/qbeast/spark/internal/sources/catalog/DefaultStagedTableTest.scala @@ -1,12 +1,9 @@ package io.qbeast.spark.internal.sources.catalog import io.qbeast.spark.QbeastIntegrationTestSpec +import org.apache.spark.SparkConf import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.connector.catalog.Identifier -import org.apache.spark.sql.connector.write.LogicalWriteInfo -import org.apache.spark.sql.types.StructType -import org.apache.spark.sql.util.CaseInsensitiveStringMap - +import org.apache.spark.sql.connector.catalog.{Identifier} import scala.collection.JavaConverters._ class DefaultStagedTableTest extends QbeastIntegrationTestSpec with CatalogTestSuite { @@ -112,8 +109,8 @@ class DefaultStagedTableTest extends QbeastIntegrationTestSpec with CatalogTestS catalog.listTables(defaultNamespace) shouldBe Array() }) - it should "throw exception when creating default WriteBuilder" in - withQbeastContextSparkAndTmpWarehouse((spark, tmpWarehouse) => { + it should "commit the metadata on commit" in withQbeastContextSparkAndTmpWarehouse( + (spark, tmpWarehouse) => { val tableIdentifier = Identifier.of(Array("default"), "students") val catalog = sessionCatalog(spark) val underlyingTable = catalog.createTable( @@ -125,15 +122,51 @@ class DefaultStagedTableTest extends QbeastIntegrationTestSpec with CatalogTestS val defaultStagedTable = DefaultStagedTable .apply(tableIdentifier, underlyingTable, catalog) - val logicalWriteInfo = new LogicalWriteInfo { - override def options(): CaseInsensitiveStringMap = CaseInsensitiveStringMap.empty() + defaultStagedTable.commitStagedChanges() - override def queryId(): String = "1" + catalog.listTables(defaultNamespace) shouldBe Array(tableIdentifier) + }) - override def schema(): StructType = schema - } + it should "throw exception when creating default WriteBuilder" in + withQbeastContextSparkAndTmpWarehouse((spark, tmpWarehouse) => { + val tableIdentifier = Identifier.of(Array("default"), "students") + val catalog = sessionCatalog(spark) + val underlyingTable = catalog.createTable( + tableIdentifier, + schema, + Array.empty, + Map.empty[String, String].asJava) + + val defaultStagedTable = DefaultStagedTable + .apply(tableIdentifier, underlyingTable, catalog) an[AnalysisException] shouldBe thrownBy( - defaultStagedTable.newWriteBuilder(logicalWriteInfo)) + defaultStagedTable.newWriteBuilder(fakeLogicalWriteInfo)) }) + + it should "use the right builder when table SupportsWrites" in withTmpDir(tmpDir => + withExtendedSpark(sparkConf = new SparkConf() + .setMaster("local[8]") + .set("spark.sql.extensions", "io.qbeast.spark.internal.QbeastSparkSessionExtension") + .set("spark.sql.warehouse.dir", tmpDir) + .set("spark.sql.catalog.spark_catalog", "org.apache.spark.sql.delta.catalog.DeltaCatalog") + .set( + "spark.sql.catalog.qbeast_catalog", + "io.qbeast.spark.internal.sources.catalog.QbeastCatalog"))((spark) => { + + val tableIdentifier = Identifier.of(Array("default"), "students") + val catalog = sessionCatalog(spark) + val underlyingTable = catalog.createTable( + tableIdentifier, + schema, + Array.empty, + Map("provider" -> "delta").asJava) + + val defaultStagedTable = DefaultStagedTable + .apply(tableIdentifier, underlyingTable, catalog) + + defaultStagedTable + .newWriteBuilder(fakeLogicalWriteInfo) + .build() shouldBe a[Any] // could be any type writeBuilder + })) } diff --git a/src/test/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalogIntegrationTest.scala b/src/test/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalogIntegrationTest.scala index 01d183074..19911e369 100644 --- a/src/test/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalogIntegrationTest.scala +++ b/src/test/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalogIntegrationTest.scala @@ -109,9 +109,11 @@ class QbeastCatalogIntegrationTest extends QbeastIntegrationTestSpec with Catalo it should "throw an error when no columnsToIndex is specified" in withQbeastContextSparkAndTmpWarehouse((spark, tmpDir) => { + an[AnalysisException] shouldBe thrownBy( spark.sql("CREATE OR REPLACE TABLE student (id INT, name STRING, age INT)" + " USING qbeast")) + }) it should "throw an error when trying to replace a non-qbeast table" in @@ -136,4 +138,16 @@ class QbeastCatalogIntegrationTest extends QbeastIntegrationTestSpec with Catalo }) + it should "throw an error when adding data to a view" in + withQbeastContextSparkAndTmpWarehouse((spark, tmpDir) => { + + val data = createTestData(spark) + data.write.format("qbeast").option("columnsToIndex", "id").saveAsTable("students") + val indexed = spark.read.table("students") + indexed.createOrReplaceTempView("studentsView") + an[AnalysisException] shouldBe thrownBy( + data.write.format("qbeast").insertInto("studentsView")) + + }) + } From 92f8988ea3b14caf0ec465b4cb390570d5595f36 Mon Sep 17 00:00:00 2001 From: osopardo1 Date: Tue, 11 Oct 2022 13:01:43 +0200 Subject: [PATCH 44/49] Remove SparkTransformations and add tests for QbeastTableImpl, QbeastCatalogUtils and general QbeastCatalog behaviour --- .../internal/sources/QbeastDataSource.scala | 7 +- .../sources/catalog/QbeastCatalog.scala | 47 +++++++-- .../sources/catalog/QbeastCatalogUtils.scala | 38 +++----- .../internal/sources/v2/QbeastTableImpl.scala | 28 +++--- .../spark/sql/SparkTransformUtils.scala | 84 ---------------- .../sources/QbeastTableImplTest.scala | 96 +++++++++++++++++++ .../sources/catalog/CatalogTestSuite.scala | 8 +- .../QbeastCatalogIntegrationTest.scala | 38 ++++---- .../sources/catalog/QbeastCatalogTest.scala | 51 ++++++++++ .../utils/QbeastSQLIntegrationTest.scala | 17 +++- 10 files changed, 258 insertions(+), 156 deletions(-) delete mode 100644 src/main/scala/org/apache/spark/sql/SparkTransformUtils.scala create mode 100644 src/test/scala/io/qbeast/spark/internal/sources/QbeastTableImplTest.scala diff --git a/src/main/scala/io/qbeast/spark/internal/sources/QbeastDataSource.scala b/src/main/scala/io/qbeast/spark/internal/sources/QbeastDataSource.scala index 873b2d80d..06e9a1e32 100644 --- a/src/main/scala/io/qbeast/spark/internal/sources/QbeastDataSource.scala +++ b/src/main/scala/io/qbeast/spark/internal/sources/QbeastDataSource.scala @@ -9,6 +9,7 @@ import io.qbeast.spark.internal.QbeastOptions import io.qbeast.spark.internal.sources.v2.QbeastTableImpl import io.qbeast.spark.table.IndexedTableFactory import org.apache.hadoop.fs.{FileStatus, Path} +import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.connector.catalog._ import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.sources.{ @@ -63,15 +64,15 @@ class QbeastDataSource private[sources] (private val tableFactory: IndexedTableF "cubeSize" -> currentRevision.desiredCubeSize.toString) val tableProperties = properties.asScala.toMap ++ indexProperties new QbeastTableImpl( - tableId.id, + TableIdentifier(tableId.id), new Path(tableId.id), tableProperties, - None, + Some(schema), None, tableFactory) } else { new QbeastTableImpl( - tableId.id, + TableIdentifier(tableId.id), new Path(tableId.id), properties.asScala.toMap, Some(schema), diff --git a/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalog.scala b/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalog.scala index b4fe4a04b..c34c7cc20 100644 --- a/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalog.scala +++ b/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalog.scala @@ -5,7 +5,14 @@ package io.qbeast.spark.internal.sources.catalog import io.qbeast.context.QbeastContext import io.qbeast.spark.internal.QbeastOptions.checkQbeastProperties -import io.qbeast.spark.internal.sources.v2.QbeastStagedTableImpl +import io.qbeast.spark.internal.sources.v2.{QbeastStagedTableImpl, QbeastTableImpl} +import org.apache.hadoop.fs.Path +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.analysis.{ + NoSuchDatabaseException, + NoSuchNamespaceException, + NoSuchTableException +} import org.apache.spark.sql.{SparkCatalogUtils, SparkSession} import org.apache.spark.sql.connector.catalog._ import org.apache.spark.sql.connector.expressions.Transform @@ -26,6 +33,8 @@ class QbeastCatalog[T <: TableCatalog with SupportsNamespaces] with SupportsNamespaces with StagingTableCatalog { + private lazy val spark = SparkSession.active + private val tableFactory = QbeastContext.indexedTableFactory private var delegatedCatalog: CatalogPlugin = null @@ -44,11 +53,21 @@ class QbeastCatalog[T <: TableCatalog with SupportsNamespaces] } override def loadTable(ident: Identifier): Table = { - val superTable = getSessionCatalog().loadTable(ident) - if (QbeastCatalogUtils.isQbeastProvider(superTable.properties().asScala.get("provider"))) { - QbeastCatalogUtils.loadQbeastTable(superTable, tableFactory) - } else { - superTable + try { + getSessionCatalog().loadTable(ident) match { + case table + if QbeastCatalogUtils.isQbeastProvider(table.properties().asScala.get("provider")) => + QbeastCatalogUtils.loadQbeastTable(table, tableFactory) + case o => o + } + } catch { + case _: NoSuchDatabaseException | _: NoSuchNamespaceException | _: NoSuchTableException + if QbeastCatalogUtils.isPathTable(ident) => + new QbeastTableImpl( + TableIdentifier(ident.name(), ident.namespace().headOption), + new Path(ident.name()), + Map.empty, + tableFactory = tableFactory) } } @@ -60,9 +79,19 @@ class QbeastCatalog[T <: TableCatalog with SupportsNamespaces] if (QbeastCatalogUtils.isQbeastProvider(properties.asScala.get("provider"))) { checkQbeastProperties(properties.asScala.toMap) - QbeastCatalogUtils.loadQbeastTable( - getSessionCatalog().createTable(ident, schema, partitions, properties), - tableFactory) + // Create the table + QbeastCatalogUtils.createQbeastTable( + ident, + schema, + partitions, + properties, + Map.empty, + dataFrame = None, + TableCreationMode.CREATE_TABLE, + tableFactory, + spark.sessionState.catalog) + // Load the table + loadTable(ident) } else { getSessionCatalog().createTable(ident, schema, partitions, properties) } diff --git a/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalogUtils.scala b/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalogUtils.scala index bee91f995..738fb0f2c 100644 --- a/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalogUtils.scala +++ b/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalogUtils.scala @@ -20,7 +20,6 @@ import org.apache.spark.sql.{ DataFrame, SaveMode, SparkSession, - SparkTransformUtils, V1TableQbeast } @@ -60,7 +59,7 @@ object QbeastCatalogUtils { } /** Checks if a table already exists for the provided identifier. */ - def getTableIfExists( + def getExistingTableIfExists( table: TableIdentifier, existingSessionCatalog: SessionCatalog): Option[CatalogTable] = { // If this is a path identifier, we cannot return an existing CatalogTable. The Create command @@ -108,17 +107,6 @@ object QbeastCatalogUtils { val isPathTable = QbeastCatalogUtils.isPathTable(ident) - // Checks if the location properties are coherent - if (isPathTable - && allTableProperties.containsKey("location") - // The location property can be qualified and different from the path in the identifier, so - // we check `endsWith` here. - && Option(allTableProperties.get("location")).exists(!_.endsWith(ident.name()))) { - throw AnalysisExceptionFactory.create( - s"CREATE TABLE contains two different locations: ${ident.name()} " + - s"and ${allTableProperties.get("location")}.") - } - // Get table location val location = if (isPathTable) { Option(ident.name()) @@ -133,7 +121,7 @@ object QbeastCatalogUtils { val locUriOpt = location.map(CatalogUtils.stringToURI) val id = TableIdentifier(ident.name(), ident.namespace().lastOption) - val existingTableOpt = QbeastCatalogUtils.getTableIfExists(id, existingSessionCatalog) + val existingTableOpt = QbeastCatalogUtils.getExistingTableIfExists(id, existingSessionCatalog) val loc = locUriOpt .orElse(existingTableOpt.flatMap(_.storage.locationUri)) .getOrElse(existingSessionCatalog.defaultTablePath(id)) @@ -142,9 +130,15 @@ object QbeastCatalogUtils { val storage = DataSource .buildStorageFormatFromOptions(writeOptions) .copy(locationUri = Option(loc)) - val commentOpt = Option(allTableProperties.get("comment")) - val (partitionColumns, bucketSpec) = SparkTransformUtils.convertTransforms(partitions) + + if (partitions.nonEmpty) { + throw AnalysisExceptionFactory + .create( + "Qbeast Format does not support partitioning/bucketing. " + + "You may still want to use your partition columns as columnsToIndex " + + "to get all the benefits of data skipping. ") + } // Create the CatalogTable representation for updating the Catalog val table = new CatalogTable( @@ -153,8 +147,8 @@ object QbeastCatalogUtils { storage = storage, schema = schema, provider = Some("qbeast"), - partitionColumnNames = partitionColumns, - bucketSpec = bucketSpec, + partitionColumnNames = Seq.empty, + bucketSpec = None, properties = allTableProperties.asScala.toMap, comment = commentOpt) @@ -246,15 +240,13 @@ object QbeastCatalogUtils { val path: String = if (catalogTable.tableType == CatalogTableType.EXTERNAL) { // If it's an EXTERNAL TABLE, we can find the path through the Storage Properties catalogTable.storage.locationUri.get.toString - } else if (catalogTable.tableType == CatalogTableType.MANAGED) { + } else { // If it's a MANAGED TABLE, the location is set in the former catalogTable catalogTable.location.toString - } else { - // Otherwise, TODO - throw AnalysisExceptionFactory.create("No path found for table " + table.name()) } + new QbeastTableImpl( - catalogTable.identifier.identifier, + catalogTable.identifier, new Path(path), prop.asScala.toMap, Some(schema), diff --git a/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastTableImpl.scala b/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastTableImpl.scala index c5c72f3fb..bdf29788b 100644 --- a/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastTableImpl.scala +++ b/src/main/scala/io/qbeast/spark/internal/sources/v2/QbeastTableImpl.scala @@ -3,14 +3,14 @@ */ package io.qbeast.spark.internal.sources.v2 -import io.qbeast.context.QbeastContext._ import io.qbeast.core.model.QTableID import io.qbeast.spark.internal.sources.QbeastBaseRelation import org.apache.spark.sql.connector.catalog.TableCapability._ import io.qbeast.spark.table.IndexedTableFactory import org.apache.hadoop.fs.Path +import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog.CatalogTable -import org.apache.spark.sql.{AnalysisExceptionFactory, V2toV1Fallback} +import org.apache.spark.sql.{SparkSession, V2toV1Fallback} import org.apache.spark.sql.connector.catalog.{SupportsWrite, Table, TableCapability} import org.apache.spark.sql.connector.write.{LogicalWriteInfo, WriteBuilder} import org.apache.spark.sql.sources.BaseRelation @@ -28,9 +28,9 @@ import scala.collection.JavaConverters._ * @param tableFactory the IndexedTable Factory */ class QbeastTableImpl private[sources] ( - identifier: String, + tableIdentifier: TableIdentifier, path: Path, - options: Map[String, String], + options: Map[String, String] = Map.empty, schema: Option[StructType] = None, catalogTable: Option[CatalogTable] = None, private val tableFactory: IndexedTableFactory) @@ -44,12 +44,17 @@ class QbeastTableImpl private[sources] ( private val indexedTable = tableFactory.getIndexedTable(tableId) - override def name(): String = identifier + private lazy val table: CatalogTable = + if (catalogTable.isDefined) catalogTable.get + else { + // Get table Metadata if no catalog table is provided + SparkSession.active.sessionState.catalog + .getTableMetadata(tableIdentifier) + } - override def schema(): StructType = { - if (schema.isDefined) schema.get - else metadataManager.loadCurrentSchema(tableId) - } + override def name(): String = tableIdentifier.identifier + + override def schema(): StructType = if (schema.isDefined) schema.get else table.schema override def capabilities(): util.Set[TableCapability] = Set(ACCEPT_ANY_SCHEMA, BATCH_READ, V1_BATCH_WRITE, OVERWRITE_BY_FILTER, TRUNCATE).asJava @@ -65,9 +70,6 @@ class QbeastTableImpl private[sources] ( override def properties(): util.Map[String, String] = options.asJava - override def v1Table: CatalogTable = { - if (catalogTable.isDefined) catalogTable.get - else throw AnalysisExceptionFactory.create("No catalog table defined") - } + override def v1Table: CatalogTable = table } diff --git a/src/main/scala/org/apache/spark/sql/SparkTransformUtils.scala b/src/main/scala/org/apache/spark/sql/SparkTransformUtils.scala deleted file mode 100644 index 64f96e40b..000000000 --- a/src/main/scala/org/apache/spark/sql/SparkTransformUtils.scala +++ /dev/null @@ -1,84 +0,0 @@ -/* - * Copyright 2021 Qbeast Analytics, S.L. - */ -package org.apache.spark.sql - -import org.apache.spark.sql.catalyst.catalog.BucketSpec -import org.apache.spark.sql.connector.expressions.{ - FieldReference, - IdentityTransform, - Literal, - NamedReference, - Transform -} -import org.apache.spark.sql.types.IntegerType - -import scala.collection.mutable - -object SparkTransformUtils { - - /** - * Copy of V2 convertTransforms - * @param partitions - * @return - */ - def convertTransforms(partitions: Seq[Transform]): (Seq[String], Option[BucketSpec]) = { - val identityCols = new mutable.ArrayBuffer[String] - var bucketSpec = Option.empty[BucketSpec] - - partitions.map { - case IdentityTransform(FieldReference(Seq(col))) => - identityCols += col - - case BucketTransform(numBuckets, bucketCols, sortCols) => - bucketSpec = Some( - BucketSpec( - numBuckets, - bucketCols.map(_.fieldNames.head), - sortCols.map(_.fieldNames.head))) - - case _ => - throw AnalysisExceptionFactory.create( - "Operation not supported, non partition based transformation") - } - - (identityCols, bucketSpec) - } - -} - -object BucketTransform { - - def unapply(transform: Transform): Option[(Int, Seq[NamedReference], Seq[NamedReference])] = { - val arguments = transform.arguments() - if (transform.name() == "sorted_bucket") { - var posOfLit: Int = -1 - var numOfBucket: Int = -1 - arguments.zipWithIndex.foreach { - case (literal: Literal[_], i) if literal.dataType() == IntegerType => - numOfBucket = literal.value().asInstanceOf[Integer] - posOfLit = i - case _ => - } - Some( - ( - numOfBucket, - arguments.take(posOfLit).map(_.asInstanceOf[NamedReference]), - arguments.drop(posOfLit + 1).map(_.asInstanceOf[NamedReference]))) - } else if (transform.name() == "bucket") { - val numOfBucket = arguments(0) match { - case literal: Literal[_] if literal.dataType() == IntegerType => - literal.value().asInstanceOf[Integer] - case _ => throw new IllegalStateException("invalid bucket transform") - } - Some( - ( - numOfBucket, - arguments.drop(1).map(_.asInstanceOf[NamedReference]), - Seq.empty[FieldReference])) - } else { - None - } - } - -} diff --git a/src/test/scala/io/qbeast/spark/internal/sources/QbeastTableImplTest.scala b/src/test/scala/io/qbeast/spark/internal/sources/QbeastTableImplTest.scala new file mode 100644 index 000000000..9e8d3f452 --- /dev/null +++ b/src/test/scala/io/qbeast/spark/internal/sources/QbeastTableImplTest.scala @@ -0,0 +1,96 @@ +package io.qbeast.spark.internal.sources + +import io.qbeast.spark.QbeastIntegrationTestSpec +import io.qbeast.spark.internal.sources.catalog.CatalogTestSuite +import io.qbeast.spark.internal.sources.v2.{QbeastTableImpl, QbeastWriteBuilder} +import org.apache.hadoop.fs.Path +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.connector.catalog.Identifier +import org.apache.spark.sql.connector.catalog.TableCapability._ + +import scala.collection.JavaConverters._ + +class QbeastTableImplTest extends QbeastIntegrationTestSpec with CatalogTestSuite { + + def createQbeastTableImpl(tmpDir: String): QbeastTableImpl = { + new QbeastTableImpl( + TableIdentifier("students"), + new Path(tmpDir), + Map.empty, + Some(schema), + None, + indexedTableFactory) + + } + + "QbeastTableImpl" should "load the schema" in + withQbeastContextSparkAndTmpWarehouse((spark, tmpDir) => { + + val qbeastTableImpl = createQbeastTableImpl(tmpDir) + qbeastTableImpl.schema() shouldBe schema + + }) + + it should "get the name" in + withQbeastContextSparkAndTmpWarehouse((_, tmpDir) => { + + val qbeastTableImpl = createQbeastTableImpl(tmpDir) + qbeastTableImpl.name() shouldBe "students" + + }) + + it should "load capabilities" in + withQbeastContextSparkAndTmpWarehouse((_, tmpDir) => { + val qbeastTableImpl = createQbeastTableImpl(tmpDir) + qbeastTableImpl.capabilities() shouldBe Set( + ACCEPT_ANY_SCHEMA, + BATCH_READ, + V1_BATCH_WRITE, + OVERWRITE_BY_FILTER, + TRUNCATE).asJava + }) + + it should "create a QbeastWriteBuilder" in + withQbeastContextSparkAndTmpWarehouse((_, tmpDir) => { + val qbeastTableImpl = createQbeastTableImpl(tmpDir) + qbeastTableImpl.newWriteBuilder(fakeLogicalWriteInfo) shouldBe a[QbeastWriteBuilder] + }) + + it should "load properties" in + withQbeastContextSparkAndTmpWarehouse((_, tmpDir) => { + val properties = Map("provider" -> "qbeast", "columnsToIndex" -> "id") + val qbeastTableImpl = new QbeastTableImpl( + TableIdentifier("students"), + new Path(tmpDir), + properties, + Some(schema), + None, + indexedTableFactory) + + qbeastTableImpl.properties() shouldBe properties.asJava + + }) + + it should "load metadata from the Catalog" in + withQbeastContextSparkAndTmpWarehouse((spark, tmpDir) => { + + val qbeastCatalog = createQbeastCatalog(spark) + val identifier = Identifier.of(defaultNamespace, "students") + val tableIdentifier = TableIdentifier(identifier.name(), identifier.namespace().headOption) + val properties = Map.empty[String, String] + qbeastCatalog.createTable(identifier, schema, Array.empty, properties.asJava) + + val qbeastTableImpl = new QbeastTableImpl( + tableIdentifier, + new Path(tmpDir), + properties, + None, + None, + indexedTableFactory) + + qbeastTableImpl.schema() shouldBe schema + qbeastTableImpl.v1Table shouldBe spark.sessionState.catalog.getTableMetadata( + tableIdentifier) + }) + +} diff --git a/src/test/scala/io/qbeast/spark/internal/sources/catalog/CatalogTestSuite.scala b/src/test/scala/io/qbeast/spark/internal/sources/catalog/CatalogTestSuite.scala index 55397a998..67ee407e2 100644 --- a/src/test/scala/io/qbeast/spark/internal/sources/catalog/CatalogTestSuite.scala +++ b/src/test/scala/io/qbeast/spark/internal/sources/catalog/CatalogTestSuite.scala @@ -1,6 +1,8 @@ package io.qbeast.spark.internal.sources.catalog import io.qbeast.TestClasses.Student +import io.qbeast.context.QbeastContext +import io.qbeast.spark.table.IndexedTableFactory import org.apache.spark.sql.{DataFrame, SparkCatalogUtils, SparkSession} import org.apache.spark.sql.connector.catalog.{ StagingTableCatalog, @@ -34,14 +36,16 @@ trait CatalogTestSuite { 1.to(10).map(i => Student(i, i.toString, Random.nextInt())) } - val fakeLogicalWriteInfo = new LogicalWriteInfo { + val fakeLogicalWriteInfo: LogicalWriteInfo = new LogicalWriteInfo { override def options(): CaseInsensitiveStringMap = CaseInsensitiveStringMap.empty() override def queryId(): String = "1" - override def schema(): StructType = schema + override def schema(): StructType = CatalogTestSuite.this.schema } + lazy val indexedTableFactory: IndexedTableFactory = QbeastContext.indexedTableFactory + def sessionCatalog(spark: SparkSession): TableCatalog = { SparkCatalogUtils.getV2SessionCatalog(spark).asInstanceOf[TableCatalog] diff --git a/src/test/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalogIntegrationTest.scala b/src/test/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalogIntegrationTest.scala index 19911e369..a62a21dca 100644 --- a/src/test/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalogIntegrationTest.scala +++ b/src/test/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalogIntegrationTest.scala @@ -53,7 +53,7 @@ class QbeastCatalogIntegrationTest extends QbeastIntegrationTestSpec with Catalo }) - it should "replace table" in withQbeastContextSparkAndTmpWarehouse((spark, tmpDir) => { + it should "replace table" in withQbeastContextSparkAndTmpWarehouse((spark, _) => { // Create table first (must be in qbeast format) spark.sql( @@ -75,21 +75,20 @@ class QbeastCatalogIntegrationTest extends QbeastIntegrationTestSpec with Catalo }) - it should "create or replace table" in withQbeastContextSparkAndTmpWarehouse( - (spark, tmpDir) => { + it should "create or replace table" in withQbeastContextSparkAndTmpWarehouse((spark, _) => { - spark.sql( - "CREATE OR REPLACE TABLE student (id INT, name STRING, age INT)" + - " USING qbeast OPTIONS ('columnsToIndex'='id')") + spark.sql( + "CREATE OR REPLACE TABLE student (id INT, name STRING, age INT)" + + " USING qbeast OPTIONS ('columnsToIndex'='id')") - val table = spark.read.table("student") - table.schema shouldBe schema - table.count() shouldBe 0 + val table = spark.read.table("student") + table.schema shouldBe schema + table.count() shouldBe 0 - }) + }) it should "create table and insert data as select" in withQbeastContextSparkAndTmpWarehouse( - (spark, tmpDir) => { + (spark, _) => { spark.sql( "CREATE OR REPLACE TABLE student (id INT, name STRING, age INT)" + @@ -108,7 +107,7 @@ class QbeastCatalogIntegrationTest extends QbeastIntegrationTestSpec with Catalo }) it should "throw an error when no columnsToIndex is specified" in - withQbeastContextSparkAndTmpWarehouse((spark, tmpDir) => { + withQbeastContextSparkAndTmpWarehouse((spark, _) => { an[AnalysisException] shouldBe thrownBy( spark.sql("CREATE OR REPLACE TABLE student (id INT, name STRING, age INT)" + @@ -117,7 +116,7 @@ class QbeastCatalogIntegrationTest extends QbeastIntegrationTestSpec with Catalo }) it should "throw an error when trying to replace a non-qbeast table" in - withQbeastContextSparkAndTmpWarehouse((spark, tmpDir) => { + withQbeastContextSparkAndTmpWarehouse((spark, _) => { spark.sql( "CREATE TABLE student (id INT, name STRING, age INT)" + @@ -130,7 +129,7 @@ class QbeastCatalogIntegrationTest extends QbeastIntegrationTestSpec with Catalo }) it should "throw an error when replacing non-existing table" in - withQbeastContextSparkAndTmpWarehouse((spark, tmpDir) => { + withQbeastContextSparkAndTmpWarehouse((spark, _) => { an[AnalysisException] shouldBe thrownBy( spark.sql("REPLACE TABLE student (id INT, name STRING, age INT)" + @@ -138,15 +137,12 @@ class QbeastCatalogIntegrationTest extends QbeastIntegrationTestSpec with Catalo }) - it should "throw an error when adding data to a view" in - withQbeastContextSparkAndTmpWarehouse((spark, tmpDir) => { + it should "throw an error when using partitioning/bucketing" in + withQbeastContextSparkAndTmpWarehouse((spark, _) => { - val data = createTestData(spark) - data.write.format("qbeast").option("columnsToIndex", "id").saveAsTable("students") - val indexed = spark.read.table("students") - indexed.createOrReplaceTempView("studentsView") an[AnalysisException] shouldBe thrownBy( - data.write.format("qbeast").insertInto("studentsView")) + spark.sql("CREATE OR REPLACE TABLE student (id INT, name STRING, age INT)" + + " USING qbeast OPTIONS ('columnsToIndex'='id') PARTITIONED BY (id)")) }) diff --git a/src/test/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalogTest.scala b/src/test/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalogTest.scala index f71dbf6d5..4e7eed630 100644 --- a/src/test/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalogTest.scala +++ b/src/test/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalogTest.scala @@ -1,6 +1,10 @@ package io.qbeast.spark.internal.sources.catalog import io.qbeast.spark.QbeastIntegrationTestSpec +import io.qbeast.spark.internal.sources.v2.{QbeastStagedTableImpl, QbeastTableImpl} +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.analysis.CannotReplaceMissingTableException import org.apache.spark.sql.connector.catalog.{ CatalogExtension, CatalogPlugin, @@ -214,4 +218,51 @@ class QbeastCatalogTest extends QbeastIntegrationTestSpec with CatalogTestSuite an[IllegalArgumentException] shouldBe thrownBy( qbeastCatalog.setDelegateCatalog(fakeCatalog)) }) + + it should "create a table with PATH" in withQbeastContextSparkAndTmpWarehouse( + (spark, tmpDir) => { + + val qbeastCatalog = createQbeastCatalog(spark) + val tableIdentifierPath = Identifier.of(Array("default"), tmpDir + "/student") + + QbeastCatalogUtils.isPathTable(tableIdentifierPath) shouldBe true + + val stagedTable = qbeastCatalog.stageCreate( + tableIdentifierPath, + schema, + Array.empty, + Map("provider" -> "qbeast", "columnsToIndex" -> "id").asJava) + + stagedTable shouldBe an[QbeastStagedTableImpl] + qbeastCatalog.loadTable(tableIdentifierPath) shouldBe an[QbeastTableImpl] + + }) + + "QbeastCatalogUtils" should "throw an error when trying to replace a non-existing table" in + withQbeastContextSparkAndTmpWarehouse((spark, _) => { + an[CannotReplaceMissingTableException] shouldBe thrownBy( + QbeastCatalogUtils.createQbeastTable( + Identifier.of(defaultNamespace, "students"), + schema, + Array.empty, + Map.empty[String, String].asJava, + Map.empty, + None, + TableCreationMode.REPLACE_TABLE, + indexedTableFactory, + spark.sessionState.catalog)) + }) + + it should "throw an error when inserting data into an external view" in + withQbeastContextSparkAndTmpWarehouse((spark, _) => { + + val data = createTestData(spark) + data.write.format("qbeast").option("columnsToIndex", "id").saveAsTable("students") + spark.sql("CREATE VIEW students_view AS SELECT * from students") + + an[AnalysisException] shouldBe thrownBy( + QbeastCatalogUtils + .getExistingTableIfExists(TableIdentifier("students_view"), spark.sessionState.catalog)) + }) + } diff --git a/src/test/scala/io/qbeast/spark/utils/QbeastSQLIntegrationTest.scala b/src/test/scala/io/qbeast/spark/utils/QbeastSQLIntegrationTest.scala index c022cf746..287151acd 100644 --- a/src/test/scala/io/qbeast/spark/utils/QbeastSQLIntegrationTest.scala +++ b/src/test/scala/io/qbeast/spark/utils/QbeastSQLIntegrationTest.scala @@ -50,7 +50,7 @@ class QbeastSQLIntegrationTest extends QbeastIntegrationTestSpec { spark.sql( s"CREATE TABLE student (id INT, name STRING, age INT) USING qbeast " + - "TBLPROPERTIES ('columnsToIndex'='id')") + "OPTIONS ('columnsToIndex'='id')") spark.sql("INSERT INTO table student SELECT * FROM data") @@ -106,6 +106,21 @@ class QbeastSQLIntegrationTest extends QbeastIntegrationTestSpec { }) + it should "create EXTERNAL table" in withQbeastContextSparkAndTmpDir((spark, tmpDir) => { + spark.sql( + "CREATE EXTERNAL TABLE student (id INT, name STRING, age INT) " + + "USING qbeast " + + "OPTIONS ('columnsToIndex'='id') " + + s"LOCATION '$tmpDir'") + + val table = spark.sql("SELECT * from student") + + table.count() shouldBe 0 + + table.columns.toSet shouldBe Set("id", "name", "age") + + }) + it should "throw an error when using different path locations" in withQbeastContextSparkAndTmpWarehouse((spark, tmpDir) => { From f0dc97bf0cf4ccb7b05a18146463fac84ca96c6d Mon Sep 17 00:00:00 2001 From: osopardo1 Date: Thu, 13 Oct 2022 15:30:50 +0200 Subject: [PATCH 45/49] Add Adria as Developer in pomExtra --- build.sbt | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/build.sbt b/build.sbt index eecdec206..86ea77089 100644 --- a/build.sbt +++ b/build.sbt @@ -146,6 +146,11 @@ ThisBuild / pomExtra := Pol Santamaria https://github.com/polsm91 + + Adricu8 + Adria Correas + https://github.com/Adricu8 + // Scalafmt settings From 6501112ea97d056d43b7c0dc3184aebc00e2c9e5 Mon Sep 17 00:00:00 2001 From: osopardo1 Date: Fri, 14 Oct 2022 11:31:52 +0200 Subject: [PATCH 46/49] Add test without location --- .../utils/QbeastSparkIntegrationTest.scala | 20 +++++++++++++++++++ 1 file changed, 20 insertions(+) diff --git a/src/test/scala/io/qbeast/spark/utils/QbeastSparkIntegrationTest.scala b/src/test/scala/io/qbeast/spark/utils/QbeastSparkIntegrationTest.scala index 2cbccab13..7541857ec 100644 --- a/src/test/scala/io/qbeast/spark/utils/QbeastSparkIntegrationTest.scala +++ b/src/test/scala/io/qbeast/spark/utils/QbeastSparkIntegrationTest.scala @@ -37,6 +37,26 @@ class QbeastSparkIntegrationTest extends QbeastIntegrationTestSpec { } it should "work with SaveAsTable" in withQbeastContextSparkAndTmpWarehouse { (spark, tmpDir) => + { + + val data = createTestData(spark) + data.write + .format("qbeast") + .option("columnsToIndex", "id") + .saveAsTable("qbeast") + + val indexed = spark.read.table("qbeast") + + indexed.count() shouldBe data.count() + + indexed.columns.toSet shouldBe data.columns.toSet + + assertSmallDatasetEquality(indexed, data, orderedComparison = false, ignoreNullable = true) + + } + } + + it should "work with Location" in withQbeastContextSparkAndTmpWarehouse { (spark, tmpDir) => { val data = createTestData(spark) From 7f58c4dc3cd7e09b252395c83b96f8411e22ce22 Mon Sep 17 00:00:00 2001 From: osopardo1 Date: Tue, 25 Oct 2022 16:01:27 +0200 Subject: [PATCH 47/49] Add different method inputs for isQbeastProvider method --- .../spark/internal/sources/catalog/QbeastCatalog.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalog.scala b/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalog.scala index c34c7cc20..d2ddf4566 100644 --- a/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalog.scala +++ b/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalog.scala @@ -77,7 +77,7 @@ class QbeastCatalog[T <: TableCatalog with SupportsNamespaces] partitions: Array[Transform], properties: util.Map[String, String]): Table = { - if (QbeastCatalogUtils.isQbeastProvider(properties.asScala.get("provider"))) { + if (QbeastCatalogUtils.isQbeastProvider(properties)) { checkQbeastProperties(properties.asScala.toMap) // Create the table QbeastCatalogUtils.createQbeastTable( @@ -110,7 +110,7 @@ class QbeastCatalog[T <: TableCatalog with SupportsNamespaces] schema: StructType, partitions: Array[Transform], properties: util.Map[String, String]): StagedTable = { - if (QbeastCatalogUtils.isQbeastProvider(properties.asScala.get("provider"))) { + if (QbeastCatalogUtils.isQbeastProvider(properties)) { new QbeastStagedTableImpl( ident, schema, @@ -134,7 +134,7 @@ class QbeastCatalog[T <: TableCatalog with SupportsNamespaces] schema: StructType, partitions: Array[Transform], properties: util.Map[String, String]): StagedTable = { - if (QbeastCatalogUtils.isQbeastProvider(properties.asScala.get("provider"))) { + if (QbeastCatalogUtils.isQbeastProvider(properties)) { new QbeastStagedTableImpl( ident, schema, @@ -159,7 +159,7 @@ class QbeastCatalog[T <: TableCatalog with SupportsNamespaces] schema: StructType, partitions: Array[Transform], properties: util.Map[String, String]): StagedTable = { - if (QbeastCatalogUtils.isQbeastProvider(properties.asScala.get("provider"))) { + if (QbeastCatalogUtils.isQbeastProvider(properties)) { new QbeastStagedTableImpl( ident, schema, From 509aa02459901fc010f29329e206b7e21ea59bf9 Mon Sep 17 00:00:00 2001 From: osopardo1 Date: Tue, 25 Oct 2022 16:02:11 +0200 Subject: [PATCH 48/49] Add different method inputs for isQbeastProvider --- .../spark/internal/sources/catalog/QbeastCatalogUtils.scala | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalogUtils.scala b/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalogUtils.scala index 738fb0f2c..1e8cefe00 100644 --- a/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalogUtils.scala +++ b/src/main/scala/io/qbeast/spark/internal/sources/catalog/QbeastCatalogUtils.scala @@ -45,6 +45,12 @@ object QbeastCatalogUtils { provider.isDefined && provider.get == QBEAST_PROVIDER_NAME } + def isQbeastProvider(properties: Map[String, String]): Boolean = isQbeastProvider( + properties.get("provider")) + + def isQbeastProvider(properties: util.Map[String, String]): Boolean = isQbeastProvider( + properties.asScala.toMap) + /** * Checks if an Identifier is set with a path * @param ident the Identifier From 2b5c00bbcaecc89d649f18389103c629eb2d6499 Mon Sep 17 00:00:00 2001 From: osopardo1 Date: Tue, 25 Oct 2022 16:02:39 +0200 Subject: [PATCH 49/49] Add rule for passing properties on ReplaceAsTableStatement --- .../internal/rules/SaveAsTableRule.scala | 15 +++++++-- .../utils/QbeastSparkIntegrationTest.scala | 33 ++++++++++++++++--- 2 files changed, 40 insertions(+), 8 deletions(-) diff --git a/src/main/scala/io/qbeast/spark/internal/rules/SaveAsTableRule.scala b/src/main/scala/io/qbeast/spark/internal/rules/SaveAsTableRule.scala index 7c2fe8b03..0fbbd6531 100644 --- a/src/main/scala/io/qbeast/spark/internal/rules/SaveAsTableRule.scala +++ b/src/main/scala/io/qbeast/spark/internal/rules/SaveAsTableRule.scala @@ -3,9 +3,14 @@ */ package io.qbeast.spark.internal.rules +import io.qbeast.spark.internal.sources.catalog.QbeastCatalogUtils.isQbeastProvider import org.apache.spark.internal.Logging import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.catalyst.plans.logical.{CreateTableAsSelect, LogicalPlan} +import org.apache.spark.sql.catalyst.plans.logical.{ + CreateTableAsSelect, + LogicalPlan, + ReplaceTableAsSelect +} import org.apache.spark.sql.catalyst.rules.Rule /** @@ -19,11 +24,15 @@ class SaveAsTableRule(spark: SparkSession) extends Rule[LogicalPlan] with Loggin // We need to pass the writeOptions as properties to the creation of the table // to make sure columnsToIndex is present plan transformDown { - case saveAsSelect: CreateTableAsSelect - if saveAsSelect.properties.get("provider").contains("qbeast") => + case saveAsSelect: CreateTableAsSelect if isQbeastProvider(saveAsSelect.properties) => val options = saveAsSelect.writeOptions val finalProperties = saveAsSelect.properties ++ options saveAsSelect.copy(properties = finalProperties) + case replaceAsSelect: ReplaceTableAsSelect + if isQbeastProvider(replaceAsSelect.properties) => + val options = replaceAsSelect.writeOptions + val finalProperties = replaceAsSelect.properties ++ options + replaceAsSelect.copy(properties = finalProperties) } } diff --git a/src/test/scala/io/qbeast/spark/utils/QbeastSparkIntegrationTest.scala b/src/test/scala/io/qbeast/spark/utils/QbeastSparkIntegrationTest.scala index 7541857ec..e4454328c 100644 --- a/src/test/scala/io/qbeast/spark/utils/QbeastSparkIntegrationTest.scala +++ b/src/test/scala/io/qbeast/spark/utils/QbeastSparkIntegrationTest.scala @@ -10,7 +10,7 @@ class QbeastSparkIntegrationTest extends QbeastIntegrationTestSpec { private val students = 1.to(10).map(i => Student(i, i.toString, Random.nextInt())) - private def createTestData(spark: SparkSession): DataFrame = { + private def createStudentsTestData(spark: SparkSession): DataFrame = { import spark.implicits._ students.toDF() } @@ -18,7 +18,7 @@ class QbeastSparkIntegrationTest extends QbeastIntegrationTestSpec { "The QbeastDataSource" should "work with DataFrame API" in withQbeastContextSparkAndTmpDir { (spark, tmpDir) => { - val data = createTestData(spark) + val data = createStudentsTestData(spark) data.write.format("qbeast").option("columnsToIndex", "id").save(tmpDir) val indexed = spark.read.format("qbeast").load(tmpDir) @@ -39,7 +39,7 @@ class QbeastSparkIntegrationTest extends QbeastIntegrationTestSpec { it should "work with SaveAsTable" in withQbeastContextSparkAndTmpWarehouse { (spark, tmpDir) => { - val data = createTestData(spark) + val data = createStudentsTestData(spark) data.write .format("qbeast") .option("columnsToIndex", "id") @@ -59,7 +59,7 @@ class QbeastSparkIntegrationTest extends QbeastIntegrationTestSpec { it should "work with Location" in withQbeastContextSparkAndTmpWarehouse { (spark, tmpDir) => { - val data = createTestData(spark) + val data = createStudentsTestData(spark) val location = tmpDir + "/external" data.write .format("qbeast") @@ -81,7 +81,7 @@ class QbeastSparkIntegrationTest extends QbeastIntegrationTestSpec { it should "work with InsertInto" in withQbeastContextSparkAndTmpWarehouse { (spark, tmpDir) => { - val data = createTestData(spark) + val data = createStudentsTestData(spark) val location = tmpDir + "/external" data.write .format("qbeast") @@ -107,4 +107,27 @@ class QbeastSparkIntegrationTest extends QbeastIntegrationTestSpec { } } + it should "work with path and saveAsTable" in withQbeastContextSparkAndTmpDir( + (spark, tmpDir) => { + + val data = createStudentsTestData(spark) + data.createOrReplaceTempView("data") + spark + .sql(s"SELECT * FROM data") + .write + .option("columnsToIndex", "id,name") + .option("path", tmpDir) + .mode("overwrite") + .format("qbeast") + .saveAsTable("data_qbeast") + + val indexed = spark.read.format("qbeast").load(tmpDir) + + indexed.count() shouldBe data.count() + + indexed.columns.toSet shouldBe data.columns.toSet + + assertSmallDatasetEquality(indexed, data, orderedComparison = false, ignoreNullable = true) + }) + }