-
Notifications
You must be signed in to change notification settings - Fork 1k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
feat: add KSQL processing log message on uncaught streams exceptions
- Loading branch information
Showing
10 changed files
with
247 additions
and
8 deletions.
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
96 changes: 96 additions & 0 deletions
96
ksqldb-engine/src/main/java/io/confluent/ksql/util/KafkaStreamsThreadError.java
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,96 @@ | ||
/* | ||
* Copyright 2020 Confluent Inc. | ||
* | ||
* Licensed under the Confluent Community License (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.confluent.io/confluent-community-license | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT | ||
* WARRANTIES OF ANY KIND, either express or implied. See the License for the | ||
* specific language governing permissions and limitations under the License. | ||
*/ | ||
|
||
package io.confluent.ksql.util; | ||
|
||
import static io.confluent.ksql.logging.processing.ProcessingLogMessageSchema.MessageType; | ||
import static java.util.Objects.requireNonNull; | ||
|
||
import io.confluent.ksql.logging.processing.ProcessingLogConfig; | ||
import io.confluent.ksql.logging.processing.ProcessingLogMessageSchema; | ||
import io.confluent.ksql.logging.processing.ProcessingLogger; | ||
import java.util.Objects; | ||
import org.apache.kafka.connect.data.SchemaAndValue; | ||
import org.apache.kafka.connect.data.Struct; | ||
|
||
public final class KafkaStreamsThreadError implements ProcessingLogger.ErrorMessage { | ||
public static KafkaStreamsThreadError of( | ||
final String errorMsg, | ||
final Thread thread, | ||
final Throwable exception | ||
) { | ||
return new KafkaStreamsThreadError(errorMsg, thread, exception); | ||
} | ||
|
||
private final String errorMsg; | ||
private final Thread thread; | ||
private final Throwable exception; | ||
|
||
private KafkaStreamsThreadError( | ||
final String errorMsg, | ||
final Thread thread, | ||
final Throwable exception | ||
) { | ||
this.errorMsg = requireNonNull(errorMsg, "errorMsg"); | ||
this.thread = requireNonNull(thread, "thread"); | ||
this.exception = requireNonNull(exception, "exception"); | ||
} | ||
|
||
@Override | ||
public boolean equals(final Object o) { | ||
if (this == o) { | ||
return true; | ||
} | ||
if (o == null || getClass() != o.getClass()) { | ||
return false; | ||
} | ||
final KafkaStreamsThreadError that = (KafkaStreamsThreadError) o; | ||
return Objects.equals(errorMsg, that.errorMsg) | ||
&& Objects.equals(thread.getName(), thread.getName()) | ||
&& Objects.equals(exception.getClass(), that.exception.getClass()) | ||
&& Objects.equals(exception.toString(), that.exception.toString()); | ||
} | ||
|
||
@Override | ||
public int hashCode() { | ||
return Objects.hash(thread, exception); | ||
} | ||
|
||
@Override | ||
public SchemaAndValue get(final ProcessingLogConfig config) { | ||
final Struct struct = new Struct(ProcessingLogMessageSchema.PROCESSING_LOG_SCHEMA) | ||
.put(ProcessingLogMessageSchema.TYPE, | ||
MessageType.KAFKA_STREAMS_THREAD_ERROR.getTypeId()) | ||
.put(ProcessingLogMessageSchema.KAFKA_STREAMS_THREAD_ERROR, | ||
streamsThreadError()); | ||
|
||
return new SchemaAndValue(ProcessingLogMessageSchema.PROCESSING_LOG_SCHEMA, struct); | ||
} | ||
|
||
private Struct streamsThreadError() { | ||
final Struct threadError = new Struct(MessageType.KAFKA_STREAMS_THREAD_ERROR.getSchema()) | ||
.put( | ||
ProcessingLogMessageSchema.KAFKA_STREAMS_THREAD_ERROR_FIELD_MESSAGE, | ||
errorMsg) | ||
.put( | ||
ProcessingLogMessageSchema.KAFKA_STREAMS_THREAD_ERROR_FIELD_NAME, | ||
thread.getName()) | ||
.put( | ||
ProcessingLogMessageSchema.KAFKA_STREAMS_THREAD_ERROR_FIELD_CAUSE, | ||
ErrorMessageUtil.getErrorMessages(exception)); | ||
|
||
return threadError; | ||
} | ||
} |
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
76 changes: 76 additions & 0 deletions
76
ksqldb-engine/src/test/java/io/confluent/ksql/util/KafkaStreamsThreadErrorTest.java
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,76 @@ | ||
/* | ||
* Copyright 2020 Confluent Inc. | ||
* | ||
* Licensed under the Confluent Community License (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.confluent.io/confluent-community-license | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT | ||
* WARRANTIES OF ANY KIND, either express or implied. See the License for the | ||
* specific language governing permissions and limitations under the License. | ||
*/ | ||
|
||
package io.confluent.ksql.util; | ||
|
||
import static org.hamcrest.MatcherAssert.assertThat; | ||
import static org.hamcrest.Matchers.equalTo; | ||
import static org.hamcrest.Matchers.notNullValue; | ||
|
||
import io.confluent.ksql.logging.processing.ProcessingLogConfig; | ||
import io.confluent.ksql.logging.processing.ProcessingLogMessageSchema; | ||
import io.confluent.ksql.logging.processing.ProcessingLogger.ErrorMessage; | ||
import java.util.Collections; | ||
import org.apache.kafka.connect.data.SchemaAndValue; | ||
import org.apache.kafka.connect.data.Struct; | ||
import org.junit.Test; | ||
import org.junit.runner.RunWith; | ||
import org.mockito.junit.MockitoJUnitRunner; | ||
|
||
@RunWith(MockitoJUnitRunner.class) | ||
public class KafkaStreamsThreadErrorTest { | ||
private final ProcessingLogConfig config = new ProcessingLogConfig(Collections.emptyMap()); | ||
|
||
private Thread thread = new Thread("thread-1"); | ||
private Throwable error = new Exception("cause0"); | ||
|
||
@Test | ||
public void shouldBuildKafkaStreamThreadErrorCorrectly() { | ||
// Given: | ||
final ErrorMessage errorMessage = KafkaStreamsThreadError.of("errorMsg", thread, error); | ||
|
||
// When: | ||
final SchemaAndValue msgAndSchema = errorMessage.get(config); | ||
|
||
// Then: | ||
assertThat(msgAndSchema.schema(), equalTo(ProcessingLogMessageSchema.PROCESSING_LOG_SCHEMA)); | ||
final Struct msg = (Struct) msgAndSchema.value(); | ||
assertThat( | ||
msg.get(ProcessingLogMessageSchema.TYPE), | ||
equalTo(ProcessingLogMessageSchema.MessageType.KAFKA_STREAMS_THREAD_ERROR.getTypeId()) | ||
); | ||
assertThat( | ||
msg.get(ProcessingLogMessageSchema.KAFKA_STREAMS_THREAD_ERROR), | ||
notNullValue() | ||
); | ||
final Struct kafkaStreamsThreadError = | ||
msg.getStruct(ProcessingLogMessageSchema.KAFKA_STREAMS_THREAD_ERROR); | ||
assertThat( | ||
kafkaStreamsThreadError.get( | ||
ProcessingLogMessageSchema.KAFKA_STREAMS_THREAD_ERROR_FIELD_MESSAGE), | ||
equalTo("errorMsg") | ||
); | ||
assertThat( | ||
kafkaStreamsThreadError.get( | ||
ProcessingLogMessageSchema.KAFKA_STREAMS_THREAD_ERROR_FIELD_NAME), | ||
equalTo("thread-1") | ||
); | ||
assertThat( | ||
kafkaStreamsThreadError.get( | ||
ProcessingLogMessageSchema.KAFKA_STREAMS_THREAD_ERROR_FIELD_CAUSE), | ||
equalTo(ErrorMessageUtil.getErrorMessages(error)) | ||
); | ||
} | ||
} |
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
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
Oops, something went wrong.