Skip to content

Commit

Permalink
rename PUT to update
Browse files Browse the repository at this point in the history
  • Loading branch information
eason-yuchen-liu committed Jul 1, 2024
1 parent 12298b2 commit 104ba9c
Show file tree
Hide file tree
Showing 2 changed files with 34 additions and 33 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -57,9 +57,10 @@ object RecordType extends Enumeration {

def getRecordTypeAsString(recordType: RecordType): String = {
recordType match {
case PUT_RECORD => "PUT"
case DELETE_RECORD => "DELETE"
case _ => "UNDEFINED"
case PUT_RECORD => "update"
case DELETE_RECORD => "delete"
case _ => throw StateStoreErrors.unsupportedOperationException(
"getRecordTypeAsString", recordType.toString)
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -132,9 +132,9 @@ abstract class StateDataSourceChangeDataReadSuite extends StateDataSourceTestBas
.load(tempDir.getAbsolutePath)

val expectedDf = Seq(
Row(Row(null), Row(4), "PUT", 0L, 0),
Row(Row(null), Row(8), "PUT", 1L, 0),
Row(Row(null), Row(10), "PUT", 2L, 0)
Row(Row(null), Row(4), "update", 0L, 0),
Row(Row(null), Row(8), "update", 1L, 0),
Row(Row(null), Row(10), "update", 2L, 0)
)

checkAnswer(stateDf, expectedDf)
Expand Down Expand Up @@ -162,18 +162,18 @@ abstract class StateDataSourceChangeDataReadSuite extends StateDataSourceTestBas
.load(tempDir.getAbsolutePath)

val expectedDf = Seq(
Row(Row(3), Row(1), "PUT", 0L, 1),
Row(Row(3), Row(2), "PUT", 1L, 1),
Row(Row(5), Row(1), "PUT", 1L, 1),
Row(Row(3), Row(3), "PUT", 2L, 1),
Row(Row(5), Row(2), "PUT", 2L, 1),
Row(Row(4), Row(1), "PUT", 0L, 2),
Row(Row(4), Row(2), "PUT", 1L, 2),
Row(Row(4), Row(3), "PUT", 2L, 2),
Row(Row(1), Row(1), "PUT", 0L, 3),
Row(Row(2), Row(1), "PUT", 0L, 4),
Row(Row(2), Row(2), "PUT", 1L, 4),
Row(Row(6), Row(1), "PUT", 2L, 4)
Row(Row(3), Row(1), "update", 0L, 1),
Row(Row(3), Row(2), "update", 1L, 1),
Row(Row(5), Row(1), "update", 1L, 1),
Row(Row(3), Row(3), "update", 2L, 1),
Row(Row(5), Row(2), "update", 2L, 1),
Row(Row(4), Row(1), "update", 0L, 2),
Row(Row(4), Row(2), "update", 1L, 2),
Row(Row(4), Row(3), "update", 2L, 2),
Row(Row(1), Row(1), "update", 0L, 3),
Row(Row(2), Row(1), "update", 0L, 4),
Row(Row(2), Row(2), "update", 1L, 4),
Row(Row(6), Row(1), "update", 2L, 4)
)

checkAnswer(stateDf, expectedDf)
Expand Down Expand Up @@ -201,12 +201,12 @@ abstract class StateDataSourceChangeDataReadSuite extends StateDataSourceTestBas
.load(tempDir.getAbsolutePath)

val expectedDf = Seq(
Row(Row(1), Row(null), "PUT", 0L, 3),
Row(Row(2), Row(null), "PUT", 0L, 4),
Row(Row(3), Row(null), "PUT", 0L, 1),
Row(Row(4), Row(null), "PUT", 0L, 2),
Row(Row(5), Row(null), "PUT", 1L, 1),
Row(Row(6), Row(null), "PUT", 2L, 4)
Row(Row(1), Row(null), "update", 0L, 3),
Row(Row(2), Row(null), "update", 0L, 4),
Row(Row(3), Row(null), "update", 0L, 1),
Row(Row(4), Row(null), "update", 0L, 2),
Row(Row(5), Row(null), "update", 1L, 1),
Row(Row(6), Row(null), "update", 2L, 4)
)

checkAnswer(stateDf, expectedDf)
Expand Down Expand Up @@ -238,11 +238,11 @@ abstract class StateDataSourceChangeDataReadSuite extends StateDataSourceTestBas
.load(tempDir.getAbsolutePath)

val keyWithIndexToValueExpectedDf = Seq(
Row(Row(3, 0L), Row(3, 3L, false), "PUT", 1L, 1),
Row(Row(4, 0L), Row(4, 4L, true), "PUT", 1L, 2),
Row(Row(1, 0L), Row(1, 1L, false), "PUT", 0L, 3),
Row(Row(2, 0L), Row(2, 2L, false), "PUT", 0L, 4),
Row(Row(2, 0L), Row(2, 2L, true), "PUT", 0L, 4)
Row(Row(3, 0L), Row(3, 3L, false), "update", 1L, 1),
Row(Row(4, 0L), Row(4, 4L, true), "update", 1L, 2),
Row(Row(1, 0L), Row(1, 1L, false), "update", 0L, 3),
Row(Row(2, 0L), Row(2, 2L, false), "update", 0L, 4),
Row(Row(2, 0L), Row(2, 2L, true), "update", 0L, 4)
)

checkAnswer(keyWithIndexToValueDf, keyWithIndexToValueExpectedDf)
Expand All @@ -255,10 +255,10 @@ abstract class StateDataSourceChangeDataReadSuite extends StateDataSourceTestBas
.load(tempDir.getAbsolutePath)

val keyToNumValuesDfExpectedDf = Seq(
Row(Row(3), Row(1L), "PUT", 1L, 1),
Row(Row(4), Row(1L), "PUT", 1L, 2),
Row(Row(1), Row(1L), "PUT", 0L, 3),
Row(Row(2), Row(1L), "PUT", 0L, 4)
Row(Row(3), Row(1L), "update", 1L, 1),
Row(Row(4), Row(1L), "update", 1L, 2),
Row(Row(1), Row(1L), "update", 0L, 3),
Row(Row(2), Row(1L), "update", 0L, 4)
)

checkAnswer(keyToNumValuesDf, keyToNumValuesDfExpectedDf)
Expand Down

0 comments on commit 104ba9c

Please sign in to comment.