Skip to content

Commit d503833

Browse files
authored
Merge pull request #124 from osopardo1/42-table-writes-catalog
Add Table properties to Qbeast
2 parents 014aa0e + 2b5c00b commit d503833

39 files changed

+2721
-243
lines changed

README.md

+19-15
Original file line numberDiff line numberDiff line change
@@ -92,6 +92,7 @@ export SPARK_HOME=$PWD/spark-3.1.1-bin-hadoop3.2
9292
```bash
9393
$SPARK_HOME/bin/spark-shell \
9494
--conf spark.sql.extensions=io.qbeast.spark.internal.QbeastSparkSessionExtension \
95+
--conf spark.sql.catalog.spark_catalog=io.qbeast.spark.internal.sources.catalog.QbeastCatalog \
9596
--packages io.qbeast:qbeast-spark_2.12:0.2.0,io.delta:delta-core_2.12:1.0.0
9697
```
9798

@@ -118,6 +119,24 @@ csv_df.write
118119
.save(tmp_dir)
119120
```
120121

122+
#### SQL Syntax.
123+
You can create a table with Qbeast with the help of `QbeastCatalog`.
124+
125+
```scala
126+
spark.sql(
127+
"CREATE TABLE student (id INT, name STRING, age INT) " +
128+
"USING qbeast OPTIONS ('columnsToIndex'='id')")
129+
130+
```
131+
132+
Use **`INSERT INTO`** to add records to the new table. It will update the index in a **dynamic** fashion when new data is inserted.
133+
134+
```scala
135+
136+
spark.sql("INSERT INTO table student SELECT * FROM visitor_students")
137+
138+
```
139+
121140
### 3. Load the dataset
122141
Load the newly indexed dataset.
123142

@@ -151,21 +170,6 @@ qbeastTable.getIndexMetrics()
151170
qbeastTable.analyze()
152171
```
153172

154-
The format supports **Spark SQL** syntax.
155-
It also updates the index in a **dynamic** fashion when new data is inserted.
156-
157-
```scala
158-
val newData = Seq(1, 2, 3, 4).toDF("value")
159-
160-
newData.createOrReplaceTempView("newTable")
161-
162-
spark.sql("insert into table myTable select * from newTable")
163-
164-
spark.sql("insert into table myTable (value) values (4)")
165-
166-
167-
```
168-
169173
Go to [QbeastTable documentation](./docs/QbeastTable.md) for more detailed information.
170174

171175
# Dependencies and Version Compatibility

build.sbt

+5
Original file line numberDiff line numberDiff line change
@@ -146,6 +146,11 @@ ThisBuild / pomExtra :=
146146
<name>Pol Santamaria</name>
147147
<url>https://github.com/polsm91</url>
148148
</developer>
149+
<developer>
150+
<id>Adricu8</id>
151+
<name>Adria Correas</name>
152+
<url>https://github.com/Adricu8</url>
153+
</developer>
149154
</developers>
150155

151156
// Scalafmt settings

core/src/main/scala/io/qbeast/core/model/MetadataManager.scala

+13
Original file line numberDiff line numberDiff line change
@@ -63,4 +63,17 @@ trait MetadataManager[DataSchema, FileDescriptor] {
6363
knownAnnounced: Set[CubeId],
6464
oldReplicatedSet: ReplicatedSet): Boolean
6565

66+
/**
67+
* Checks if there's an existing log directory for the table
68+
* @param tableID the table ID
69+
* @return
70+
*/
71+
def existsLog(tableID: QTableID): Boolean
72+
73+
/**
74+
* Creates an initial log directory
75+
* @param tableID
76+
*/
77+
def createLog(tableID: QTableID): Unit
78+
6679
}

src/main/scala/io/qbeast/spark/delta/SparkDeltaMetadataManager.scala

+19
Original file line numberDiff line numberDiff line change
@@ -62,4 +62,23 @@ object SparkDeltaMetadataManager extends MetadataManager[StructType, FileAction]
6262
diff.nonEmpty
6363
}
6464

65+
/**
66+
* Checks if there's an existing log directory for the table
67+
*
68+
* @param tableID the table ID
69+
* @return
70+
*/
71+
override def existsLog(tableID: QTableID): Boolean = {
72+
loadDeltaQbeastLog(tableID).deltaLog.tableExists
73+
}
74+
75+
/**
76+
* Creates an initial log directory
77+
*
78+
* @param tableID
79+
*/
80+
override def createLog(tableID: QTableID): Unit = {
81+
loadDeltaQbeastLog(tableID).deltaLog.createLogDirectory()
82+
}
83+
6584
}

src/main/scala/io/qbeast/spark/internal/QbeastOptions.scala

+16
Original file line numberDiff line numberDiff line change
@@ -3,6 +3,7 @@
33
*/
44
package io.qbeast.spark.internal
55

6+
import io.qbeast.core.model.QTableID
67
import io.qbeast.spark.index.ColumnsToIndex
78
import org.apache.spark.qbeast.config.DEFAULT_CUBE_SIZE
89
import org.apache.spark.sql.AnalysisExceptionFactory
@@ -20,6 +21,7 @@ case class QbeastOptions(columnsToIndex: Seq[String], cubeSize: Int)
2021
object QbeastOptions {
2122
val COLUMNS_TO_INDEX = "columnsToIndex"
2223
val CUBE_SIZE = "cubeSize"
24+
val PATH = "path"
2325

2426
private def getColumnsToIndex(options: Map[String, String]): Seq[String] = {
2527
val encodedColumnsToIndex = options.getOrElse(
@@ -49,4 +51,18 @@ object QbeastOptions {
4951
QbeastOptions(columnsToIndex, desiredCubeSize)
5052
}
5153

54+
def loadTableIDFromParameters(parameters: Map[String, String]): QTableID = {
55+
new QTableID(
56+
parameters.getOrElse(
57+
PATH, {
58+
throw AnalysisExceptionFactory.create("'path' is not specified")
59+
}))
60+
}
61+
62+
def checkQbeastProperties(parameters: Map[String, String]): Unit = {
63+
require(
64+
parameters.contains("columnsToIndex") || parameters.contains("columnstoindex"),
65+
throw AnalysisExceptionFactory.create("'columnsToIndex is not specified"))
66+
}
67+
5268
}

src/main/scala/io/qbeast/spark/internal/QbeastSparkSessionExtension.scala

+9-1
Original file line numberDiff line numberDiff line change
@@ -4,7 +4,7 @@
44
package io.qbeast.spark.internal
55

66
import io.delta.sql.DeltaSparkSessionExtension
7-
import io.qbeast.spark.internal.rules.{SampleRule}
7+
import io.qbeast.spark.internal.rules.{QbeastAnalysis, SampleRule, SaveAsTableRule}
88
import org.apache.spark.sql.SparkSessionExtensions
99

1010
/**
@@ -16,9 +16,17 @@ class QbeastSparkSessionExtension extends DeltaSparkSessionExtension {
1616

1717
super.apply(extensions)
1818

19+
extensions.injectResolutionRule { session =>
20+
new QbeastAnalysis(session)
21+
}
22+
1923
extensions.injectOptimizerRule { session =>
2024
new SampleRule(session)
2125
}
26+
27+
extensions.injectOptimizerRule { session =>
28+
new SaveAsTableRule(session)
29+
}
2230
}
2331

2432
}
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,41 @@
1+
/*
2+
* Copyright 2021 Qbeast Analytics, S.L.
3+
*/
4+
package io.qbeast.spark.internal.rules
5+
6+
import io.qbeast.spark.internal.sources.v2.QbeastTableImpl
7+
import org.apache.spark.sql.SparkSession
8+
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
9+
import org.apache.spark.sql.catalyst.rules.Rule
10+
import org.apache.spark.sql.execution.datasources.LogicalRelation
11+
import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
12+
13+
/**
14+
* Analyzes and resolves the Spark Plan before Optimization
15+
* @param spark the SparkSession
16+
*/
17+
class QbeastAnalysis(spark: SparkSession) extends Rule[LogicalPlan] {
18+
19+
/**
20+
* Returns the V1Relation from a V2Relation
21+
* @param dataSourceV2Relation the V2Relation
22+
* @param table the underlying table
23+
* @return the LogicalRelation
24+
*/
25+
private def toV1Relation(
26+
dataSourceV2Relation: DataSourceV2Relation,
27+
table: QbeastTableImpl): LogicalRelation = {
28+
29+
val underlyingRelation = table.toBaseRelation
30+
LogicalRelation(underlyingRelation, dataSourceV2Relation.output, None, isStreaming = false)
31+
32+
}
33+
34+
override def apply(plan: LogicalPlan): LogicalPlan = plan transformDown {
35+
// This rule is a hack to return a V1 relation for reading
36+
// Because we didn't implemented SupportsRead on QbeastTableImpl yet
37+
case v2Relation @ DataSourceV2Relation(t: QbeastTableImpl, _, _, _, _) =>
38+
toV1Relation(v2Relation, t)
39+
}
40+
41+
}
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,39 @@
1+
/*
2+
* Copyright 2021 Qbeast Analytics, S.L.
3+
*/
4+
package io.qbeast.spark.internal.rules
5+
6+
import io.qbeast.spark.internal.sources.catalog.QbeastCatalogUtils.isQbeastProvider
7+
import org.apache.spark.internal.Logging
8+
import org.apache.spark.sql.SparkSession
9+
import org.apache.spark.sql.catalyst.plans.logical.{
10+
CreateTableAsSelect,
11+
LogicalPlan,
12+
ReplaceTableAsSelect
13+
}
14+
import org.apache.spark.sql.catalyst.rules.Rule
15+
16+
/**
17+
* Rule class that enforces to pass all the write options to the Table Implementation
18+
* @param spark the SparkSession
19+
*/
20+
class SaveAsTableRule(spark: SparkSession) extends Rule[LogicalPlan] with Logging {
21+
22+
override def apply(plan: LogicalPlan): LogicalPlan = {
23+
// When CreateTableAsSelect statement is in place for qbeast
24+
// We need to pass the writeOptions as properties to the creation of the table
25+
// to make sure columnsToIndex is present
26+
plan transformDown {
27+
case saveAsSelect: CreateTableAsSelect if isQbeastProvider(saveAsSelect.properties) =>
28+
val options = saveAsSelect.writeOptions
29+
val finalProperties = saveAsSelect.properties ++ options
30+
saveAsSelect.copy(properties = finalProperties)
31+
case replaceAsSelect: ReplaceTableAsSelect
32+
if isQbeastProvider(replaceAsSelect.properties) =>
33+
val options = replaceAsSelect.writeOptions
34+
val finalProperties = replaceAsSelect.properties ++ options
35+
replaceAsSelect.copy(properties = finalProperties)
36+
}
37+
}
38+
39+
}

src/main/scala/io/qbeast/spark/internal/sources/QbeastBaseRelation.scala

+51-35
Original file line numberDiff line numberDiff line change
@@ -3,33 +3,25 @@
33
*/
44
package io.qbeast.spark.internal.sources
55

6-
import io.qbeast.core.model.{QTableID}
76
import org.apache.spark.sql.sources.BaseRelation
87
import org.apache.spark.sql.sources.InsertableRelation
9-
10-
import org.apache.spark.sql.{SQLContext}
11-
import org.apache.spark.sql.types.{StructType, StructField}
8+
import org.apache.spark.sql.SQLContext
9+
import org.apache.spark.sql.types.{StructField, StructType}
1210
import org.apache.spark.sql.DataFrame
1311
import org.apache.spark.sql.SparkSession
1412
import io.qbeast.spark.delta.OTreeIndex
1513
import org.apache.spark.sql.execution.datasources.HadoopFsRelation
1614
import io.qbeast.spark.table.IndexedTable
1715
import io.qbeast.context.QbeastContext
18-
import org.apache.hadoop.fs.{Path}
19-
import org.apache.spark.sql.catalyst.catalog.{BucketSpec}
16+
import org.apache.hadoop.fs.Path
17+
import org.apache.spark.sql.catalyst.catalog.BucketSpec
2018
import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat
2119

2220
/**
2321
* Companion object for QbeastBaseRelation
2422
*/
2523
object QbeastBaseRelation {
2624

27-
/**
28-
* Creates a QbeastBaseRelation instance
29-
* @param tableID the identifier of the table
30-
* @return the QbeastBaseRelation
31-
*/
32-
3325
/**
3426
* Returns a HadoopFsRelation that contains all of the data present
3527
* in the table. This relation will be continually updated
@@ -39,48 +31,72 @@ object QbeastBaseRelation {
3931
* @param sqlContext the SQLContext
4032
* @return the HadoopFsRelation
4133
*/
42-
def createRelation(sqlContext: SQLContext, table: IndexedTable): BaseRelation = {
34+
def createRelation(
35+
sqlContext: SQLContext,
36+
table: IndexedTable,
37+
options: Map[String, String]): BaseRelation = {
4338

4439
val spark = SparkSession.active
4540
val tableID = table.tableID
4641
val snapshot = QbeastContext.metadataManager.loadSnapshot(tableID)
4742
val schema = QbeastContext.metadataManager.loadCurrentSchema(tableID)
48-
val revision = snapshot.loadLatestRevision
49-
val columnsToIndex = revision.columnTransformers.map(row => row.columnName).mkString(",")
50-
val cubeSize = revision.desiredCubeSize
51-
val parameters =
52-
Map[String, String]("columnsToIndex" -> columnsToIndex, "cubeSize" -> cubeSize.toString())
43+
if (snapshot.isInitial) {
44+
// If the Table is initial, read empty relation
45+
// This could happen if we CREATE/REPLACE TABLE without inserting data
46+
// In this case, we use the options variable
47+
new HadoopFsRelation(
48+
OTreeIndex(spark, new Path(tableID.id)),
49+
partitionSchema = StructType(Seq.empty[StructField]),
50+
dataSchema = schema,
51+
bucketSpec = None,
52+
new ParquetFileFormat(),
53+
options)(spark) with InsertableRelation {
54+
def insert(data: DataFrame, overwrite: Boolean): Unit = {
55+
table.save(data, options, append = !overwrite)
56+
}
57+
}
58+
} else {
59+
// If the table contains data, initialize it
60+
val revision = snapshot.loadLatestRevision
61+
val columnsToIndex = revision.columnTransformers.map(row => row.columnName).mkString(",")
62+
val cubeSize = revision.desiredCubeSize
63+
val parameters =
64+
Map[String, String]("columnsToIndex" -> columnsToIndex, "cubeSize" -> cubeSize.toString())
5365

54-
val path = new Path(tableID.id)
55-
val fileIndex = OTreeIndex(spark, path)
56-
val bucketSpec: Option[BucketSpec] = None
57-
val file = new ParquetFileFormat()
66+
val path = new Path(tableID.id)
67+
val fileIndex = OTreeIndex(spark, path)
68+
val bucketSpec: Option[BucketSpec] = None
69+
val file = new ParquetFileFormat()
5870

59-
new HadoopFsRelation(
60-
fileIndex,
61-
partitionSchema = StructType(Seq.empty[StructField]),
62-
dataSchema = schema,
63-
bucketSpec = bucketSpec,
64-
file,
65-
parameters)(spark) with InsertableRelation {
66-
def insert(data: DataFrame, overwrite: Boolean): Unit = {
67-
table.save(data, parameters, append = !overwrite)
71+
new HadoopFsRelation(
72+
fileIndex,
73+
partitionSchema = StructType(Seq.empty[StructField]),
74+
dataSchema = schema,
75+
bucketSpec = bucketSpec,
76+
file,
77+
parameters)(spark) with InsertableRelation {
78+
def insert(data: DataFrame, overwrite: Boolean): Unit = {
79+
table.save(data, parameters, append = !overwrite)
80+
}
6881
}
6982
}
7083
}
7184

7285
/**
7386
* Function that can be called from a QbeastBaseRelation object to create a
7487
* new QbeastBaseRelation with a new tableID.
75-
* @param tableID the identifier of the table
7688
* @param indexedTable the indexed table
7789
* @return BaseRelation for the new table in Qbeast format
7890
*/
79-
def forQbeastTable(tableID: QTableID, indexedTable: IndexedTable): BaseRelation = {
91+
def forQbeastTable(indexedTable: IndexedTable): BaseRelation = {
92+
forQbeastTableWithOptions(indexedTable, Map.empty)
93+
}
8094

95+
def forQbeastTableWithOptions(
96+
indexedTable: IndexedTable,
97+
withOptions: Map[String, String]): BaseRelation = {
8198
val spark = SparkSession.active
82-
createRelation(spark.sqlContext, indexedTable)
83-
99+
createRelation(spark.sqlContext, indexedTable, withOptions)
84100
}
85101

86102
}

0 commit comments

Comments
 (0)