Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

KAFKA-5157: Options for handling corrupt data during deserialization #3423

Conversation

@enothereska
Copy link
Contributor

@enothereska enothereska commented Jun 23, 2017

@enothereska
Copy link
Contributor Author

@enothereska enothereska commented Jun 23, 2017

More tests coming, but basic structure should be in place.

@asfgit
Copy link

@asfgit asfgit commented Jun 23, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk8-scala2.12/5648/
Test PASSed (JDK 8 and Scala 2.12).

@asfgit
Copy link

@asfgit asfgit commented Jun 23, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk7-scala2.11/5662/
Test PASSed (JDK 7 and Scala 2.11).

@enothereska enothereska changed the title KAFKA-5157: Options for handling corrupt data during deserialization [WiP] KAFKA-5157: Options for handling corrupt data during deserialization Jun 23, 2017
@enothereska
Copy link
Contributor Author

@enothereska enothereska commented Jun 23, 2017

@mjsax @dguy @bbejeck @guozhangwang this is the implementation for KIP-161. Have a look when you can. Thanks.

@asfgit
Copy link

@asfgit asfgit commented Jun 23, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk7-scala2.11/5667/
Test PASSed (JDK 7 and Scala 2.11).

@asfgit
Copy link

@asfgit asfgit commented Jun 23, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk8-scala2.12/5653/
Test PASSed (JDK 8 and Scala 2.12).

Copy link
Member

@mjsax mjsax left a comment

Thanks for the patch. Some minor comments.

@@ -239,6 +241,13 @@
private static final String STATE_DIR_DOC = "Directory location for state store.";

/**
* {@code default.deserialization.exception.handler}

This comment has been minimized.

@mjsax

mjsax Jun 24, 2017
Member

Nit: Can you please add this in alphabetic order. Thx.

@@ -339,6 +348,11 @@
CommonClientConfigs.DEFAULT_SECURITY_PROTOCOL,
Importance.MEDIUM,
CommonClientConfigs.SECURITY_PROTOCOL_DOC)
.define(DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG,

This comment has been minimized.

@mjsax

mjsax Jun 24, 2017
Member

Nit. As above. Alphabetic within the "medium" group.

@@ -339,6 +348,11 @@
CommonClientConfigs.DEFAULT_SECURITY_PROTOCOL,
Importance.MEDIUM,
CommonClientConfigs.SECURITY_PROTOCOL_DOC)
.define(DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG,
Type.CLASS,
LogAndFailExceptionHandler.class.getName(),

This comment has been minimized.

@mjsax

mjsax Jun 24, 2017
Member

Nit: remove getName()

This comment has been minimized.

@enothereska

enothereska Jun 26, 2017
Author Contributor

They all have getName. Should I remove all? Otherwise it will look inconsistent.

This comment has been minimized.

@mjsax

mjsax Jun 26, 2017
Member

Sounds good to me. Not sure why we started with this annoying .getName() in the first place...

private static final Logger log = LoggerFactory.getLogger(StreamThread.class);

@Override
public DeserializationHandlerResponse handle(ProcessorContext context,

This comment has been minimized.

@mjsax

mjsax Jun 24, 2017
Member

Nit: add final

}

@Override
public void configure(Map<String, ?> configs) {

This comment has been minimized.

@mjsax

mjsax Jun 24, 2017
Member

final

@@ -129,6 +135,55 @@ public void shouldProcessRecordsForOtherTopic() throws Exception {
assertEquals(0, sourceOne.numReceived);
}

@Test(expected = StreamsException.class)

This comment has been minimized.

@mjsax

mjsax Jun 24, 2017
Member

We should not use expected here but try { globalStateTask.update(...); fail(...): } catch(ignore) {} pattern. Also, the ConsumerRecord() should be instantiated outside/before the try-catch blog.

}


@Test(expected = StreamsException.class)

This comment has been minimized.

@mjsax

mjsax Jun 24, 2017
Member

as above

@Test(expected = StreamsException.class)
public void shouldThrowOnNegativeTimestamp() {
final List<ConsumerRecord<byte[], byte[]>> records = Collections.singletonList(
new ConsumerRecord<>("topic", 1, 1, -1L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue));

final RecordQueue queue = new RecordQueue(new TopicPartition(topics[0], 1),
new MockSourceNode<>(topics, intDeserializer, intDeserializer),
new FailOnInvalidTimestamp());
new FailOnInvalidTimestamp(), new LogAndContinueExceptionHandler(), null);

This comment has been minimized.

@mjsax

mjsax Jun 24, 2017
Member

Nit: please one argument per line.

@@ -158,7 +189,7 @@ public void shouldDropOnNegativeTimestamp() {

final RecordQueue queue = new RecordQueue(new TopicPartition(topics[0], 1),
new MockSourceNode<>(topics, intDeserializer, intDeserializer),
new LogAndSkipOnInvalidTimestamp());
new LogAndSkipOnInvalidTimestamp(), new LogAndContinueExceptionHandler(), null);

This comment has been minimized.

@mjsax

mjsax Jun 24, 2017
Member

as above.

@@ -205,7 +206,7 @@ public ProcessorTopologyTestDriver(final StreamsConfig config,
final GlobalStateManagerImpl stateManager = new GlobalStateManagerImpl(globalTopology, globalConsumer, stateDirectory);
globalStateTask = new GlobalStateUpdateTask(globalTopology,
new GlobalProcessorContextImpl(config, stateManager, streamsMetrics, cache),
stateManager
stateManager, new LogAndContinueExceptionHandler()

This comment has been minimized.

@mjsax

mjsax Jun 24, 2017
Member

as above

This comment has been minimized.

@mjsax

mjsax Jul 5, 2017
Member

Guess you missed this one :)

@asfgit
Copy link

@asfgit asfgit commented Jun 26, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk7-scala2.11/5705/
Test PASSed (JDK 7 and Scala 2.11).

@asfgit
Copy link

@asfgit asfgit commented Jun 26, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk8-scala2.12/5691/
Test PASSed (JDK 8 and Scala 2.12).

@asfgit
Copy link

@asfgit asfgit commented Jun 26, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk7-scala2.11/5707/
Test PASSed (JDK 7 and Scala 2.11).

@asfgit
Copy link

@asfgit asfgit commented Jun 26, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk8-scala2.12/5693/
Test PASSed (JDK 8 and Scala 2.12).

@enothereska
Copy link
Contributor Author

@enothereska enothereska commented Jul 3, 2017

@dguy @guozhangwang any further comments? Thanks.

Copy link
Contributor

@dguy dguy left a comment

Thanks @enothereska - i've left a few comments

final Exception exception) {

log.warn("Deserialization exception {}. Processor context is {} and record is {}",
exception.toString(), context, record);

This comment has been minimized.

@dguy

dguy Jul 3, 2017
Contributor

do we want the exception stacktrace?

final ConsumerRecord<byte[], byte[]> record,
final Exception exception) {

log.warn("Deserialization exception {}. Processor context is {} and record is {}",

This comment has been minimized.

@dguy

dguy Jul 3, 2017
Contributor

Same here

} catch (Exception e) {
DeserializationExceptionHandler.DeserializationHandlerResponse response =
deserializationExceptionHandler.handle(processorContext, rawRecord, e);
if (response.id == DeserializationExceptionHandler.DeserializationHandlerResponse.FAIL.id) {

This comment has been minimized.

@dguy

dguy Jul 3, 2017
Contributor

Why not just if(response == DeserializationExceptionHandler.DeserializationHandlerResponse.FAIL) ?

try {
return deserialize(rawRecord);
} catch (Exception e) {
DeserializationExceptionHandler.DeserializationHandlerResponse response =

This comment has been minimized.

@dguy

dguy Jul 3, 2017
Contributor

nit: final

// catch and process if we have a deserialization handler
try {
return deserialize(rawRecord);
} catch (Exception e) {

This comment has been minimized.

@dguy

dguy Jul 3, 2017
Contributor

Should we add a metric for records skipped due to deserialization errors?

This comment has been minimized.

@enothereska

enothereska Jul 4, 2017
Author Contributor

Yes good idea. I'll piggyback on this KIP.

final byte[] key,
final byte[] recordValue,
boolean failExpected) {
ConsumerRecord record = new ConsumerRecord<>("t2", 1, 1,

This comment has been minimized.

@dguy

dguy Jul 3, 2017
Contributor

nit: final

@@ -140,14 +142,45 @@ public void shouldThrowStreamsExceptionWhenValueDeserializationFails() throws Ex
queue.addRawRecords(records);
}

@Test
public void shouldNotThrowStreamsExceptionWhenKeyDeserializationFailsWithSkipHandler() throws Exception {
RecordQueue queue = new RecordQueue(new TopicPartition(topics[0], 1),

This comment has been minimized.

@dguy

dguy Jul 3, 2017
Contributor

maybe extract to a field as it is also used in the test below

public ConsumerRecord<Object, Object> tryDeserialize(final ProcessorContext processorContext,
ConsumerRecord<byte[], byte[]> rawRecord) {

if (deserializationExceptionHandler == null) {

This comment has been minimized.

@dguy

dguy Jul 3, 2017
Contributor

Can we assume and/or ensure that this is non-null? Then we won't need this check here and can just have a single path through this method. I think we default to LogAndFail... if nothing is set - right?

@asfgit
Copy link

@asfgit asfgit commented Jul 4, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk7-scala2.11/5887/
Test PASSed (JDK 7 and Scala 2.11).

@enothereska
Copy link
Contributor Author

@enothereska enothereska commented Jul 4, 2017

Thanks @dguy

@asfgit
Copy link

@asfgit asfgit commented Jul 4, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk7-scala2.11/5889/
Test PASSed (JDK 7 and Scala 2.11).

@asfgit
Copy link

@asfgit asfgit commented Jul 4, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk8-scala2.12/5874/
Test PASSed (JDK 8 and Scala 2.12).

Copy link
Contributor

@dguy dguy left a comment

Thanks @enothereska. I've left one comment. I think the only other thing remaining is whether or not we should implement the pattern for setting state that @guozhangwang suggested

public DeserializationHandlerResponse handle(final ProcessorContext context,
final ConsumerRecord<byte[], byte[]> record,
final Exception exception) {
StringWriter sWriter = new StringWriter();

This comment has been minimized.

@dguy

dguy Jul 4, 2017
Contributor

You can just pass the exception as the last parameter to i.e, log.warn(" p1: {} p2: {}", p1, p2, exception)

Also, just realised that ProcessorContextImpl doesn't have a toString. Perhaps we just need to log the taskId, topic, partition, and offset? i.e.,
log.warn("Exception caught during Deserialization, taskId: {}, topic:{}, partition:{}, offset:{}", context.taskId(), record.topic(), record.partition(), record.offset(), exception)

Same in the other exception handler

This comment has been minimized.

@enothereska

enothereska Jul 4, 2017
Author Contributor

If I pass just the exception, it reverts to exception.toString(). It won't print the stack trace. As a typical NPE example:

  • with just exception passed: java.lang.NullPointerException
  • with stack trace:```
    java.lang.NullPointerException
    at org.rocksdb.RocksDB.get(RocksDB.java:791)
    at org.apache.kafka.streams.state.internals.RocksDBStore.getInternal(RocksDBStore.java:235)
    at org.apache.kafka.streams.state.internals.RocksDBStore.get(RocksDBStore.java:219)
    at org.apache.kafka.streams.state.internals.ChangeLoggingKeyValueBytesStore.delete(ChangeLoggingKeyValueBytesStore.java:80)
    at org.apache.kafka.streams.state.internals.ChangeLoggingKeyValueStore.delete(ChangeLoggingKeyValueStore.java:96)

This comment has been minimized.

@enothereska

enothereska Jul 4, 2017
Author Contributor

Shall we add a toString to ProcessorContextImpl? That sounds useful.

This comment has been minimized.

@dguy

dguy Jul 4, 2017
Contributor

@enothereska

If I pass just the exception, it reverts to exception.toString(). It won't print the stack trace.

My guess is that you are adding a {} for the exception, in which case it probably will just print the string. You don't need to add it. For example, from another one we have:
log.warn("{} Failed offset commits {}: ", logPrefix, consumedOffsetsAndMetadata, e)

This comment has been minimized.

@dguy

dguy Jul 4, 2017
Contributor

Having toString on ProcessorContextImpl might be useful, but it also might be too much information for this particular log message

@enothereska
Copy link
Contributor Author

@enothereska enothereska commented Jul 4, 2017

I've left one comment. I think the only other thing remaining is whether or not we should implement the pattern for setting state.

Thanks @dguy I'm not sure what this refers to. I looked back but I don't see @guozhangwang 's comment. Could you elaborate? Thanks.

@dguy
Copy link
Contributor

@dguy dguy commented Jul 4, 2017

@enothereska doh! I've got my PRs crossed! ignore me

@asfgit
Copy link

@asfgit asfgit commented Jul 5, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk7-scala2.11/5899/
Test PASSed (JDK 7 and Scala 2.11).

Copy link
Contributor

@dguy dguy left a comment

1 more comment otherwise LGTM. @mjsax can you please make another pass over it?

@@ -80,8 +88,14 @@ public TopicPartition partition() {
* @return the size of this queue
*/
public int addRawRecords(Iterable<ConsumerRecord<byte[], byte[]>> rawRecords) {
ConsumerRecord<Object, Object> record = null;

This comment has been minimized.

@dguy

dguy Jul 5, 2017
Contributor

I don't see any reason for this variable to not be declared inside the loop as it was previously

@asfgit
Copy link

@asfgit asfgit commented Jul 5, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk8-scala2.12/5884/
Test PASSed (JDK 8 and Scala 2.12).

@asfgit
Copy link

@asfgit asfgit commented Jul 5, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk7-scala2.11/5903/
Test PASSed (JDK 7 and Scala 2.11).

@asfgit
Copy link

@asfgit asfgit commented Jul 5, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk8-scala2.12/5888/
Test PASSed (JDK 8 and Scala 2.12).

Copy link
Member

@mjsax mjsax left a comment

Some comments.

import org.apache.kafka.common.Configurable;
import org.apache.kafka.streams.processor.ProcessorContext;

public interface DeserializationExceptionHandler extends Configurable {

This comment has been minimized.

@mjsax

mjsax Jul 5, 2017
Member

We should have proper class JavaDocs for all public interfaces.


public interface DeserializationExceptionHandler extends Configurable {
/**
* Inspect a record and the exception received

This comment has been minimized.

@mjsax

mjsax Jul 5, 2017
Member

Nit: . missing at the end

public final short id;

DeserializationHandlerResponse(int id, String name) {
this.id = (short) id;

This comment has been minimized.

@mjsax

mjsax Jul 5, 2017
Member

iIf we go with short, the parameter should be short, too?

/** the permanent and immutable id of an API--this can't change ever */
public final short id;

DeserializationHandlerResponse(int id, String name) {

This comment has been minimized.

@mjsax

mjsax Jul 5, 2017
Member

nit: add final

final Exception exception) {

log.warn("Exception caught during Deserialization, " +
"taskId: {}, topic:{}, partition:{}, offset:{}",

This comment has been minimized.

@mjsax

mjsax Jul 5, 2017
Member

Nit: some whitespaces are missing

final Exception exception) {

log.warn("Exception caught during Deserialization, " +
"taskId: {}, topic:{}, partition:{}, offset:{}",

This comment has been minimized.

@mjsax

mjsax Jul 5, 2017
Member

Nit: fix indention and some whitespaces are missing

@@ -63,7 +56,7 @@ public GlobalStateUpdateTask(final ProcessorTopology topology,
for (final String storeName : storeNames) {
final String sourceTopic = storeNameToTopic.get(storeName);
final SourceNode source = topology.source(sourceTopic);
deserializers.put(sourceTopic, new SourceNodeAndDeserializer(source, new SourceNodeRecordDeserializer(source)));
deserializers.put(sourceTopic, new SourceNodeRecordDeserializer(source, this.deserializationExceptionHandler));

This comment has been minimized.

@mjsax

mjsax Jul 5, 2017
Member

Nit: remove this

final DeserializationExceptionHandler.DeserializationHandlerResponse response =
deserializationExceptionHandler.handle(processorContext, rawRecord, e);
if (response == DeserializationExceptionHandler.DeserializationHandlerResponse.FAIL) {
throw e;

This comment has been minimized.

@mjsax

mjsax Jul 5, 2017
Member

Should we wrap this with a StreamsException? This way, we could add a detailed error message and explain in the msg text that uses can set a different exception handler etc (similar to timestamp extractor?) -- so users don't need to ask at the mailing list :)

@@ -205,7 +206,7 @@ public ProcessorTopologyTestDriver(final StreamsConfig config,
final GlobalStateManagerImpl stateManager = new GlobalStateManagerImpl(globalTopology, globalConsumer, stateDirectory);
globalStateTask = new GlobalStateUpdateTask(globalTopology,
new GlobalProcessorContextImpl(config, stateManager, streamsMetrics, cache),
stateManager
stateManager, new LogAndContinueExceptionHandler()

This comment has been minimized.

@mjsax

mjsax Jul 5, 2017
Member

Guess you missed this one :)

@@ -186,6 +187,8 @@ public NodeMetrics(StreamsMetrics metrics, String name, String sensorNamePrefix)
this.nodeCreationSensor = metrics.addLatencyAndThroughputSensor(scope, sensorNamePrefix + "." + name, "create", Sensor.RecordingLevel.DEBUG, tagKey, tagValue);
this.nodeDestructionSensor = metrics.addLatencyAndThroughputSensor(scope, sensorNamePrefix + "." + name, "destroy", Sensor.RecordingLevel.DEBUG, tagKey, tagValue);
this.sourceNodeForwardSensor = metrics.addThroughputSensor(scope, sensorNamePrefix + "." + name, "forward", Sensor.RecordingLevel.DEBUG, tagKey, tagValue);
this.sourceNodeSkippedDueToDeserializationError = metrics.addThroughputSensor(scope, sensorNamePrefix + "." + name, "skippedDueToDeserializationError", Sensor.RecordingLevel.DEBUG, tagKey, tagValue);

This comment has been minimized.

@mjsax

mjsax Jul 5, 2017
Member

Nit: remove empty line

@asfgit
Copy link

@asfgit asfgit commented Jul 6, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk8-scala2.12/5904/
Test PASSed (JDK 8 and Scala 2.12).

@asfgit
Copy link

@asfgit asfgit commented Jul 6, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk7-scala2.11/5919/
Test PASSed (JDK 7 and Scala 2.11).

@asfgit
Copy link

@asfgit asfgit commented Jul 7, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk8-scala2.12/5943/
Test FAILed (JDK 8 and Scala 2.12).

@asfgit
Copy link

@asfgit asfgit commented Jul 7, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk7-scala2.11/5958/
Test FAILed (JDK 7 and Scala 2.11).

@enothereska
Copy link
Contributor Author

@enothereska enothereska commented Jul 7, 2017

Could not write standard input into: Gradle build daemon

@enothereska
Copy link
Contributor Author

@enothereska enothereska commented Jul 7, 2017

retest this please

@asfgit
Copy link

@asfgit asfgit commented Jul 7, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk7-scala2.11/5959/
Test FAILed (JDK 7 and Scala 2.11).

@enothereska
Copy link
Contributor Author

@enothereska enothereska commented Jul 7, 2017

org.apache.kafka.streams.integration.QueryableStateIntegrationTest.shouldAllowToQueryAfterThreadDied failed. There is already a JIRA for it https://issues.apache.org/jira/browse/KAFKA-5566.

@asfgit
Copy link

@asfgit asfgit commented Jul 7, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk8-scala2.12/5944/
Test FAILed (JDK 8 and Scala 2.12).

@enothereska
Copy link
Contributor Author

@enothereska enothereska commented Jul 7, 2017

@dguy I don't have anything else to add to this. Fixing the QueryableState in separate PR. Thanks. Also I just merged with trunk.

@asfgit
Copy link

@asfgit asfgit commented Jul 7, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk7-scala2.11/5962/
Test PASSed (JDK 7 and Scala 2.11).

@asfgit
Copy link

@asfgit asfgit commented Jul 7, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk8-scala2.12/5947/
Test PASSed (JDK 8 and Scala 2.12).

@enothereska
Copy link
Contributor Author

@enothereska enothereska commented Jul 10, 2017

@guozhangwang any more comments?

@dguy
Copy link
Contributor

@dguy dguy commented Jul 10, 2017

retest this please

@asfgit
Copy link

@asfgit asfgit commented Jul 10, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk7-scala2.11/6015/
Test PASSed (JDK 7 and Scala 2.11).

@asfgit
Copy link

@asfgit asfgit commented Jul 10, 2017

Refer to this link for build results (access rights to CI server needed):
https://builds.apache.org/job/kafka-pr-jdk8-scala2.12/6000/
Test PASSed (JDK 8 and Scala 2.12).

@dguy
dguy approved these changes Jul 10, 2017
Copy link
Contributor

@dguy dguy left a comment

Thanks @enothereska, LGTM

@asfgit asfgit closed this in a1f97c8 Jul 10, 2017
Copy link
Contributor

@guozhangwang guozhangwang left a comment

Left some comments on this PR. Sorry for the late reply.

sourceNode.nodeMetrics.sourceNodeSkippedDueToDeserializationError.record();
}
}
return null;

This comment has been minimized.

@guozhangwang

guozhangwang Jul 10, 2017
Contributor

We should return null in the else branch above, in case we add more options in the response that are beyond Fail or Continue.

@@ -186,6 +187,7 @@ public NodeMetrics(StreamsMetrics metrics, String name, String sensorNamePrefix)
this.nodeCreationSensor = metrics.addLatencyAndThroughputSensor(scope, sensorNamePrefix + "." + name, "create", Sensor.RecordingLevel.DEBUG, tagKey, tagValue);
this.nodeDestructionSensor = metrics.addLatencyAndThroughputSensor(scope, sensorNamePrefix + "." + name, "destroy", Sensor.RecordingLevel.DEBUG, tagKey, tagValue);
this.sourceNodeForwardSensor = metrics.addThroughputSensor(scope, sensorNamePrefix + "." + name, "forward", Sensor.RecordingLevel.DEBUG, tagKey, tagValue);
this.sourceNodeSkippedDueToDeserializationError = metrics.addThroughputSensor(scope, sensorNamePrefix + "." + name, "skippedDueToDeserializationError", Sensor.RecordingLevel.DEBUG, tagKey, tagValue);

This comment has been minimized.

@guozhangwang

guozhangwang Jul 10, 2017
Contributor

As this PR gets in already, we probably need to rebase the metrics fix PR to remove the . + name as well for this new metric as well.

This comment has been minimized.

@guozhangwang

guozhangwang Jul 10, 2017
Contributor

Also I'm wondering what's the right approach of this metric: we already have a thread-level skippedRecordsSensor that covers this as it records total fetched records - total enqueued records. And then we have this source-node level metrics that only tracks the skippedRecordsDuetoDeserError. On the other hand, on the thread-level we do not have any other throughput metrics except the skippedRecordsSensor.

So should we just have a higher-hierarchy sensor at the thread level for number of skipped records rates (for whatever reason), or do we want to have a lower source-node level sensor for skipped records rates of different causes? In either way I think the current hierarchy is a bit messed and should be fixed. WDYT @dguy @enothereska ?

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Linked issues

Successfully merging this pull request may close these issues.

None yet

5 participants
You can’t perform that action at this time.