Skip to content

Commit

Permalink
DP-2918: continue on data error (#62)
Browse files Browse the repository at this point in the history
* DP-2918: drop or serialise collections in schema inference as well

* DP-2918: add a test for dropping arrays in normalization

* DP-2918: Return Either in normalizer, and fail in the task when normalization fails

* DP-2918: Put failing records in DLQ

* DP-2918: Change exception name

* DP-2918: Introduce ErrorPolicyConfig class

* DP-2918: Tests for ErrorPolicyConfig

* DP-2918: Add continueOninvalidInput configuration

* DP-2918: Call errand record reporter only for InvalidInputException

* DP-2918: Change names of a test

* DP-2918: Scalafmt

* DP-2918: Fix maxretries reset in case of continue on error

* DP-2918: Add comment

* DP-2918: InvalidInputErrorHandler test

* DP-2918: Fix InvalidInputErrorHandler

* DP-2918: Integrate InvalidInputErrorHandler into EmsSinkTask

* DP-2918: Add a comment

* DP-2918: Add e2e test

* DP-2918: Remove a comment

* DP-2918: Scalafmt

* DP-2918: remove a println
  • Loading branch information
nicmart authored Mar 11, 2024
1 parent 5f9eaca commit f020cc8
Show file tree
Hide file tree
Showing 24 changed files with 562 additions and 224 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,6 @@ package com.celonis.kafka.connect.ems.config

import cats.implicits._
import com.celonis.kafka.connect.ems.config.EmsSinkConfigConstants._
import com.celonis.kafka.connect.ems.errors.ErrorPolicy
import com.celonis.kafka.connect.ems.storage.FileSystemOperations
import com.celonis.kafka.connect.transform.FlattenerConfig
import com.celonis.kafka.connect.transform.PreConversionConfig
Expand All @@ -36,9 +35,8 @@ final case class EmsSinkConfig(
target: String,
connectionId: Option[String],
authorization: AuthorizationHeader,
errorPolicy: ErrorPolicy,
errorPolicyConfig: ErrorPolicyConfig,
commitPolicy: CommitPolicyConfig,
retries: RetryConfig,
workingDir: Path,
parquet: ParquetConfig,
primaryKeys: List[String],
Expand Down Expand Up @@ -106,8 +104,7 @@ object EmsSinkConfig {
url <- extractURL(props)
table <- extractTargetTable(props)
authorization <- AuthorizationHeader.extract(props)
error <- ErrorPolicy.extract(props)
retry <- RetryConfig.extractRetry(props)
errorPolicyConfig <- ErrorPolicyConfig.extract(props)
useInMemoryFs = PropertiesHelper.getBoolean(props, USE_IN_MEMORY_FS_KEY).getOrElse(USE_IN_MEMORY_FS_DEFAULT)
allowNullsAsPks = PropertiesHelper.getBoolean(props, NULL_PK_KEY).getOrElse(NULL_PK_KEY_DEFAULT)
tempDir <- if (useInMemoryFs) Right(FileSystemOperations.InMemoryPseudoDir) else extractWorkingDirectory(props)
Expand Down Expand Up @@ -135,9 +132,8 @@ object EmsSinkConfig {
table,
connectionId,
authorization,
error,
errorPolicyConfig,
commitPolicy,
retry,
tempDir,
parquetConfig,
primaryKeys,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -64,10 +64,10 @@ object EmsSinkConfigConstants {
| Default is $PARQUET_ROW_GROUP_SIZE_BYTES_DEFAULT.
| """.stripMargin

val NBR_OF_RETRIES_KEY = s"$CONNECTOR_PREFIX.max.retries"
val NBR_OF_RETRIES_DOC =
val ERROR_POLICY_RETRIES_KEY = s"$CONNECTOR_PREFIX.max.retries"
val ERROR_POLICY_RETRIES_DOC =
"The maximum number of times to re-attempt to write the records before the task is marked as failed."
val NBR_OF_RETIRES_DEFAULT: Int = 10
val ERROR_POLICY_RETRIES_DEFAULT: Int = 10

val ERROR_POLICY_KEY = s"$CONNECTOR_PREFIX.error.policy"
val ERROR_POLICY_DOC: String =
Expand All @@ -76,7 +76,7 @@ object EmsSinkConfigConstants {
| There are three available options:
| CONTINUE - the error is swallowed
| THROW - the error is allowed to propagate.
| RETRY - The exception causes the Connect framework to retry the message. The number of retries is set by $NBR_OF_RETRIES_KEY.
| RETRY - The exception causes the Connect framework to retry the message. The number of retries is set by $ERROR_POLICY_RETRIES_KEY.
|All errors will be logged automatically, even if the code swallows them.
""".stripMargin
val ERROR_POLICY_DEFAULT = "THROW"
Expand All @@ -85,6 +85,11 @@ object EmsSinkConfigConstants {
val ERROR_RETRY_INTERVAL_DOC = "The time in milliseconds between retries."
val ERROR_RETRY_INTERVAL_DEFAULT: Long = 60000L

val ERROR_CONTINUE_ON_INVALID_INPUT_KEY = s"$CONNECTOR_PREFIX.error.policy.continue.on.invalid.input"
val ERROR_CONTINUE_ON_INVALID_INPUT_DOC: String =
"If set to 'true', connector will continue when invalid input errors occur. Invalid records will be sent to the DLQ, if present. "
val ERROR_CONTINUE_ON_INVALID_INPUT_DEFAULT = false

val FALLBACK_VARCHAR_LENGTH_KEY = s"$CONNECTOR_PREFIX.data.fallback.varchar.length"
val FALLBACK_VARCHAR_LENGTH_DOC =
"Optional parameter representing the STRING (VARCHAR) length when the schema is created in EMS. Must be greater than 0 and smaller or equal than 65000"
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -134,15 +134,15 @@ object EmsSinkConfigDef {
ERROR_POLICY_DOC,
)
.define(
NBR_OF_RETRIES_KEY,
ERROR_POLICY_RETRIES_KEY,
Type.INT,
NBR_OF_RETIRES_DEFAULT,
ERROR_POLICY_RETRIES_DEFAULT,
Importance.MEDIUM,
NBR_OF_RETRIES_DOC,
ERROR_POLICY_RETRIES_DOC,
"Error",
2,
ConfigDef.Width.LONG,
NBR_OF_RETRIES_KEY,
ERROR_POLICY_RETRIES_KEY,
)
.define(
ERROR_RETRY_INTERVAL,
Expand All @@ -155,6 +155,13 @@ object EmsSinkConfigDef {
ConfigDef.Width.LONG,
ERROR_RETRY_INTERVAL,
)
.define(
ERROR_CONTINUE_ON_INVALID_INPUT_KEY,
Type.BOOLEAN,
ERROR_CONTINUE_ON_INVALID_INPUT_DEFAULT,
Importance.MEDIUM,
ERROR_CONTINUE_ON_INVALID_INPUT_DOC,
)
.define(
PARQUET_ROW_GROUP_SIZE_BYTES_KEY,
Type.INT,
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,75 @@
/*
* Copyright 2024 Celonis SE
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package com.celonis.kafka.connect.ems.config

import cats.syntax.either._
import com.celonis.kafka.connect.ems.config.EmsSinkConfigConstants.ERROR_CONTINUE_ON_INVALID_INPUT_DEFAULT
import com.celonis.kafka.connect.ems.config.EmsSinkConfigConstants.ERROR_CONTINUE_ON_INVALID_INPUT_KEY
import com.celonis.kafka.connect.ems.config.EmsSinkConfigConstants.ERROR_POLICY_DOC
import com.celonis.kafka.connect.ems.config.EmsSinkConfigConstants.ERROR_POLICY_KEY
import com.celonis.kafka.connect.ems.config.ErrorPolicyConfig.ErrorPolicyType
import com.celonis.kafka.connect.ems.config.ErrorPolicyConfig.ErrorPolicyType.CONTINUE
import com.celonis.kafka.connect.ems.config.ErrorPolicyConfig.ErrorPolicyType.RETRY
import com.celonis.kafka.connect.ems.config.ErrorPolicyConfig.ErrorPolicyType.THROW
import com.celonis.kafka.connect.ems.config.PropertiesHelper.error
import com.celonis.kafka.connect.ems.config.PropertiesHelper.getBoolean
import com.celonis.kafka.connect.ems.config.PropertiesHelper.nonEmptyStringOr
import com.celonis.kafka.connect.ems.errors.ErrorPolicy
import com.celonis.kafka.connect.ems.errors.InvalidInputErrorHandler
import org.apache.kafka.connect.sink.ErrantRecordReporter

final case class ErrorPolicyConfig(
policyType: ErrorPolicyType,
retryConfig: RetryConfig,
continueOnInvalidInput: Boolean,
) {
lazy val errorPolicy: ErrorPolicy =
policyType match {
case ErrorPolicyType.THROW => ErrorPolicy.Throw
case ErrorPolicyType.CONTINUE => ErrorPolicy.Continue
case ErrorPolicyType.RETRY => ErrorPolicy.Retry
}

def invalidInputErrorHandler(reporter: Option[ErrantRecordReporter]): InvalidInputErrorHandler =
new InvalidInputErrorHandler(continueOnInvalidInput, reporter)
}

object ErrorPolicyConfig {
sealed trait ErrorPolicyType
object ErrorPolicyType {
case object THROW extends ErrorPolicyType
case object CONTINUE extends ErrorPolicyType
case object RETRY extends ErrorPolicyType
}

def extract(props: Map[String, _]): Either[String, ErrorPolicyConfig] =
for {
policyType <- extractType(props)
retryConfig <- RetryConfig.extractRetry(props)
continueOnInvalidInput =
getBoolean(props, ERROR_CONTINUE_ON_INVALID_INPUT_KEY).getOrElse(ERROR_CONTINUE_ON_INVALID_INPUT_DEFAULT)
} yield ErrorPolicyConfig(policyType, retryConfig, continueOnInvalidInput = continueOnInvalidInput)

private def extractType(props: Map[String, _]): Either[String, ErrorPolicyType] =
nonEmptyStringOr(props, ERROR_POLICY_KEY, ERROR_POLICY_DOC).map(_.toUpperCase)
.flatMap {
case "THROW" => THROW.asRight
case "RETRY" => RETRY.asRight
case "CONTINUE" => CONTINUE.asRight
case _ => error(ERROR_POLICY_KEY, ERROR_POLICY_DOC)
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -32,11 +32,11 @@ object RetryConfig {
else value.asRight[String]
case None => ERROR_RETRY_INTERVAL_DEFAULT.asRight[String]
}
retries <- PropertiesHelper.getInt(props, NBR_OF_RETRIES_KEY) match {
retries <- PropertiesHelper.getInt(props, ERROR_POLICY_RETRIES_KEY) match {
case Some(value) =>
if (value <= 0) error(NBR_OF_RETRIES_KEY, "Number of retries needs to be greater than 0.")
if (value <= 0) error(ERROR_POLICY_RETRIES_KEY, "Number of retries needs to be greater than 0.")
else value.asRight[String]
case None => NBR_OF_RETIRES_DEFAULT.asRight[String]
case None => ERROR_POLICY_RETRIES_DEFAULT.asRight[String]
}
} yield RetryConfig(retries, interval)

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,23 +17,16 @@
package com.celonis.kafka.connect.ems.errors

import org.apache.kafka.connect.errors.ConnectException
import org.http4s.DecodeFailure
import org.http4s.Status

sealed trait EmsSinkException {
def getMessage: String
}

case class UploadFailedException(status: Status, msg: String, throwable: Throwable)
final case class UploadFailedException(status: Status, msg: String, throwable: Throwable)
extends ConnectException(msg, throwable)
with EmsSinkException

case class UnexpectedUploadException(msg: String, throwable: Throwable) extends ConnectException(msg, throwable)
final case class InvalidInputException(msg: String) extends ConnectException(msg) with EmsSinkException

case class UploadInvalidResponseException(failure: DecodeFailure)
extends ConnectException(failure.getMessage(), failure)
with EmsSinkException

case class InvalidInputException(msg: String) extends ConnectException(msg) with EmsSinkException

case class FailedObfuscationException(msg: String) extends ConnectException(msg) with EmsSinkException
final case class FailedObfuscationException(msg: String) extends ConnectException(msg) with EmsSinkException
Original file line number Diff line number Diff line change
Expand Up @@ -16,24 +16,15 @@

package com.celonis.kafka.connect.ems.errors

import cats.syntax.either._
import com.celonis.kafka.connect.ems.config.EmsSinkConfigConstants.ERROR_POLICY_DOC
import com.celonis.kafka.connect.ems.config.EmsSinkConfigConstants.ERROR_POLICY_KEY
import com.celonis.kafka.connect.ems.config.PropertiesHelper.error
import com.celonis.kafka.connect.ems.config.PropertiesHelper.nonEmptyStringOr
import com.typesafe.scalalogging.StrictLogging

import enumeratum._
import enumeratum.EnumEntry.Uppercase
import org.apache.kafka.connect.errors.ConnectException
import org.apache.kafka.connect.errors.RetriableException

sealed trait ErrorPolicy extends EnumEntry with Uppercase {
sealed trait ErrorPolicy {
def handle(error: Throwable, retries: Int): Unit
}

object ErrorPolicy extends Enum[ErrorPolicy] {
val values: IndexedSeq[ErrorPolicy] = findValues
object ErrorPolicy {

case object Continue extends ErrorPolicy with StrictLogging {
override def handle(error: Throwable, retries: Int): Unit =
Expand All @@ -50,19 +41,12 @@ object ErrorPolicy extends Enum[ErrorPolicy] {
case object Retry extends ErrorPolicy with StrictLogging {
override def handle(error: Throwable, retries: Int): Unit =
if (retries == 0) {
logger.warn(s"Error policy is set to RETRY and no more attempts left.", error)
throw new ConnectException(error)
} else {
logger.warn(s"Error policy is set to RETRY. Remaining attempts [$retries]", error)
throw new RetriableException(error)
}
}

def extract(props: Map[String, _]): Either[String, ErrorPolicy] =
nonEmptyStringOr(props, ERROR_POLICY_KEY, ERROR_POLICY_DOC)
.flatMap { constant =>
ErrorPolicy.withNameInsensitiveOption(constant) match {
case Some(value) => value.asRight[String]
case None => error(ERROR_POLICY_KEY, ERROR_POLICY_DOC)
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,36 @@
/*
* Copyright 2024 Celonis SE
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package com.celonis.kafka.connect.ems.errors

import com.typesafe.scalalogging.StrictLogging
import org.apache.kafka.connect.sink.ErrantRecordReporter
import org.apache.kafka.connect.sink.SinkRecord

/** Error policies work at the batch level, while this handler works at the record level. It works only for
* InvalidInputExceptions, that are errors due to defects of single records.
*/
final class InvalidInputErrorHandler(
continueOnInvalidInput: Boolean,
errantRecordReporter: Option[ErrantRecordReporter],
) extends StrictLogging {
def handle(record: SinkRecord, error: Throwable): Unit = error match {
case _: InvalidInputException if continueOnInvalidInput =>
logger.warn("Error policy is set to CONTINUE on InvalidInput", error)
errantRecordReporter.foreach(errantRecordReporter => errantRecordReporter.report(record, error))
case _ => throw error
}
}
Loading

0 comments on commit f020cc8

Please sign in to comment.