Skip to content
Merged
Show file tree
Hide file tree
Changes from all 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
Original file line number Diff line number Diff line change
Expand Up @@ -69,7 +69,6 @@ public void assign(Set<StreamPartition<Integer>> streamPartitions)
.stream()
.map(x -> new TopicPartition(x.getStream(), x.getPartitionId()))
.collect(Collectors.toSet()));
seekToEarliest(streamPartitions);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2221,6 +2221,69 @@ public void testRunTransactionModeRollback() throws Exception
Assert.assertEquals(ImmutableList.of("f"), readSegmentColumn("dim1", desc4));
}

@Test(timeout = 60_000L)
public void testCanStartFromLaterThanEarliestOffset() throws Exception
{
if (!isIncrementalHandoffSupported) {
return;
}
final String baseSequenceName = "sequence0";
maxRowsPerSegment = Integer.MAX_VALUE;
maxTotalRows = null;

// Insert data
int numToAdd = records.size();

try (final KafkaProducer<byte[], byte[]> kafkaProducer = kafkaServer.newProducer()) {
kafkaProducer.initTransactions();
kafkaProducer.beginTransaction();
for (ProducerRecord<byte[], byte[]> record : records) {
kafkaProducer.send(record).get();
}
kafkaProducer.commitTransaction();
}

Map<String, Object> consumerProps = kafkaServer.consumerProperties();
consumerProps.put("max.poll.records", "1");

final SeekableStreamPartitions<Integer, Long> startPartitions = new SeekableStreamPartitions<>(
topic,
ImmutableMap.of(
0,
0L,
1,
1L
)
);

final SeekableStreamPartitions<Integer, Long> endPartitions = new SeekableStreamPartitions<>(
topic,
ImmutableMap.of(
0,
10L,
1,
2L
)
);

final KafkaIndexTask task = createTask(
null,
new KafkaIndexTaskIOConfig(
0,
baseSequenceName,
startPartitions,
endPartitions,
consumerProps,
KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS,
true,
null,
null
)
);
final ListenableFuture<TaskStatus> future = runTask(task);
Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode());
}

private List<ScanResultValue> scanData(final Task task, QuerySegmentSpec spec)
{
ScanQuery query = new Druids.ScanQueryBuilder().dataSource(
Expand Down