Skip to content

Commit

Permalink
KAFKA-5980: FailOnInvalidTimestamp does not log error
Browse files Browse the repository at this point in the history
Author: Matthias J. Sax <[email protected]>

Reviewers: Damian Guy <[email protected]>, Ted Yu <[email protected]>, Denis Bolshakov

Closes apache#3966 from mjsax/kafka-5980-FailOnInvalidTimestamp-does-not-log-error
  • Loading branch information
mjsax authored and guozhangwang committed Oct 4, 2017
1 parent 713a67f commit 5792f2f
Showing 1 changed file with 11 additions and 4 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,8 @@
import org.apache.kafka.clients.consumer.ConsumerRecord;
import org.apache.kafka.common.annotation.InterfaceStability;
import org.apache.kafka.streams.errors.StreamsException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

/**
* Retrieves embedded metadata timestamps from Kafka messages.
Expand All @@ -45,6 +47,7 @@
*/
@InterfaceStability.Evolving
public class FailOnInvalidTimestamp extends ExtractRecordMetadataTimestamp {
private static final Logger log = LoggerFactory.getLogger(FailOnInvalidTimestamp.class);

/**
* Raises an exception on every call.
Expand All @@ -60,10 +63,14 @@ public long onInvalidTimestamp(final ConsumerRecord<Object, Object> record,
final long recordTimestamp,
final long previousTimestamp)
throws StreamsException {
throw new StreamsException("Input record " + record + " has invalid (negative) timestamp. " +
"Possibly because a pre-0.10 producer client was used to write this record to Kafka without embedding a timestamp, " +
"or because the input topic was created before upgrading the Kafka cluster to 0.10+. " +
"Use a different TimestampExtractor to process this data.");

final String message = "Input record " + record + " has invalid (negative) timestamp. " +
"Possibly because a pre-0.10 producer client was used to write this record to Kafka without embedding " +
"a timestamp, or because the input topic was created before upgrading the Kafka cluster to 0.10+. " +
"Use a different TimestampExtractor to process this data.";

log.error(message);
throw new StreamsException(message);
}

}

0 comments on commit 5792f2f

Please sign in to comment.