Skip to content

Commit ad3e34d

Browse files
committed
HOTFIX: fix broken commit
Fixes broken commit for KAFKA-17872
1 parent 587c3cf commit ad3e34d

File tree

1 file changed

+4
-7
lines changed

1 file changed

+4
-7
lines changed

streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java

Lines changed: 4 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -167,7 +167,7 @@ public class StreamTaskTest {
167167

168168
private final MockSourceNode<Integer, Integer> source1 = new MockSourceNode<>(intDeserializer, intDeserializer);
169169
private final MockSourceNode<Integer, Integer> source2 = new MockSourceNode<>(intDeserializer, intDeserializer);
170-
private final MockSourceNode<Integer, Integer> source3 = new MockSourceNode<>(intDeserializer, intDeserializer) {
170+
private final MockSourceNode<Integer, Integer> source3 = new MockSourceNode<Integer, Integer>(intDeserializer, intDeserializer) {
171171
@Override
172172
public void process(final Record<Integer, Integer> record) {
173173
throw new RuntimeException("KABOOM!");
@@ -178,7 +178,7 @@ public void close() {
178178
throw new RuntimeException("KABOOM!");
179179
}
180180
};
181-
private final MockSourceNode<Integer, Integer> timeoutSource = new MockSourceNode<>(intDeserializer, intDeserializer) {
181+
private final MockSourceNode<Integer, Integer> timeoutSource = new MockSourceNode<Integer, Integer>(intDeserializer, intDeserializer) {
182182
@Override
183183
public void process(final Record<Integer, Integer> record) {
184184
throw new TimeoutException("Kaboom!");
@@ -442,7 +442,7 @@ public void shouldAutoOffsetResetIfNoCommittedOffsetFound() {
442442
task.addPartitionsForOffsetReset(Collections.singleton(partition1));
443443

444444
final AtomicReference<AssertionError> shouldNotSeek = new AtomicReference<>();
445-
try (final MockConsumer<byte[], byte[]> consumer = new MockConsumer<>(OffsetResetStrategy.EARLIEST) {
445+
try (final MockConsumer<byte[], byte[]> consumer = new MockConsumer<byte[], byte[]>(OffsetResetStrategy.EARLIEST) {
446446
@Override
447447
public void seek(final TopicPartition partition, final long offset) {
448448
final AssertionError error = shouldNotSeek.get();
@@ -2203,6 +2203,7 @@ public void shouldCheckpointOffsetsOnPostCommit() {
22032203
when(stateManager.taskId()).thenReturn(taskId);
22042204
when(stateManager.taskType()).thenReturn(TaskType.ACTIVE);
22052205
final long offset = 543L;
2206+
final long consumedOffset = 345L;
22062207

22072208
when(recordCollector.offsets()).thenReturn(singletonMap(changelogPartition, offset));
22082209
when(stateManager.changelogOffsets())
@@ -2600,7 +2601,6 @@ public void shouldUpdateOffsetIfAllRecordsHaveInvalidTimestamp() {
26002601
task.resumePollingForPartitionsWithAvailableSpace();
26012602
consumer.poll(Duration.ZERO);
26022603
task.addRecords(partition1, records);
2603-
task.updateNextOffsets(partition1, new OffsetAndMetadata(offset + 1, Optional.empty(), ""));
26042604
task.updateLags();
26052605

26062606
assertTrue(task.process(offset));
@@ -2632,7 +2632,6 @@ public void shouldUpdateOffsetIfValidRecordFollowsInvalidTimestamp() {
26322632
task.resumePollingForPartitionsWithAvailableSpace();
26332633
consumer.poll(Duration.ZERO);
26342634
task.addRecords(partition1, records);
2635-
task.updateNextOffsets(partition1, new OffsetAndMetadata(offset + 1, Optional.empty(), ""));
26362635
task.updateLags();
26372636

26382637
assertTrue(task.process(offset));
@@ -2661,8 +2660,6 @@ public void shouldUpdateOffsetIfInvalidTimestampeRecordFollowsValid() {
26612660
task.resumePollingForPartitionsWithAvailableSpace();
26622661
consumer.poll(Duration.ZERO);
26632662
task.addRecords(partition1, records);
2664-
task.updateNextOffsets(partition1, new OffsetAndMetadata(offset + 1, Optional.empty(), ""));
2665-
26662663
task.updateLags();
26672664

26682665
assertTrue(task.process(offset));

0 commit comments

Comments
 (0)