mirror of https://github.com/apache/kafka.git
KAFKA-8953: Rename UsePreviousTimeOnInvalidTimestamp to UsePartitionTimeOnInvalidTimestamp (#7633)
Implements KIP-530 Reviewer: Matthias J. Sax <matthias@confluent.io>
This commit is contained in:
parent
5d0cb1419c
commit
deafc56fed
|
@ -511,7 +511,7 @@
|
|||
silently drop the record.
|
||||
This log-and-skip strategy allows Kafka Streams to make progress instead of failing if there are records with an
|
||||
invalid built-in timestamp in your input data.</li>
|
||||
<li><a class="reference external" href="/{{version}}/javadoc/org/apache/kafka/streams/processor/UsePreviousTimeOnInvalidTimestamp.html">UsePreviousTimeOnInvalidTimestamp</a>.
|
||||
<li><a class="reference external" href="/{{version}}/javadoc/org/apache/kafka/streams/processor/UsePartitionTimeOnInvalidTimestamp.html">UsePartitionTimeOnInvalidTimestamp</a>.
|
||||
This extractor returns the record’s built-in timestamp if it is valid (i.e. not negative). If the record does not
|
||||
have a valid built-in timestamps, the extractor returns the previously extracted valid timestamp from a record of the
|
||||
same topic partition as the current record as a timestamp estimation. In case that no timestamp can be estimated, it
|
||||
|
|
|
@ -72,6 +72,12 @@
|
|||
More details about the new config <code>StreamsConfig#TOPOLOGY_OPTIMIZATION</code> can be found in <a href="https://cwiki.apache.org/confluence/display/KAFKA/KIP-295%3A+Add+Streams+Configuration+Allowing+for+Optional+Topology+Optimization">KIP-295</a>.
|
||||
</p>
|
||||
|
||||
<h3><a id="streams_api_changes_250" href="#streams_api_changes_250">Streams API changes in 2.5.0</a></h3>
|
||||
<p>
|
||||
As of 2.5.0 Kafka we deprecated <code>UsePreviousTimeOnInvalidTimestamp</code> and replaced it with <code>UsePartitionTimeOnInvalidTimeStamp</code> as per
|
||||
<a href="https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=130028807">KIP-530</a>
|
||||
</p>
|
||||
|
||||
<h3><a id="streams_api_changes_240" href="#streams_api_changes_240">Streams API changes in 2.4.0</a></h3>
|
||||
<p>
|
||||
As of 2.4.0 Kafka Streams offers a KTable-KTable foreign-key join (as per <a href="https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+Support+non-key+joining+in+KTable">KIP-213</a>).
|
||||
|
|
|
@ -39,7 +39,7 @@ import org.apache.kafka.clients.consumer.ConsumerRecord;
|
|||
*
|
||||
* @see FailOnInvalidTimestamp
|
||||
* @see LogAndSkipOnInvalidTimestamp
|
||||
* @see UsePreviousTimeOnInvalidTimestamp
|
||||
* @see UsePartitionTimeOnInvalidTimestamp
|
||||
* @see WallclockTimestampExtractor
|
||||
*/
|
||||
abstract class ExtractRecordMetadataTimestamp implements TimestampExtractor {
|
||||
|
|
|
@ -41,7 +41,7 @@ import org.slf4j.LoggerFactory;
|
|||
* If you need <i>processing-time</i> semantics, use {@link WallclockTimestampExtractor}.
|
||||
*
|
||||
* @see LogAndSkipOnInvalidTimestamp
|
||||
* @see UsePreviousTimeOnInvalidTimestamp
|
||||
* @see UsePartitionTimeOnInvalidTimestamp
|
||||
* @see WallclockTimestampExtractor
|
||||
*/
|
||||
public class FailOnInvalidTimestamp extends ExtractRecordMetadataTimestamp {
|
||||
|
|
|
@ -42,7 +42,7 @@ import org.slf4j.LoggerFactory;
|
|||
* If you need <i>processing-time</i> semantics, use {@link WallclockTimestampExtractor}.
|
||||
*
|
||||
* @see FailOnInvalidTimestamp
|
||||
* @see UsePreviousTimeOnInvalidTimestamp
|
||||
* @see UsePartitionTimeOnInvalidTimestamp
|
||||
* @see WallclockTimestampExtractor
|
||||
*/
|
||||
public class LogAndSkipOnInvalidTimestamp extends ExtractRecordMetadataTimestamp {
|
||||
|
|
|
@ -0,0 +1,67 @@
|
|||
/*
|
||||
* 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, a new timestamp will be inferred from the current stream-time.
|
||||
* <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 WallclockTimestampExtractor
|
||||
*/
|
||||
|
||||
public class UsePartitionTimeOnInvalidTimestamp extends ExtractRecordMetadataTimestamp {
|
||||
/**
|
||||
* Returns the current stream-time as new timestamp for the record.
|
||||
*
|
||||
* @param record a data record
|
||||
* @param recordTimestamp the timestamp extractor from the record
|
||||
* @param partitionTime the highest extracted valid timestamp of the current record's partition˙ (could be -1 if unknown)
|
||||
* @return the provided highest extracted valid timestamp as new timestamp for the record
|
||||
* @throws StreamsException if highest extracted valid timestamp is unknown
|
||||
*/
|
||||
@Override
|
||||
public long onInvalidTimestamp(final ConsumerRecord<Object, Object> record,
|
||||
final long recordTimestamp,
|
||||
final long partitionTime)
|
||||
throws StreamsException {
|
||||
if (partitionTime < 0) {
|
||||
throw new StreamsException("Could not infer new timestamp for input record " + record
|
||||
+ " because partition time is unknown.");
|
||||
}
|
||||
return partitionTime;
|
||||
}
|
||||
}
|
|
@ -22,7 +22,7 @@ import org.apache.kafka.streams.errors.StreamsException;
|
|||
/**
|
||||
* Retrieves embedded metadata timestamps from Kafka messages.
|
||||
* If a record has a negative (invalid) timestamp, a new timestamp will be inferred from the current stream-time.
|
||||
* <p></p>
|
||||
* <p>
|
||||
* Embedded metadata timestamp was introduced in "KIP-32: Add timestamps to Kafka message" for the new
|
||||
* 0.10+ Kafka message format.
|
||||
* <p>
|
||||
|
@ -37,11 +37,13 @@ import org.apache.kafka.streams.errors.StreamsException;
|
|||
* using this extractor effectively provides <i>ingestion-time</i> semantics.
|
||||
* <p>
|
||||
* If you need <i>processing-time</i> semantics, use {@link WallclockTimestampExtractor}.
|
||||
*
|
||||
* @deprecated since 2.5. Use {@link UsePartitionTimeOnInvalidTimestamp} instead
|
||||
* @see FailOnInvalidTimestamp
|
||||
* @see LogAndSkipOnInvalidTimestamp
|
||||
* @see WallclockTimestampExtractor
|
||||
*/
|
||||
@SuppressWarnings("deprecation")
|
||||
@Deprecated
|
||||
public class UsePreviousTimeOnInvalidTimestamp extends ExtractRecordMetadataTimestamp {
|
||||
|
||||
/**
|
||||
|
|
|
@ -28,7 +28,7 @@ import org.apache.kafka.clients.consumer.ConsumerRecord;
|
|||
*
|
||||
* @see FailOnInvalidTimestamp
|
||||
* @see LogAndSkipOnInvalidTimestamp
|
||||
* @see UsePreviousTimeOnInvalidTimestamp
|
||||
* @see UsePartitionTimeOnInvalidTimestamp
|
||||
*/
|
||||
public class WallclockTimestampExtractor implements TimestampExtractor {
|
||||
|
||||
|
|
|
@ -0,0 +1,56 @@
|
|||
/*
|
||||
* 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;
|
||||
import org.junit.Test;
|
||||
|
||||
import static org.hamcrest.CoreMatchers.is;
|
||||
import static org.hamcrest.MatcherAssert.assertThat;
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
public class UsePartitionTimeOnInvalidTimestampTest extends TimestampExtractorTest {
|
||||
|
||||
@Test
|
||||
public void extractMetadataTimestamp() {
|
||||
testExtractMetadataTimestamp(new UsePartitionTimeOnInvalidTimestamp());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void usePartitionTimeOnInvalidTimestamp() {
|
||||
final long partitionTime = 42;
|
||||
|
||||
final TimestampExtractor extractor = new UsePartitionTimeOnInvalidTimestamp();
|
||||
final long timestamp = extractor.extract(
|
||||
new ConsumerRecord<>("anyTopic", 0, 0, null, null),
|
||||
partitionTime
|
||||
);
|
||||
|
||||
assertThat(timestamp, is(partitionTime));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldThrowStreamsException() {
|
||||
final TimestampExtractor extractor = new UsePartitionTimeOnInvalidTimestamp();
|
||||
final ConsumerRecord<Object, Object> record = new ConsumerRecord<>("anyTopic", 0, 0, null, null);
|
||||
try {
|
||||
extractor.extract(record, -1);
|
||||
fail("should have thrown StreamsException");
|
||||
} catch (final StreamsException expected) { }
|
||||
}
|
||||
}
|
|
@ -24,8 +24,8 @@ import static org.hamcrest.CoreMatchers.is;
|
|||
import static org.hamcrest.MatcherAssert.assertThat;
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
@SuppressWarnings("deprecation")
|
||||
public class UsePreviousTimeOnInvalidTimestampTest extends TimestampExtractorTest {
|
||||
|
||||
@Test
|
||||
public void extractMetadataTimestamp() {
|
||||
testExtractMetadataTimestamp(new UsePreviousTimeOnInvalidTimestamp());
|
||||
|
|
Loading…
Reference in New Issue