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-4393: Improve invalid/negative TS handling #2117

Closed
wants to merge 8 commits into from
Closed
Show file tree
Hide file tree
Changes from 7 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
2 changes: 1 addition & 1 deletion checkstyle/import-control.xml
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@
<allow pkg="javax.management" />
<allow pkg="org.slf4j" />
<allow pkg="org.junit" />
<allow pkg="org.hamcrest" />
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is this absolutely needed? cc @ijuma

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

IMO - it is useful and i don't see why it should not be allowed.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'm OK with it although @junrao seemed less convinced last time the subject came up.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I did talk to @ijuma and @guozhangwang and both told me they are ok with the change. Why no allowing hamcrest for tests?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

As long as the usage is valid, this is fine for me.

<allow pkg="org.easymock" />
<allow pkg="org.powermock" />
<allow pkg="java.security" />
Expand Down Expand Up @@ -149,7 +150,6 @@
<allow pkg="scala" />
<allow pkg="scala.collection" />
<allow pkg="org.I0Itec.zkclient" />
<allow pkg="org.hamcrest" />
</subpackage>

<subpackage name="state">
Expand Down
13 changes: 13 additions & 0 deletions docs/upgrade.html
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,19 @@
limitations under the License.
-->

<h4><a id="upgrade_10_2" href="#upgrade_10_2">Upgrading from 0.8.x, 0.9.x, 0.10.0.X, or 0.10.1.X to 0.10.2.0</a></h4>

<p><b>For a rolling upgrade:</b></p>

<ol>
<li>Upgrading a Kafka Streams Applications:
<ul>
<li>You need to recompile your code. Just swapping the jar file will not work and will break your appliation.</li>
<li>If you use a custom timestamp extractor, you will need to update this code, because the <code>TimestampExtractor</code> interface got changed.</li>
</ul>
</li>
</ol>

<h4><a id="upgrade_10_1" href="#upgrade_10_1">Upgrading from 0.8.x, 0.9.x or 0.10.0.X to 0.10.1.0</a></h4>
0.10.1.0 has wire protocol changes. By following the recommended rolling upgrade plan below, you guarantee no downtime during the upgrade.
However, please notice the <a href="#upgrade_10_1_breaking">Potential breaking changes in 0.10.1.0</a> before upgrade.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,7 @@
public class JsonTimestampExtractor implements TimestampExtractor {

@Override
public long extract(ConsumerRecord<Object, Object> record) {
public long extract(final ConsumerRecord<Object, Object> record, final long previousTimestamp) {
if (record.value() instanceof PageViewTypedDemo.PageView) {
return ((PageViewTypedDemo.PageView) record.value()).timestamp;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,7 @@
import org.apache.kafka.common.serialization.Serde;
import org.apache.kafka.common.serialization.Serdes;
import org.apache.kafka.streams.errors.StreamsException;
import org.apache.kafka.streams.processor.ConsumerRecordTimestampExtractor;
import org.apache.kafka.streams.processor.FailOnInvalidTimestamp;
import org.apache.kafka.streams.processor.DefaultPartitionGrouper;
import org.apache.kafka.streams.processor.internals.StreamPartitionAssignor;
import org.apache.kafka.streams.processor.internals.StreamThread;
Expand Down Expand Up @@ -172,7 +172,7 @@ public class StreamsConfig extends AbstractConfig {
REPLICATION_FACTOR_DOC)
.define(TIMESTAMP_EXTRACTOR_CLASS_CONFIG,
Type.CLASS,
ConsumerRecordTimestampExtractor.class.getName(),
FailOnInvalidTimestamp.class.getName(),
Importance.MEDIUM,
TIMESTAMP_EXTRACTOR_CLASS_DOC)
.define(PARTITION_GROUPER_CLASS_CONFIG,
Expand Down

This file was deleted.

Original file line number Diff line number Diff line change
@@ -0,0 +1,77 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You 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 org.apache.kafka.streams.processor;

import org.apache.kafka.clients.consumer.ConsumerRecord;

/**
* Retrieves embedded metadata timestamps from Kafka messages.
* If a record has a negative (invalid) timestamp value, an error handler method is called.
* <p>
* Embedded metadata timestamp was introduced in "KIP-32: Add timestamps to Kafka message" for the new
* 0.10+ Kafka message format.
* <p>
* Here, "embedded metadata" refers to the fact that compatible Kafka producer clients automatically and
* transparently embed such timestamps into message metadata they send to Kafka, which can then be retrieved
* via this timestamp extractor.
* <p>
* If the embedded metadata timestamp represents <i>CreateTime</i> (cf. Kafka broker setting
* {@code message.timestamp.type} and Kafka topic setting {@code log.message.timestamp.type}),
* this extractor effectively provides <i>event-time</i> semantics.
* If <i>LogAppendTime</i> is used as broker/topic setting to define the embedded metadata timestamps,
* using this extractor effectively provides <i>ingestion-time</i> semantics.
* <p>
* If you need <i>processing-time</i> semantics, use {@link WallclockTimestampExtractor}.
*
* @see FailOnInvalidTimestamp
* @see LogAndSkipOnInvalidTimestamp
* @see UsePreviousTimeOnInvalidTimestamp
* @see WallclockTimestampExtractor
*/
abstract class ExtractRecordMetadataTimestamp implements TimestampExtractor {

/**
* Extracts the embedded metadata timestamp from the given {@link ConsumerRecord}.
*
* @param record a data record
* @param previousTimestamp the latest extracted valid timestamp of the current record's partition˙ (could be -1 if unknown)
* @return the embedded metadata timestamp of the given {@link ConsumerRecord}
*/
@Override
public long extract(final ConsumerRecord<Object, Object> record, final long previousTimestamp) {
final long timestamp = record.timestamp();

if (timestamp < 0) {
return onInvalidTimestamp(record, timestamp, previousTimestamp);
}

return timestamp;
}

/**
* Called if no valid timestamp is embedded in the record meta data.
*
* @param record a data record
* @param recordTimestamp the timestamp extractor from the record
* @param previousTimestamp the latest extracted valid timestamp of the current record's partition˙ (could be -1 if unknown)
* @return a new timestamp for the record (if negative, record will not be processed but dropped silently)
*/
public abstract long onInvalidTimestamp(final ConsumerRecord<Object, Object> record,
final long recordTimestamp,
final long previousTimestamp);
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,68 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You 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 org.apache.kafka.streams.processor;

import org.apache.kafka.clients.consumer.ConsumerRecord;
import org.apache.kafka.streams.errors.StreamsException;

/**
* Retrieves embedded metadata timestamps from Kafka messages.
* If a record has a negative (invalid) timestamp value, this extractor raises an exception.
* <p>
* Embedded metadata timestamp was introduced in "KIP-32: Add timestamps to Kafka message" for the new
* 0.10+ Kafka message format.
* <p>
* Here, "embedded metadata" refers to the fact that compatible Kafka producer clients automatically and
* transparently embed such timestamps into message metadata they send to Kafka, which can then be retrieved
* via this timestamp extractor.
* <p>
* If the embedded metadata timestamp represents <i>CreateTime</i> (cf. Kafka broker setting
* {@code message.timestamp.type} and Kafka topic setting {@code log.message.timestamp.type}),
* this extractor effectively provides <i>event-time</i> semantics.
* If <i>LogAppendTime</i> is used as broker/topic setting to define the embedded metadata timestamps,
* using this extractor effectively provides <i>ingestion-time</i> semantics.
* <p>
* If you need <i>processing-time</i> semantics, use {@link WallclockTimestampExtractor}.
*
* @see LogAndSkipOnInvalidTimestamp
* @see UsePreviousTimeOnInvalidTimestamp
* @see WallclockTimestampExtractor
*/
public class FailOnInvalidTimestamp extends ExtractRecordMetadataTimestamp {

/**
* Raises an exception on every call.
*
* @param record a data record
* @param recordTimestamp the timestamp extractor from the record
* @param previousTimestamp the latest extracted valid timestamp of the current record's partition˙ (could be -1 if unknown)
* @return nothing; always raises an exception
* @throws StreamsException on every invocation
*/
@Override
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.");
}

}
Original file line number Diff line number Diff line change
@@ -0,0 +1,69 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You 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 org.apache.kafka.streams.processor;

import org.apache.kafka.clients.consumer.ConsumerRecord;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

/**
* Retrieves embedded metadata timestamps from Kafka messages.
* If a record has a negative (invalid) timestamp value the timestamp is returned as-is;
* in addition, a WARN message is logged in your application.
* Returning the timestamp as-is results in dropping the record, i.e., the record will not be processed.
* <p>
* Embedded metadata timestamp was introduced in "KIP-32: Add timestamps to Kafka message" for the new
* 0.10+ Kafka message format.
* <p>
* Here, "embedded metadata" refers to the fact that compatible Kafka producer clients automatically and
* transparently embed such timestamps into message metadata they send to Kafka, which can then be retrieved
* via this timestamp extractor.
* <p>
* If the embedded metadata timestamp represents <i>CreateTime</i> (cf. Kafka broker setting
* {@code message.timestamp.type} and Kafka topic setting {@code log.message.timestamp.type}),
* this extractor effectively provides <i>event-time</i> semantics.
* If <i>LogAppendTime</i> is used as broker/topic setting to define the embedded metadata timestamps,
* using this extractor effectively provides <i>ingestion-time</i> semantics.
* <p>
* If you need <i>processing-time</i> semantics, use {@link WallclockTimestampExtractor}.
*
* @see FailOnInvalidTimestamp
* @see UsePreviousTimeOnInvalidTimestamp
* @see WallclockTimestampExtractor
*/
public class LogAndSkipOnInvalidTimestamp extends ExtractRecordMetadataTimestamp {
private static final Logger log = LoggerFactory.getLogger(LogAndSkipOnInvalidTimestamp.class);

/**
* Writes a log WARN message when the extracted timestamp is invalid (negative) but returns the invalid timestamp as-is,
* which ultimately causes the record to be skipped and not to be processed.
*
* @param record a data record
* @param recordTimestamp the timestamp extractor from the record
* @param previousTimestamp the latest extracted valid timestamp of the current record's partition˙ (could be -1 if unknown)
* @return the originally extracted timestamp of the record
*/
@Override
public long onInvalidTimestamp(final ConsumerRecord<Object, Object> record,
final long recordTimestamp,
final long previousTimestamp) {
log.warn("Input record {} will be dropped because it has an invalid (negative) timestamp.", record);
return recordTimestamp;
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -27,17 +27,19 @@
public interface TimestampExtractor {

/**
* Extracts a timestamp from a record.
* Extracts a timestamp from a record. The timestamp must be positive to be considered a valid timestamp.
* Returning a negative timestamp will cause the record not to be processed but rather silently skipped.
* <p>
* The extracted timestamp MUST represent the milliseconds since midnight, January 1, 1970 UTC.
*
* <p>
* It is important to note that this timestamp may become the message timestamp for any messages sent to changelogs updated by {@link KTable}s
* and joins. The message timestamp is used for log retention and log rolling, so using nonsensical values may result in
* excessive log rolling and therefore broker performance degradation.
*
*
* @param record a data record
* @return the timestamp of the record
* @param record a data record
* @param previousTimestamp the latest extracted valid timestamp of the current record's partition˙ (could be -1 if unknown)
* @return the timestamp of the record
*/
long extract(ConsumerRecord<Object, Object> record);
long extract(ConsumerRecord<Object, Object> record, long previousTimestamp);
}
Loading