Skip to content

Commit 97c6ab7

Browse files
authored
Merge pull request #208 from osopardo1/v0.4.x
Revert #200 and new version
2 parents c1a5d65 + 96da313 commit 97c6ab7

25 files changed

+353
-127
lines changed

build.sbt

+1-1
Original file line numberDiff line numberDiff line change
@@ -1,7 +1,7 @@
11
import Dependencies._
22
import xerial.sbt.Sonatype._
33

4-
val mainVersion = "0.3.3"
4+
val mainVersion = "0.4.0"
55

66
lazy val qbeastCore = (project in file("core"))
77
.settings(

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

+16
Original file line numberDiff line numberDiff line change
@@ -58,6 +58,22 @@ trait MetadataManager[DataSchema, FileDescriptor] {
5858
*/
5959
def updateTable(tableID: QTableID, tableChanges: TableChanges): Unit
6060

61+
/**
62+
* This function checks if there's a conflict. A conflict happens if there
63+
* are new cubes that have been optimized but they were not announced.
64+
*
65+
* @param tableID the table ID
66+
* @param revisionID the revision ID
67+
* @param knownAnnounced the cubes we know they were announced when the write operation started.
68+
* @param oldReplicatedSet the old replicated set
69+
* @return true if there is a conflict, false otherwise
70+
*/
71+
def hasConflicts(
72+
tableID: QTableID,
73+
revisionID: RevisionID,
74+
knownAnnounced: Set[CubeId],
75+
oldReplicatedSet: ReplicatedSet): Boolean
76+
6177
/**
6278
* Checks if there's an existing log directory for the table
6379
* @param tableID the table ID

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

+13-13
Original file line numberDiff line numberDiff line change
@@ -2,24 +2,24 @@ package io.qbeast.core.model
22

33
/**
44
* Container class for Qbeast file's metadata
5-
*
6-
* @param path the file path
7-
* @param cube the cube identifier
8-
* @param revision the revision identifier
9-
* @param minWeight the minimum weight of element
10-
* @param maxWeight the maximum weight of element
11-
* @param replicated the file is replicated
12-
* @param elementCount the number of elements
13-
* @param size the size in bytes
14-
* @param modificationTime the modification timestamp
5+
* @param path
6+
* @param cube
7+
* @param revision
8+
* @param minWeight
9+
* @param maxWeight
10+
* @param state
11+
* @param elementCount
12+
* @param size
13+
* @param modificationTime
1514
*/
15+
1616
case class QbeastBlock(
1717
path: String,
1818
cube: String,
1919
revision: Long,
2020
minWeight: Weight,
2121
maxWeight: Weight,
22-
replicated: Boolean,
22+
state: String,
2323
elementCount: Long,
2424
size: Long,
2525
modificationTime: Long)
@@ -30,7 +30,7 @@ case class QbeastBlock(
3030
object QbeastBlock {
3131

3232
private val metadataKeys =
33-
Set("minWeight", "maxWeight", "replicated", "revision", "elementCount", "cube")
33+
Set("minWeight", "maxWeight", "state", "revision", "elementCount", "cube")
3434

3535
private def checkBlockMetadata(blockMetadata: Map[String, String]): Unit = {
3636
metadataKeys.foreach(key =>
@@ -60,7 +60,7 @@ object QbeastBlock {
6060
blockMetadata("revision").toLong,
6161
Weight(blockMetadata("minWeight").toInt),
6262
Weight(blockMetadata("maxWeight").toInt),
63-
blockMetadata("replicated").toBoolean,
63+
blockMetadata("state"),
6464
blockMetadata("elementCount").toLong,
6565
size,
6666
modificationTime)

core/src/test/scala/io/qbeast/core/model/JSONSerializationTests.scala

+1-6
Original file line numberDiff line numberDiff line change
@@ -1,11 +1,6 @@
11
package io.qbeast.core.model
22

3-
import io.qbeast.core.transform.{
4-
HashTransformation,
5-
LinearTransformation,
6-
Transformation,
7-
Transformer
8-
}
3+
import io.qbeast.core.transform.{HashTransformation, LinearTransformation, Transformation, Transformer}
94
import org.scalatest.flatspec.AnyFlatSpec
105
import org.scalatest.matchers.should.Matchers
116

core/src/test/scala/io/qbeast/core/model/QbeastBlockTest.scala

+3-3
Original file line numberDiff line numberDiff line change
@@ -9,15 +9,15 @@ class QbeastBlockTest extends AnyFlatSpec with Matchers {
99
"minWeight" -> "19217",
1010
"cube" -> "",
1111
"maxWeight" -> "11111111",
12-
"replicated" -> "true",
12+
"state" -> "FlOODED",
1313
"revision" -> "1",
1414
"elementCount" -> "777")
1515

1616
val qbeastBlock = QbeastBlock("path", blockMetadata, 0L, 0L)
1717
qbeastBlock.cube shouldBe ""
1818
qbeastBlock.minWeight shouldBe Weight(19217)
1919
qbeastBlock.maxWeight shouldBe Weight(11111111)
20-
qbeastBlock.replicated shouldBe true
20+
qbeastBlock.state shouldBe "FlOODED"
2121
qbeastBlock.revision shouldBe 1
2222
qbeastBlock.elementCount shouldBe 777
2323
}
@@ -32,7 +32,7 @@ class QbeastBlockTest extends AnyFlatSpec with Matchers {
3232
"minWeight" -> "19217",
3333
"cube" -> "",
3434
"maxWeight" -> "11111111",
35-
"replicated" -> "false",
35+
"state" -> "FlOODED",
3636
"revision" -> "bad_type",
3737
"elementCount" -> "777")
3838

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

+2-2
Original file line numberDiff line numberDiff line change
@@ -4,7 +4,7 @@
44
package io.qbeast.spark.delta
55

66
import io.qbeast.core.model.{CubeId, QTableID, Revision}
7-
import io.qbeast.spark.utils.TagColumns
7+
import io.qbeast.spark.utils.{State, TagColumns}
88
import org.apache.hadoop.fs.Path
99
import org.apache.spark.sql.delta.DeltaLog
1010
import org.apache.spark.sql.{DataFrame, SparkSession}
@@ -64,7 +64,7 @@ case class CubeDataLoader(tableID: QTableID) {
6464
.where(
6565
TagColumns.revision === lit(revision.revisionID.toString) &&
6666
TagColumns.cube === lit(cube.string) &&
67-
TagColumns.replicated === lit(false.toString()))
67+
TagColumns.state != lit(State.ANNOUNCED))
6868
.collect()
6969

7070
val fileNames = cubeBlocks.map(f => new Path(tableID.id, f.path).toString)

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

+30-1
Original file line numberDiff line numberDiff line change
@@ -44,6 +44,24 @@ case class DeltaQbeastSnapshot(protected override val snapshot: Snapshot)
4444
}
4545
}
4646

47+
/**
48+
* Constructs replicated set for each revision
49+
*
50+
* @return a map of revision identifier and replicated set
51+
*/
52+
private val replicatedSetsMap: Map[RevisionID, ReplicatedSet] = {
53+
val listReplicatedSets = metadataMap.filterKeys(_.startsWith(MetadataConfig.replicatedSet))
54+
55+
listReplicatedSets.map { case (key: String, json: String) =>
56+
val revisionID = key.split('.').last.toLong
57+
val revision = getRevision(revisionID)
58+
val replicatedSet = mapper
59+
.readValue[Set[String]](json, classOf[Set[String]])
60+
.map(revision.createCubeId)
61+
(revisionID, replicatedSet)
62+
}
63+
}
64+
4765
/**
4866
* Returns last available revision identifier
4967
*
@@ -65,6 +83,16 @@ case class DeltaQbeastSnapshot(protected override val snapshot: Snapshot)
6583
throw AnalysisExceptionFactory.create(s"No space revision available with $revisionID"))
6684
}
6785

86+
/**
87+
* Returns the replicated set for a revision identifier if exists
88+
* @param revisionID the revision identifier
89+
* @return the replicated set
90+
*/
91+
private def getReplicatedSet(revisionID: RevisionID): ReplicatedSet = {
92+
replicatedSetsMap
93+
.getOrElse(revisionID, Set.empty)
94+
}
95+
6896
/**
6997
* Returns true if a revision with a specific revision identifier exists
7098
*
@@ -92,7 +120,8 @@ case class DeltaQbeastSnapshot(protected override val snapshot: Snapshot)
92120
*/
93121
override def loadIndexStatus(revisionID: RevisionID): IndexStatus = {
94122
val revision = getRevision(revisionID)
95-
new IndexStatusBuilder(this, revision).build()
123+
val replicatedSet = getReplicatedSet(revisionID)
124+
new IndexStatusBuilder(this, revision, replicatedSet).build()
96125
}
97126

98127
/**

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

+13-38
Original file line numberDiff line numberDiff line change
@@ -5,15 +5,11 @@ package io.qbeast.spark.delta
55

66
import io.qbeast.core.model._
77
import io.qbeast.spark.delta.QbeastMetadataSQL._
8+
import io.qbeast.spark.utils.State.FLOODED
89
import io.qbeast.spark.utils.TagColumns
9-
import org.apache.spark.sql.Dataset
10-
import org.apache.spark.sql.SparkSession
1110
import org.apache.spark.sql.delta.actions.AddFile
12-
import org.apache.spark.sql.functions.col
13-
import org.apache.spark.sql.functions.collect_list
14-
import org.apache.spark.sql.functions.lit
15-
import org.apache.spark.sql.functions.min
16-
import org.apache.spark.sql.functions.sum
11+
import org.apache.spark.sql.functions.{col, collect_list, lit, min, sum}
12+
import org.apache.spark.sql.{Dataset, SparkSession}
1713

1814
import scala.collection.immutable.SortedMap
1915

@@ -24,7 +20,11 @@ import scala.collection.immutable.SortedMap
2420
* @param announcedSet the announced set available for the revision
2521
* @param replicatedSet the replicated set available for the revision
2622
*/
27-
private[delta] class IndexStatusBuilder(qbeastSnapshot: DeltaQbeastSnapshot, revision: Revision)
23+
private[delta] class IndexStatusBuilder(
24+
qbeastSnapshot: DeltaQbeastSnapshot,
25+
revision: Revision,
26+
replicatedSet: ReplicatedSet,
27+
announcedSet: Set[CubeId] = Set.empty)
2828
extends Serializable
2929
with StagingUtils {
3030

@@ -37,19 +37,15 @@ private[delta] class IndexStatusBuilder(qbeastSnapshot: DeltaQbeastSnapshot, rev
3737
qbeastSnapshot.loadRevisionBlocks(revision.revisionID)
3838

3939
def build(): IndexStatus = {
40-
val cubeStatuses =
40+
val cubeStatus =
4141
if (isStaging(revision)) stagingCubeStatuses
4242
else buildCubesStatuses
4343

44-
val (replicatedSet, announcedSet): (Set[CubeId], Set[CubeId]) =
45-
if (isStaging(revision)) (Set.empty, Set.empty)
46-
else buildReplicatedAndAnnouncedSets(cubeStatuses)
47-
4844
IndexStatus(
4945
revision = revision,
5046
replicatedSet = replicatedSet,
5147
announcedSet = announcedSet,
52-
cubesStatuses = cubeStatuses)
48+
cubesStatuses = cubeStatus)
5349
}
5450

5551
def stagingCubeStatuses: SortedMap[CubeId, CubeStatus] = {
@@ -64,7 +60,7 @@ private[delta] class IndexStatusBuilder(qbeastSnapshot: DeltaQbeastSnapshot, rev
6460
revision.revisionID,
6561
Weight.MinValue,
6662
maxWeight,
67-
false,
63+
FLOODED,
6864
0,
6965
addFile.size,
7066
addFile.modificationTime))
@@ -95,34 +91,13 @@ private[delta] class IndexStatusBuilder(qbeastSnapshot: DeltaQbeastSnapshot, rev
9591
.select(
9692
createCube(col("cube"), lit(ndims)).as("cubeId"),
9793
col("maxWeight"),
98-
normalizeWeight(col("maxWeight"), col("elementCount"), lit(rev.desiredCubeSize))
99-
.as("normalizedWeight"),
94+
normalizeWeight(col("maxWeight"), col("elementCount"), lit(rev.desiredCubeSize)).as(
95+
"normalizedWeight"),
10096
col("files"))
10197
.as[CubeStatus]
10298
.collect()
10399
.foreach(row => builder += row.cubeId -> row)
104100
builder.result()
105101
}
106102

107-
def buildReplicatedAndAnnouncedSets(
108-
cubeStatuses: Map[CubeId, CubeStatus]): (Set[CubeId], Set[CubeId]) = {
109-
val replicatedSet = Set.newBuilder[CubeId]
110-
val announcedSet = Set.newBuilder[CubeId]
111-
cubeStatuses.foreach { case (id, status) =>
112-
var hasReplicated = false
113-
var hasUnreplicated = false
114-
status.files.foreach(file =>
115-
if (file.replicated) hasReplicated = true
116-
else hasUnreplicated = true)
117-
if (hasReplicated) {
118-
if (hasUnreplicated) {
119-
announcedSet += id
120-
} else {
121-
replicatedSet += id
122-
}
123-
}
124-
}
125-
(replicatedSet.result(), announcedSet.result())
126-
}
127-
128103
}

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

+35-2
Original file line numberDiff line numberDiff line change
@@ -3,7 +3,7 @@
33
*/
44
package io.qbeast.spark.delta
55

6-
import io.qbeast.core.model.{Revision, StagingUtils, TableChanges, mapper}
6+
import io.qbeast.core.model.{ReplicatedSet, Revision, StagingUtils, TableChanges, mapper}
77
import io.qbeast.spark.utils.MetadataConfig
88
import io.qbeast.spark.utils.MetadataConfig.{lastRevisionID, revision}
99
import org.apache.spark.sql.SparkSession
@@ -36,10 +36,38 @@ private[delta] class QbeastMetadataOperation extends ImplicitMetadataOperation w
3636
}
3737
}
3838

39+
/**
40+
* Updates Delta Metadata Configuration with new replicated set
41+
* for given revision
42+
* @param baseConfiguration Delta Metadata Configuration
43+
* @param revision the new revision to persist
44+
* @param deltaReplicatedSet the new set of replicated cubes
45+
*/
46+
47+
private def updateQbeastReplicatedSet(
48+
baseConfiguration: Configuration,
49+
revision: Revision,
50+
deltaReplicatedSet: ReplicatedSet): Configuration = {
51+
52+
val revisionID = revision.revisionID
53+
assert(baseConfiguration.contains(s"${MetadataConfig.revision}.$revisionID"))
54+
55+
val newReplicatedSet = deltaReplicatedSet.map(_.string)
56+
// Save the replicated set of cube id's as String representation
57+
58+
baseConfiguration.updated(
59+
s"${MetadataConfig.replicatedSet}.$revisionID",
60+
mapper.writeValueAsString(newReplicatedSet))
61+
62+
}
63+
3964
private def overwriteQbeastConfiguration(baseConfiguration: Configuration): Configuration = {
4065
val revisionKeys = baseConfiguration.keys.filter(_.startsWith(MetadataConfig.revision))
66+
val replicatedSetKeys = {
67+
baseConfiguration.keys.filter(_.startsWith(MetadataConfig.replicatedSet))
68+
}
4169
val other = baseConfiguration.keys.filter(_ == MetadataConfig.lastRevisionID)
42-
val qbeastKeys = revisionKeys ++ other
70+
val qbeastKeys = revisionKeys ++ replicatedSetKeys ++ other
4371
baseConfiguration -- qbeastKeys
4472
}
4573

@@ -122,6 +150,11 @@ private[delta] class QbeastMetadataOperation extends ImplicitMetadataOperation w
122150
val configuration =
123151
if (isNewRevision || isOverwriteMode) {
124152
updateQbeastRevision(baseConfiguration, latestRevision)
153+
} else if (isOptimizeOperation) {
154+
updateQbeastReplicatedSet(
155+
baseConfiguration,
156+
latestRevision,
157+
tableChanges.announcedOrReplicatedSet)
125158
} else baseConfiguration
126159

127160
if (txn.readVersion == -1) {

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

+3-2
Original file line numberDiff line numberDiff line change
@@ -30,8 +30,9 @@ object QbeastMetadataSQL {
3030
col("size"),
3131
col("modificationTime"),
3232
weight(TagColumns.minWeight).as("minWeight"),
33-
weight(TagColumns.maxWeight).as("maxWeight"),
34-
TagColumns.replicated.cast("boolean").as("replicated"),
33+
weight(TagColumns.maxWeight)
34+
.as("maxWeight"),
35+
TagColumns.state,
3536
TagColumns.revision.cast("bigint").as("revision"),
3637
TagColumns.elementCount.cast("bigint").as("elementCount"))
3738

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

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

6-
import io.qbeast.spark.utils.TagUtils
6+
import io.qbeast.spark.utils.{State, TagUtils}
77
import org.apache.spark.sql.delta.actions.AddFile
88

99
/**
@@ -12,8 +12,7 @@ import org.apache.spark.sql.delta.actions.AddFile
1212
object ReplicatedFile {
1313

1414
def apply(addFile: AddFile): AddFile = {
15-
val newTags = addFile.tags
16-
.updated(TagUtils.replicated, true.toString())
15+
val newTags = addFile.tags.updated(TagUtils.state, State.REPLICATED)
1716
addFile.copy(tags = newTags, modificationTime = System.currentTimeMillis())
1817
}
1918

0 commit comments

Comments
 (0)