Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
57 changes: 33 additions & 24 deletions spark/src/main/scala/ai/chronon/spark/MergeJob.scala
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ import ai.chronon.api.{
Constants,
DateRange,
JoinPart,
PartitionRange,
PartitionSpec,
QueryUtils,
RelevantLeftForJoinPart,
Expand Down Expand Up @@ -47,46 +48,54 @@ class MergeJob(node: JoinMergeNode, range: DateRange, joinParts: Seq[JoinPart])(
@transient lazy val logger: Logger = LoggerFactory.getLogger(getClass)

private val join = node.join
private val leftInputTable = join.metaData.bootstrapTable
private val leftInputTable = if (join.bootstrapParts != null) {
join.metaData.bootstrapTable
} else {
JoinUtils.computeLeftSourceTableName(join)
}
// Use the node's Join's metadata for output table
private val outputTable = node.metaData.outputTable
private val dateRange = range.toPartitionRange

def run(): Unit = {
val leftDf = tableUtils.scanDf(query = null, table = leftInputTable, range = Some(dateRange))
val leftSchema = leftDf.schema
val leftDfForSchema = tableUtils.scanDf(query = null, table = leftInputTable, range = Some(dateRange))
val leftSchema = leftDfForSchema.schema
val bootstrapInfo =
BootstrapInfo.from(join, dateRange, tableUtils, Option(leftSchema), externalPartsAlreadyIncluded = true)

val rightPartsData = getRightPartsData()

val joinedDfTry =
try {
Success(
rightPartsData
.foldLeft(leftDf) { case (partialDf, (rightPart, rightDf)) =>
joinWithLeft(partialDf, rightDf, rightPart)
}
// drop all processing metadata columns
.drop(Constants.MatchedHashes, Constants.TimePartitionColumn))
} catch {
case e: Exception =>
e.printStackTrace()
Failure(e)
}
val df = processJoinedDf(joinedDfTry, leftDf, bootstrapInfo, leftDf)
df.save(outputTable, node.metaData.tableProps, autoExpand = true)
// This job benefits from a step day of 1 to avoid needing to shuffle on writing output (single partition)
dateRange.steps(days = 1).foreach { dayStep =>
val rightPartsData = getRightPartsData(dayStep)
val leftDf = tableUtils.scanDf(query = null, table = leftInputTable, range = Some(dayStep))

val joinedDfTry =
try {
Success(
rightPartsData
.foldLeft(leftDf) { case (partialDf, (rightPart, rightDf)) =>
joinWithLeft(partialDf, rightDf, rightPart)
}
// drop all processing metadata columns
.drop(Constants.MatchedHashes, Constants.TimePartitionColumn))
} catch {
case e: Exception =>
e.printStackTrace()
Failure(e)
}
Comment on lines +71 to +84
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
val joinedDfTry =
try {
Success(
rightPartsData
.foldLeft(leftDf) { case (partialDf, (rightPart, rightDf)) =>
joinWithLeft(partialDf, rightDf, rightPart)
}
// drop all processing metadata columns
.drop(Constants.MatchedHashes, Constants.TimePartitionColumn))
} catch {
case e: Exception =>
e.printStackTrace()
Failure(e)
}
val joinedDfTry =
Try {
rightPartsData
.foldLeft(leftDf) { case (partialDf, (rightPart, rightDf)) =>
joinWithLeft(partialDf, rightDf, rightPart)
}
// drop all processing metadata columns
.drop(Constants.MatchedHashes, Constants.TimePartitionColumn))
}

val df = processJoinedDf(joinedDfTry, leftDf, bootstrapInfo, leftDf)
df.save(outputTable, node.metaData.tableProps, autoExpand = true)
}
}

private def getRightPartsData(): Seq[(JoinPart, DataFrame)] = {
private def getRightPartsData(dayStep: PartitionRange): Seq[(JoinPart, DataFrame)] = {
joinParts.map { joinPart =>
// Use the RelevantLeftForJoinPart utility to get the part table name
val partTable = RelevantLeftForJoinPart.fullPartTableName(join, joinPart)
val effectiveRange =
if (join.left.dataModel != Entities && joinPart.groupBy.inferredAccuracy == Accuracy.SNAPSHOT) {
dateRange.shift(-1)
dayStep.shift(-1)
} else {
dateRange
dayStep
}
val wheres = effectiveRange.whereClauses("ds")
val sql = QueryUtils.build(null, partTable, wheres)
Expand Down
31 changes: 17 additions & 14 deletions spark/src/main/scala/ai/chronon/spark/SourceJob.scala
Original file line number Diff line number Diff line change
Expand Up @@ -42,23 +42,26 @@ class SourceJob(node: SourceWithFilterNode, range: DateRange)(implicit tableUtil
})
.getOrElse(source)

val df = tableUtils.scanDf(skewFilteredSource.query,
skewFilteredSource.table,
Some((Map(tableUtils.partitionColumn -> null) ++ timeProjection).toMap),
range = Some(dateRange))
// This job benefits from a step day of 1 to avoid needing to shuffle on writing output (single partition)
dateRange.steps(days = 1).foreach { dayStep =>
val df = tableUtils.scanDf(skewFilteredSource.query,
skewFilteredSource.table,
Some((Map(tableUtils.partitionColumn -> null) ++ timeProjection).toMap),
range = Some(dayStep))

if (df.isEmpty) {
throw new RuntimeException(s"Query produced 0 rows in range $dateRange.")
}
if (df.isEmpty) {
throw new RuntimeException(s"Query produced 0 rows in range $dayStep.")
}

val dfWithTimeCol = if (source.dataModel == Events) {
df.withTimeBasedColumn(Constants.TimePartitionColumn)
} else {
df
}
val dfWithTimeCol = if (source.dataModel == Events) {
df.withTimeBasedColumn(Constants.TimePartitionColumn)
} else {
df
}

// Save using the provided outputTable or compute one if not provided
dfWithTimeCol.save(outputTable, tableProperties = sourceWithFilter.metaData.tableProps)
// Save using the provided outputTable or compute one if not provided
dfWithTimeCol.save(outputTable, tableProperties = sourceWithFilter.metaData.tableProps)
}
}

private def formatFilterString(keys: Option[Map[String, Seq[String]]] = None): Option[String] = {
Expand Down