-
Notifications
You must be signed in to change notification settings - Fork 1
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Merge pull request #4 from cultureamp/FW-283/unify-legacy-slack-data
feat: Custom transformer to unify slack data
- Loading branch information
Showing
4 changed files
with
374 additions
and
1 deletion.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
105 changes: 105 additions & 0 deletions
105
...lin/com/cultureamp/kafka/connect/plugins/transforms/UnifyLegacySlackIntegrationPayload.kt
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,105 @@ | ||
package com.cultureamp.kafka.connect.plugins.transforms | ||
|
||
import org.apache.kafka.common.config.ConfigDef | ||
import org.apache.kafka.connect.connector.ConnectRecord | ||
import org.apache.kafka.connect.data.Field | ||
import org.apache.kafka.connect.data.Schema | ||
import org.apache.kafka.connect.data.SchemaBuilder | ||
import org.apache.kafka.connect.data.Struct | ||
import org.apache.kafka.connect.errors.DataException | ||
import org.apache.kafka.connect.transforms.Transformation | ||
import org.apache.kafka.connect.transforms.util.Requirements | ||
|
||
class UnifyLegacySlackIntegrationPayload<R : ConnectRecord<R>> : Transformation<R> { | ||
private val ignoredAttributes = arrayListOf("account_aggregate_id", "oauth_response_data.access_token", "oauth_response_data.team_id", "oauth_response_data.team_name", "oauth_response_data.bot.bot_access_token", "oauth_response_data.team.id", "oauth_response_data.team.name") | ||
private val PURPOSE = "unify legacy slack integration data" | ||
override fun configure(configs: MutableMap<String, *>?) {} | ||
|
||
override fun close() {} | ||
|
||
private fun removeIgnoredAttributes(fields: List<Field>, builder: SchemaBuilder, hierarchy: String = ""): SchemaBuilder { | ||
for (field in fields) { | ||
if ("$hierarchy${field.name()}" !in ignoredAttributes) { | ||
if (field.schema().type().getName() == "struct") { | ||
val childSchema = removeIgnoredAttributes(field.schema().fields(), SchemaBuilder.struct(), "$hierarchy${field.name()}.") | ||
// Only add child schema if is not empty | ||
if (childSchema.fields().isNotEmpty()) { | ||
builder.field(field.name(), childSchema.build()) | ||
} | ||
} else { | ||
builder | ||
.field(field.name(), field.schema()) | ||
} | ||
} | ||
} | ||
return builder | ||
} | ||
|
||
private fun populateValue(originalValues: Struct, updatedValues: Struct): Struct { | ||
val newFields = updatedValues.schema().fields() | ||
for (field in newFields) { | ||
try { | ||
if (field.schema().type().getName() == "struct") { | ||
val childValue = populateValue(Requirements.requireStruct(originalValues.get(field.name()), PURPOSE), Struct(field.schema())) | ||
updatedValues.put(field.name(), childValue) | ||
} else { | ||
updatedValues | ||
.put(field.name(), originalValues.get(field.name())) | ||
} | ||
} catch (e: DataException) { | ||
// This is catch exception thrown when field.name() in .get(field.name())) does not exists | ||
} | ||
} | ||
return updatedValues | ||
} | ||
|
||
private fun extractUnifiedValues(oauthResponseData: Struct): Triple<String, String, String> { | ||
var teamId: String | ||
var teamName: String | ||
var accessToken: String | ||
|
||
try { | ||
// Only Slack Integration OAuth V1 has "bot" child element | ||
val dot: Struct = Requirements.requireStruct(oauthResponseData.get("bot"), PURPOSE) | ||
teamId = oauthResponseData.get("team_id") as String | ||
teamName = oauthResponseData.get("team_name") as String | ||
accessToken = dot.get("bot_access_token") as String | ||
} catch (e: DataException) { | ||
// Slack Integration OAuth V2 Payload | ||
val team: Struct = Requirements.requireStruct(oauthResponseData.get("team"), PURPOSE) | ||
teamId = team.get("id") as String | ||
teamName = team.get("name") as String | ||
accessToken = oauthResponseData.get("access_token") as String | ||
} | ||
return Triple(teamId, teamName, accessToken) | ||
} | ||
|
||
override fun apply(record: R): R { | ||
val valueStruct: Struct = Requirements.requireStruct(record.value(), PURPOSE) | ||
val oauthResponseData: Struct = Requirements.requireStruct(valueStruct.get("oauth_response_data"), PURPOSE) | ||
val updatedSchemaBuilder: SchemaBuilder = removeIgnoredAttributes(valueStruct.schema().fields(), SchemaBuilder.struct()) | ||
val(teamId, teamName, accessToken) = extractUnifiedValues(oauthResponseData) | ||
|
||
// Add back the unified fields | ||
val modifiedPayloadSchema = updatedSchemaBuilder | ||
.name("com.cultureamp.murmur.slack_integrations") | ||
.field("account_aggregate_id", Schema.STRING_SCHEMA) | ||
.field("access_token", Schema.STRING_SCHEMA) | ||
.field("team_id", Schema.STRING_SCHEMA) | ||
.field("team_name", Schema.STRING_SCHEMA) | ||
.build() | ||
|
||
val updatedValuesStruct: Struct = populateValue(valueStruct, Struct(modifiedPayloadSchema)) | ||
|
||
val modifiedPayloadStruct = updatedValuesStruct | ||
.put("access_token", accessToken) | ||
.put("team_id", teamId) | ||
.put("team_name", teamName) | ||
|
||
return record.newRecord(record.topic(), record.kafkaPartition(), record.keySchema(), record.key(), modifiedPayloadSchema, modifiedPayloadStruct, record.timestamp()) | ||
} | ||
|
||
override fun config(): ConfigDef { | ||
return ConfigDef() | ||
} | ||
} |
158 changes: 158 additions & 0 deletions
158
...com/cultureamp/kafka/connect/plugins/transforms/UnifyLegacySlackIntegrationPayloadTest.kt
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,158 @@ | ||
package com.cultureamp.kafka.connect.plugins.transforms | ||
|
||
import org.apache.kafka.connect.data.Schema | ||
import org.apache.kafka.connect.data.SchemaBuilder | ||
import org.apache.kafka.connect.data.Struct | ||
import org.apache.kafka.connect.source.SourceRecord | ||
import kotlin.test.Test | ||
import kotlin.test.assertEquals | ||
|
||
class UnifyLegacySlackIntegrationPayloadTest { | ||
private val CREATED_AT = "2022-04-08T06:40:02.649Z" | ||
private val ACCESS_TOKEN = "xoxb-12345" | ||
private val ACCOUNT_ID = "account-id" | ||
private val TEAM_ID = "team-id" | ||
private val TEAM_NAME = "team-name" | ||
|
||
private fun createOAuthV1Payload(): Pair<Schema, Struct> { | ||
val botSchema = SchemaBuilder.struct() | ||
.field("bot_access_token", Schema.STRING_SCHEMA) | ||
.field("bot_user_id", Schema.STRING_SCHEMA) | ||
.build() | ||
|
||
val botStruct = Struct(botSchema) | ||
.put("bot_access_token", ACCESS_TOKEN) | ||
.put("bot_user_id", "UV8DT789F") | ||
|
||
val oauthResponseDataSchema = SchemaBuilder.struct() | ||
.field("access_token", Schema.STRING_SCHEMA) | ||
.field("team_id", Schema.STRING_SCHEMA) | ||
.field("team_name", Schema.STRING_SCHEMA) | ||
.field("bot", botSchema) | ||
.build() | ||
|
||
val oauthResponseDataStruct = Struct(oauthResponseDataSchema) | ||
.put("access_token", "a-b-c") | ||
.put("team_id", TEAM_ID) | ||
.put("team_name", TEAM_NAME) | ||
.put("bot", botStruct) | ||
|
||
val payloadSchema = SchemaBuilder.struct() | ||
.field("account_aggregate_id", Schema.STRING_SCHEMA) | ||
.field("created_at", Schema.STRING_SCHEMA) | ||
.field("oauth_response_data", oauthResponseDataSchema) | ||
.build() | ||
|
||
val payloadStruct: Struct = Struct(payloadSchema) | ||
.put("oauth_response_data", oauthResponseDataStruct) | ||
.put("account_aggregate_id", ACCOUNT_ID) | ||
.put("created_at", CREATED_AT) | ||
|
||
return payloadSchema to payloadStruct | ||
} | ||
|
||
private fun createOAuthV2Payload(): Pair<Schema, Struct> { | ||
val teamSchema = SchemaBuilder.struct() | ||
.field("id", Schema.STRING_SCHEMA) | ||
.field("name", Schema.STRING_SCHEMA) | ||
.build() | ||
|
||
val teamStruct = Struct(teamSchema) | ||
.put("id", TEAM_ID) | ||
.put("name", TEAM_NAME) | ||
|
||
val oauthResponseDataSchema = SchemaBuilder.struct() | ||
.field("access_token", Schema.STRING_SCHEMA) | ||
.field("team", teamSchema) | ||
.build() | ||
|
||
val oauthResponseDataStruct = Struct(oauthResponseDataSchema) | ||
.put("access_token", ACCESS_TOKEN) | ||
.put("team", teamStruct) | ||
|
||
val payloadSchema = SchemaBuilder.struct() | ||
.field("account_aggregate_id", Schema.STRING_SCHEMA) | ||
.field("oauth_response_data", oauthResponseDataSchema) | ||
.build() | ||
|
||
val payloadStruct: Struct = Struct(payloadSchema) | ||
.put("oauth_response_data", oauthResponseDataStruct) | ||
.put("account_aggregate_id", ACCOUNT_ID) | ||
|
||
return payloadSchema to payloadStruct | ||
} | ||
|
||
private fun oAuthV1ExpectedValue(): Pair<Schema, Struct> { | ||
val botSchema = SchemaBuilder.struct() | ||
.field("bot_user_id", Schema.STRING_SCHEMA) | ||
.build() | ||
|
||
val botStruct = Struct(botSchema) | ||
.put("bot_user_id", "UV8DT789F") | ||
|
||
val oauthResponseDataSchema = SchemaBuilder.struct() | ||
.field("bot", botSchema) | ||
.build() | ||
|
||
val oauthResponseDataStruct = Struct(oauthResponseDataSchema) | ||
.put("bot", botStruct) | ||
|
||
val expectedSchema = SchemaBuilder.struct() | ||
.name("com.cultureamp.murmur.slack_integrations") | ||
.field("created_at", Schema.STRING_SCHEMA) | ||
.field("oauth_response_data", oauthResponseDataSchema) | ||
.field("account_aggregate_id", Schema.STRING_SCHEMA) | ||
.field("access_token", Schema.STRING_SCHEMA) | ||
.field("team_id", Schema.STRING_SCHEMA) | ||
.field("team_name", Schema.STRING_SCHEMA) | ||
.build() | ||
|
||
val expectedValue = Struct(expectedSchema) | ||
.put("created_at", CREATED_AT) | ||
.put("oauth_response_data", oauthResponseDataStruct) | ||
.put("account_aggregate_id", ACCOUNT_ID) | ||
.put("access_token", ACCESS_TOKEN) | ||
.put("team_id", TEAM_ID) | ||
.put("team_name", TEAM_NAME) | ||
|
||
return expectedSchema to expectedValue | ||
} | ||
|
||
@Test | ||
fun `With Legacy Slack Integration Data`() { | ||
val transformer: UnifyLegacySlackIntegrationPayload<SourceRecord> = UnifyLegacySlackIntegrationPayload() | ||
val (payloadSchema, payloadStruct) = createOAuthV1Payload() | ||
|
||
val transformedRecord: SourceRecord = transformer.apply(SourceRecord(null, null, "test", payloadSchema, payloadStruct)) | ||
|
||
val (expectedSchema, expectedValue) = oAuthV1ExpectedValue() | ||
|
||
assertEquals(expectedValue, transformedRecord.value()) | ||
assertEquals(expectedSchema, transformedRecord.valueSchema()) | ||
} | ||
|
||
@Test | ||
fun `With Slack Integration Data`() { | ||
val transformer: UnifyLegacySlackIntegrationPayload<SourceRecord> = UnifyLegacySlackIntegrationPayload() | ||
|
||
val (payloadSchema, payloadStruct) = createOAuthV2Payload() | ||
|
||
val transformedRecord: SourceRecord = transformer.apply(SourceRecord(null, null, "test", payloadSchema, payloadStruct)) | ||
|
||
val expectedSchema = SchemaBuilder.struct() | ||
.name("com.cultureamp.murmur.slack_integrations") | ||
.field("account_aggregate_id", Schema.STRING_SCHEMA) | ||
.field("access_token", Schema.STRING_SCHEMA) | ||
.field("team_id", Schema.STRING_SCHEMA) | ||
.field("team_name", Schema.STRING_SCHEMA) | ||
.build() | ||
val expectedValue = Struct(expectedSchema) | ||
.put("account_aggregate_id", ACCOUNT_ID) | ||
.put("access_token", ACCESS_TOKEN) | ||
.put("team_id", TEAM_ID) | ||
.put("team_name", TEAM_NAME) | ||
|
||
assertEquals(expectedValue, transformedRecord.value()) | ||
assertEquals(expectedSchema, transformedRecord.valueSchema()) | ||
} | ||
} |