Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
66 commits
Select commit Hold shift + click to select a range
eaacfa6
code to write daily irs
kambstreat May 30, 2025
40b6cb2
store incremental agg and compute final IRs
kambstreat Jun 3, 2025
a014b6e
Store hops to inc tables
kambstreat Jun 7, 2025
32d559e
add code changes to generate final output from IR for AVG
kambstreat Jun 14, 2025
37293df
change function structure and variable names
kambstreat Jun 19, 2025
6263706
remove unused functions
kambstreat Jun 19, 2025
cb4325b
change function defs
kambstreat Jun 19, 2025
796ef96
make changes
kambstreat Jun 19, 2025
f218b23
change function order
kambstreat Jun 19, 2025
b1d4ee9
add new field is_incremental to python api
kambstreat Jun 20, 2025
2ab7659
get argument for isIncremental in scala spark backend
kambstreat Jun 20, 2025
238c781
add unit test for incremental groupby
kambstreat Jun 20, 2025
8edfd27
reuse table ccreation
kambstreat Jul 18, 2025
e903683
Update GroupByTest
kambstreat Jul 18, 2025
0bdc4fc
Add GroupByTest for events
kambstreat Jul 18, 2025
7987931
changes for incrementalg
kambstreat Sep 3, 2025
2b26d45
resolve merge conflicts
kambstreat Sep 3, 2025
7b62a43
add last hole logic for incrementnal bacckfill
kambstreat Sep 5, 2025
aeeb5ec
fix syntax
kambstreat Sep 5, 2025
9180d23
fix bug : backfill only for missing holes
kambstreat Sep 6, 2025
ee81672
fix none error for inc Table
kambstreat Sep 7, 2025
29a3f28
add incremental table queryable range
kambstreat Sep 19, 2025
aa16010
add logging for tableUtils
kambstreat Sep 19, 2025
ff41cc9
add log
kambstreat Sep 19, 2025
aa25f9f
fill incremental holes
kambstreat Sep 22, 2025
3efe8cd
modify incremental aggregation parts
kambstreat Oct 2, 2025
a3bece6
remove logs for debugging
kambstreat Oct 7, 2025
897d18c
fix output schema from incremenntal aggregations. Added unit tests
kambstreat Oct 12, 2025
a52d7d0
resolve merge conflicts
kambstreat Oct 12, 2025
9c446ec
resolve merge conflict
kambstreat Oct 12, 2025
ca14309
add test case to test struct of Average
kambstreat Nov 2, 2025
dfb9226
add option for isIncremental for backward compatibility
kambstreat Nov 2, 2025
7ca4dfc
resolve merge conflicts
kambstreat Nov 2, 2025
ab994bc
fix count operation from incremental IRS
kambstreat Nov 4, 2025
2714ad5
use saw tooth aggregator to compute from daily IRs
kambstreat Nov 10, 2025
d70cfa7
remove Average IR
kambstreat Nov 10, 2025
053fd9d
remove empty spaces and unused functions
kambstreat Nov 10, 2025
99f9788
testing subset of aggregations
kambstreat Nov 22, 2025
7dff16e
remove duplicate test
kambstreat Jan 27, 2026
5852a1b
add unit tests for all aggregations
kambstreat Feb 5, 2026
c29a1df
convert spark datatype to java
kambstreat Feb 7, 2026
bb9b7d2
resolve merge conflicts
kambstreat Feb 7, 2026
2f78a05
scala formatted
kambstreat Feb 7, 2026
b296d70
fix for scala 2.13
kambstreat Feb 7, 2026
3202e7d
fix for scala 2.13
kambstreat Feb 7, 2026
fa2fcba
change unit test
kambstreat Feb 7, 2026
b0e90bf
add toSeq for scala 2.13 compatibility
kambstreat Feb 7, 2026
7138a66
fix last/first tests
kambstreat Feb 9, 2026
325302c
add new test file for incremental aggregations
kambstreat Feb 9, 2026
40aa755
fix failed ci
pengyu-hou Feb 11, 2026
b847a40
remove obsolete comment
kambstreat Feb 13, 2026
7692f24
Merge branch 'kchakka/incremental' of https://github.com/kambstreat/c…
kambstreat Feb 13, 2026
b385cda
revert to master files tor spark_submit.sh and teams.json
kambstreat Feb 13, 2026
464395b
remove log statements during debug
kambstreat Feb 13, 2026
1002744
Add daily_inc suffix to incremental table
kambstreat Feb 13, 2026
cd141ab
fix bug in flatten function
kambstreat Feb 13, 2026
9a25d10
Merge branch 'kchakka/incremental' of https://github.com/kambstreat/c…
kambstreat Feb 13, 2026
a9f3c78
revert changes in GroupByTest
kambstreat Feb 17, 2026
0a8499d
Merge branch 'main' of github.com:airbnb/chronon into kchakka/increme…
pengyu-hou Feb 19, 2026
91d2768
remove comment
kambstreat Mar 1, 2026
5649e0c
Merge branch 'main' into kchakka/incremental
pengyu-hou Mar 13, 2026
c68ae66
Merge branch 'kchakka/incremental' of https://github.com/kambstreat/c…
pengyu-hou Mar 13, 2026
eea3bbf
Merge branch 'main' into kchakka/incremental
pengyu-hou Mar 28, 2026
127410c
Merge branch 'kchakka/incremental' of https://github.com/kambstreat/c…
pengyu-hou Apr 2, 2026
0dc3879
Strengthen incremental backfill test coverage and fix bugs in GroupBy…
pengyu-hou Apr 3, 2026
84d3ece
Merge branch 'main' into kchakka/incremental
pengyu-hou Apr 3, 2026
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
Original file line number Diff line number Diff line change
Expand Up @@ -70,6 +70,11 @@ class RowAggregator(val inputSchema: Seq[(String, DataType)], val aggregationPar
.toArray
.zip(columnAggregators.map(_.irType))

val incrementalOutputSchema: Array[(String, DataType)] = aggregationParts
.map(_.incrementalOutputColumnName)
.toArray
.zip(columnAggregators.map(_.irType))

val outputSchema: Array[(String, DataType)] = aggregationParts
.map(_.outputColumnName)
.toArray
Expand Down
2 changes: 2 additions & 0 deletions api/py/ai/chronon/group_by.py
Original file line number Diff line number Diff line change
Expand Up @@ -390,6 +390,7 @@ def GroupBy(
historical_backfill: Optional[bool] = None,
deprecation_date: Optional[str] = None,
description: Optional[str] = None,
is_incremental: Optional[bool] = None,
**kwargs,
) -> ttypes.GroupBy:
"""
Expand Down Expand Up @@ -608,6 +609,7 @@ def _normalize_source(source):
backfillStartDate=backfill_start_date,
accuracy=accuracy,
derivations=derivations,
isIncremental=is_incremental,
)
validate_group_by(group_by)
return group_by
6 changes: 5 additions & 1 deletion api/src/main/scala/ai/chronon/api/Extensions.scala
Original file line number Diff line number Diff line change
Expand Up @@ -97,7 +97,7 @@ object Extensions {
def cleanName: String = metaData.name.sanitize

def outputTable = s"${metaData.outputNamespace}.${metaData.cleanName}"

def incrementalOutputTable = s"${metaData.outputNamespace}.${metaData.cleanName}_daily_inc"
def preModelTransformsTable = s"${metaData.outputNamespace}.${metaData.cleanName}_pre_mt"
def outputLabelTable = s"${metaData.outputNamespace}.${metaData.cleanName}_labels"
def outputFinalView = s"${metaData.outputNamespace}.${metaData.cleanName}_labeled"
Expand Down Expand Up @@ -178,6 +178,10 @@ object Extensions {

def outputColumnName =
s"${aggregationPart.inputColumn}_$opSuffix${aggregationPart.window.suffix}${bucketSuffix}"

def incrementalOutputColumnName =
s"${aggregationPart.inputColumn}_$opSuffix${bucketSuffix}"
Copy link
Copy Markdown
Collaborator

Choose a reason for hiding this comment

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

should we still keep the aggregationPart.window.suffix? Otherwise, how do we reconstruct the final output column?

Copy link
Copy Markdown
Author

Choose a reason for hiding this comment

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

@pengyu-hou not sure I get it. I can not use the window.suffix right as the intermediate incremental is daily aggregation.


}

implicit class AggregationOps(aggregation: Aggregation) {
Expand Down
1 change: 1 addition & 0 deletions api/thrift/api.thrift
Original file line number Diff line number Diff line change
Expand Up @@ -309,6 +309,7 @@ struct GroupBy {
6: optional string backfillStartDate
// Optional derivation list
7: optional list<Derivation> derivations
8: optional bool isIncremental
}

struct JoinPart {
Expand Down
36 changes: 30 additions & 6 deletions spark/src/main/scala/ai/chronon/spark/Comparison.scala
Original file line number Diff line number Diff line change
Expand Up @@ -20,13 +20,32 @@ import org.slf4j.LoggerFactory
import ai.chronon.online.Extensions.StructTypeOps
import com.google.gson.{Gson, GsonBuilder}
import org.apache.spark.sql.DataFrame
import org.apache.spark.sql.types.{DecimalType, DoubleType, FloatType, MapType}
import org.apache.spark.sql.types.{ArrayType, DecimalType, DoubleType, FloatType, MapType, StructType}
import org.apache.spark.sql.functions.col

import java.util
import scala.collection.mutable

object Comparison {
@transient lazy val logger = LoggerFactory.getLogger(getClass)

// Flatten struct columns into individual columns so nested double fields can be compared with tolerance
private def flattenStructs(df: DataFrame): DataFrame = {
val flattenedSelects = df.schema.fields.toSeq.flatMap { field =>
field.dataType match {
case structType: StructType =>
// Flatten struct fields: struct_name.field_name -> struct_name_field_name
structType.fields.map { subField =>
col(s"${field.name}.${subField.name}").alias(s"${field.name}_${subField.name}")
}.toSeq
case _ =>
// Keep non-struct fields as-is
Seq(col(field.name))
}
}
df.select(flattenedSelects: _*)
}

// used for comparison
def sortedJson(m: Map[String, Any]): String = {
if (m == null) return null
Expand Down Expand Up @@ -69,8 +88,12 @@ object Comparison {
|""".stripMargin
)

val prefixedExpectedDf = prefixColumnName(stringifyMaps(a), s"${aName}_")
val prefixedOutputDf = prefixColumnName(stringifyMaps(b), s"${bName}_")
// Flatten structs so nested double fields can be compared with tolerance
val aFlattened = flattenStructs(stringifyMaps(a))
val bFlattened = flattenStructs(stringifyMaps(b))

val prefixedExpectedDf = prefixColumnName(aFlattened, s"${aName}_")
val prefixedOutputDf = prefixColumnName(bFlattened, s"${bName}_")

val joinExpr = keys
.map(key => prefixedExpectedDf(s"${aName}_$key") <=> prefixedOutputDf(s"${bName}_$key"))
Expand All @@ -82,15 +105,16 @@ object Comparison {
)

var finalDf = joined
// Use flattened schema for comparison
val comparisonColumns =
a.schema.fieldNames.toSet.diff(keys.toSet).toList.sorted
aFlattened.schema.fieldNames.toSet.diff(keys.toSet).toList.sorted
val colOrder =
keys.map(key => { finalDf(s"${aName}_$key").as(key) }) ++
comparisonColumns.flatMap { col =>
List(finalDf(s"${aName}_$col"), finalDf(s"${bName}_$col"))
}
// double columns need to be compared approximately
val doubleCols = a.schema.fields
// double columns need to be compared approximately (now includes flattened struct fields)
val doubleCols = aFlattened.schema.fields
.filter(field =>
field.dataType == DoubleType || field.dataType == FloatType || field.dataType.isInstanceOf[DecimalType])
.map(_.name)
Expand Down
5 changes: 5 additions & 0 deletions spark/src/main/scala/ai/chronon/spark/DataRange.scala
Original file line number Diff line number Diff line change
Expand Up @@ -54,6 +54,11 @@ case class PartitionRange(start: String, end: String)(implicit tableUtils: Table
}
}

def daysBetween: Int = {
if (start == null || end == null) 0
else Stream.iterate(start)(tableUtils.partitionSpec.after).takeWhile(_ <= end).size
}

def isSingleDay: Boolean = {
start == end
}
Expand Down
3 changes: 2 additions & 1 deletion spark/src/main/scala/ai/chronon/spark/Driver.scala
Original file line number Diff line number Diff line change
Expand Up @@ -467,7 +467,8 @@ object Driver {
tableUtils,
args.stepDays.toOption,
args.startPartitionOverride.toOption,
!args.runFirstHole()
!args.runFirstHole(),
Option(args.groupByConf.isIncremental).getOrElse(false)
)

if (args.shouldExport()) {
Expand Down
Loading