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

Convert some classes to record in Kafka #21646

Merged
merged 6 commits into from
Apr 22, 2024
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 @@ -89,7 +89,7 @@ public KafkaFilteringResult getKafkaFilterResult(
requireNonNull(partitionBeginOffsets, "partitionBeginOffsets is null");
requireNonNull(partitionEndOffsets, "partitionEndOffsets is null");

TupleDomain<ColumnHandle> constraint = kafkaTableHandle.getConstraint();
TupleDomain<ColumnHandle> constraint = kafkaTableHandle.constraint();
verify(!constraint.isNone(), "constraint is none");

if (!constraint.isAll()) {
Expand All @@ -113,23 +113,23 @@ public KafkaFilteringResult getKafkaFilterResult(
if (offsetRanged.isPresent()) {
Range range = offsetRanged.get();
partitionBeginOffsets = overridePartitionBeginOffsets(partitionBeginOffsets,
partition -> (range.getBegin() != INVALID_KAFKA_RANGE_INDEX) ? Optional.of(range.getBegin()) : Optional.empty());
partition -> (range.begin() != INVALID_KAFKA_RANGE_INDEX) ? Optional.of(range.begin()) : Optional.empty());
partitionEndOffsets = overridePartitionEndOffsets(partitionEndOffsets,
partition -> (range.getEnd() != INVALID_KAFKA_RANGE_INDEX) ? Optional.of(range.getEnd()) : Optional.empty());
partition -> (range.end() != INVALID_KAFKA_RANGE_INDEX) ? Optional.of(range.end()) : Optional.empty());
}

// push down timestamp if possible
if (offsetTimestampRanged.isPresent()) {
try (KafkaConsumer<byte[], byte[]> kafkaConsumer = consumerFactory.create(session)) {
// filter negative value to avoid java.lang.IllegalArgumentException when using KafkaConsumer offsetsForTimes
if (offsetTimestampRanged.get().getBegin() > INVALID_KAFKA_RANGE_INDEX) {
if (offsetTimestampRanged.get().begin() > INVALID_KAFKA_RANGE_INDEX) {
partitionBeginOffsets = overridePartitionBeginOffsets(partitionBeginOffsets,
partition -> findOffsetsForTimestampGreaterOrEqual(kafkaConsumer, partition, offsetTimestampRanged.get().getBegin()));
partition -> findOffsetsForTimestampGreaterOrEqual(kafkaConsumer, partition, offsetTimestampRanged.get().begin()));
}
if (isTimestampUpperBoundPushdownEnabled(session, kafkaTableHandle.getTopicName())) {
if (offsetTimestampRanged.get().getEnd() > INVALID_KAFKA_RANGE_INDEX) {
if (isTimestampUpperBoundPushdownEnabled(session, kafkaTableHandle.topicName())) {
if (offsetTimestampRanged.get().end() > INVALID_KAFKA_RANGE_INDEX) {
partitionEndOffsets = overridePartitionEndOffsets(partitionEndOffsets,
partition -> findOffsetsForTimestampGreaterOrEqual(kafkaConsumer, partition, offsetTimestampRanged.get().getEnd()));
partition -> findOffsetsForTimestampGreaterOrEqual(kafkaConsumer, partition, offsetTimestampRanged.get().end()));
}
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,33 +21,17 @@
import java.util.List;
import java.util.Map;

public class KafkaFilteringResult
{
private final List<PartitionInfo> partitionInfos;
private final Map<TopicPartition, Long> partitionBeginOffsets;
private final Map<TopicPartition, Long> partitionEndOffsets;

public KafkaFilteringResult(List<PartitionInfo> partitionInfos,
Map<TopicPartition, Long> partitionBeginOffsets,
Map<TopicPartition, Long> partitionEndOffsets)
{
this.partitionInfos = ImmutableList.copyOf(partitionInfos);
this.partitionBeginOffsets = ImmutableMap.copyOf(partitionBeginOffsets);
this.partitionEndOffsets = ImmutableMap.copyOf(partitionEndOffsets);
}
import static java.util.Objects.requireNonNull;

public List<PartitionInfo> getPartitionInfos()
{
return partitionInfos;
}

public Map<TopicPartition, Long> getPartitionBeginOffsets()
{
return partitionBeginOffsets;
}

public Map<TopicPartition, Long> getPartitionEndOffsets()
public record KafkaFilteringResult(
List<PartitionInfo> partitionInfos,
Map<TopicPartition, Long> partitionBeginOffsets,
Map<TopicPartition, Long> partitionEndOffsets)
{
public KafkaFilteringResult
{
return partitionEndOffsets;
partitionInfos = ImmutableList.copyOf(requireNonNull(partitionInfos, "partitionInfos is null"));
partitionBeginOffsets = ImmutableMap.copyOf(requireNonNull(partitionBeginOffsets, "partitionBeginOffsets is null"));
partitionEndOffsets = ImmutableMap.copyOf(requireNonNull(partitionEndOffsets, "partitionEndOffsets is null"));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -93,13 +93,13 @@ public KafkaTableHandle getTableHandle(ConnectorSession session, SchemaTableName
.map(kafkaTopicDescription -> new KafkaTableHandle(
schemaTableName.getSchemaName(),
schemaTableName.getTableName(),
kafkaTopicDescription.getTopicName(),
getDataFormat(kafkaTopicDescription.getKey()),
getDataFormat(kafkaTopicDescription.getMessage()),
kafkaTopicDescription.getKey().flatMap(KafkaTopicFieldGroup::getDataSchema),
kafkaTopicDescription.getMessage().flatMap(KafkaTopicFieldGroup::getDataSchema),
kafkaTopicDescription.getKey().flatMap(KafkaTopicFieldGroup::getSubject),
kafkaTopicDescription.getMessage().flatMap(KafkaTopicFieldGroup::getSubject),
kafkaTopicDescription.topicName(),
getDataFormat(kafkaTopicDescription.key()),
getDataFormat(kafkaTopicDescription.message()),
kafkaTopicDescription.key().flatMap(KafkaTopicFieldGroup::dataSchema),
kafkaTopicDescription.message().flatMap(KafkaTopicFieldGroup::dataSchema),
kafkaTopicDescription.key().flatMap(KafkaTopicFieldGroup::subject),
kafkaTopicDescription.message().flatMap(KafkaTopicFieldGroup::subject),
getColumnHandles(session, schemaTableName).values().stream()
.map(KafkaColumnHandle.class::cast)
.collect(toImmutableList()),
Expand All @@ -109,13 +109,13 @@ public KafkaTableHandle getTableHandle(ConnectorSession session, SchemaTableName

private static String getDataFormat(Optional<KafkaTopicFieldGroup> fieldGroup)
{
return fieldGroup.map(KafkaTopicFieldGroup::getDataFormat).orElse(DummyRowDecoder.NAME);
return fieldGroup.map(KafkaTopicFieldGroup::dataFormat).orElse(DummyRowDecoder.NAME);
}

@Override
public ConnectorTableMetadata getTableMetadata(ConnectorSession session, ConnectorTableHandle tableHandle)
{
return getTableMetadata(session, ((KafkaTableHandle) tableHandle).toSchemaTableName());
return getTableMetadata(session, ((KafkaTableHandle) tableHandle).schemaTableName());
}

@Override
Expand All @@ -129,22 +129,22 @@ public List<SchemaTableName> listTables(ConnectorSession session, Optional<Strin
@Override
public Map<String, ColumnHandle> getColumnHandles(ConnectorSession session, ConnectorTableHandle tableHandle)
{
return getColumnHandles(session, ((KafkaTableHandle) tableHandle).toSchemaTableName());
return getColumnHandles(session, ((KafkaTableHandle) tableHandle).schemaTableName());
}

private Map<String, ColumnHandle> getColumnHandles(ConnectorSession session, SchemaTableName schemaTableName)
{
KafkaTopicDescription kafkaTopicDescription = getRequiredTopicDescription(session, schemaTableName);

Stream<KafkaColumnHandle> keyColumnHandles = kafkaTopicDescription.getKey().stream()
.map(KafkaTopicFieldGroup::getFields)
Stream<KafkaColumnHandle> keyColumnHandles = kafkaTopicDescription.key().stream()
.map(KafkaTopicFieldGroup::fields)
.flatMap(Collection::stream)
.map(kafkaTopicFieldDescription -> kafkaTopicFieldDescription.getColumnHandle(true));
.map(kafkaTopicFieldDescription -> kafkaTopicFieldDescription.columnHandle(true));

Stream<KafkaColumnHandle> messageColumnHandles = kafkaTopicDescription.getMessage().stream()
.map(KafkaTopicFieldGroup::getFields)
Stream<KafkaColumnHandle> messageColumnHandles = kafkaTopicDescription.message().stream()
.map(KafkaTopicFieldGroup::fields)
.flatMap(Collection::stream)
.map(kafkaTopicFieldDescription -> kafkaTopicFieldDescription.getColumnHandle(false));
.map(kafkaTopicFieldDescription -> kafkaTopicFieldDescription.columnHandle(false));

List<KafkaColumnHandle> topicColumnHandles = concat(keyColumnHandles, messageColumnHandles)
.collect(toImmutableList());
Expand Down Expand Up @@ -204,20 +204,20 @@ private ConnectorTableMetadata getTableMetadata(ConnectorSession session, Schema

ImmutableList.Builder<ColumnMetadata> builder = ImmutableList.builder();

table.getKey().ifPresent(key -> {
List<KafkaTopicFieldDescription> fields = key.getFields();
table.key().ifPresent(key -> {
List<KafkaTopicFieldDescription> fields = key.fields();
if (fields != null) {
for (KafkaTopicFieldDescription fieldDescription : fields) {
builder.add(fieldDescription.getColumnMetadata());
builder.add(fieldDescription.columnMetadata());
}
}
});

table.getMessage().ifPresent(message -> {
List<KafkaTopicFieldDescription> fields = message.getFields();
table.message().ifPresent(message -> {
List<KafkaTopicFieldDescription> fields = message.fields();
if (fields != null) {
for (KafkaTopicFieldDescription fieldDescription : fields) {
builder.add(fieldDescription.getColumnMetadata());
builder.add(fieldDescription.columnMetadata());
}
}
});
Expand All @@ -233,23 +233,23 @@ private ConnectorTableMetadata getTableMetadata(ConnectorSession session, Schema
public Optional<ConstraintApplicationResult<ConnectorTableHandle>> applyFilter(ConnectorSession session, ConnectorTableHandle table, Constraint constraint)
{
KafkaTableHandle handle = (KafkaTableHandle) table;
TupleDomain<ColumnHandle> oldDomain = handle.getConstraint();
TupleDomain<ColumnHandle> oldDomain = handle.constraint();
TupleDomain<ColumnHandle> newDomain = oldDomain.intersect(constraint.getSummary());
if (oldDomain.equals(newDomain)) {
return Optional.empty();
}

handle = new KafkaTableHandle(
handle.getSchemaName(),
handle.getTableName(),
handle.getTopicName(),
handle.getKeyDataFormat(),
handle.getMessageDataFormat(),
handle.getKeyDataSchemaLocation(),
handle.getMessageDataSchemaLocation(),
handle.getKeySubject(),
handle.getMessageSubject(),
handle.getColumns(),
handle.schemaName(),
handle.tableName(),
handle.topicName(),
handle.keyDataFormat(),
handle.messageDataFormat(),
handle.keyDataSchemaLocation(),
handle.messageDataSchemaLocation(),
handle.keySubject(),
handle.messageSubject(),
handle.columns(),
newDomain);

return Optional.of(new ConstraintApplicationResult<>(handle, constraint.getSummary(), constraint.getExpression(), false));
Expand All @@ -273,22 +273,22 @@ public ConnectorInsertTableHandle beginInsert(ConnectorSession session, Connecto
}
// TODO: support transactional inserts https://github.com/trinodb/trino/issues/4303
KafkaTableHandle table = (KafkaTableHandle) tableHandle;
List<KafkaColumnHandle> actualColumns = table.getColumns().stream()
List<KafkaColumnHandle> actualColumns = table.columns().stream()
.filter(columnHandle -> !columnHandle.isInternal() && !columnHandle.isHidden())
.collect(toImmutableList());

checkArgument(columns.equals(actualColumns), "Unexpected columns!\nexpected: %s\ngot: %s", actualColumns, columns);

return new KafkaTableHandle(
table.getSchemaName(),
table.getTableName(),
table.getTopicName(),
table.getKeyDataFormat(),
table.getMessageDataFormat(),
table.getKeyDataSchemaLocation(),
table.getMessageDataSchemaLocation(),
table.getKeySubject(),
table.getMessageSubject(),
table.schemaName(),
table.tableName(),
table.topicName(),
table.keyDataFormat(),
table.messageDataFormat(),
table.keyDataSchemaLocation(),
table.messageDataSchemaLocation(),
table.keySubject(),
table.messageSubject(),
actualColumns,
TupleDomain.none());
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -68,7 +68,7 @@ public ConnectorPageSink createPageSink(ConnectorTransactionHandle transactionHa

ImmutableList.Builder<EncoderColumnHandle> keyColumns = ImmutableList.builder();
ImmutableList.Builder<EncoderColumnHandle> messageColumns = ImmutableList.builder();
handle.getColumns().forEach(col -> {
handle.columns().forEach(col -> {
if (col.isInternal()) {
throw new IllegalArgumentException(format("unexpected internal column '%s'", col.getName()));
}
Expand All @@ -89,8 +89,8 @@ public ConnectorPageSink createPageSink(ConnectorTransactionHandle transactionHa
toRowEncoderSpec(handle, messageColumns.build(), MESSAGE));

return new KafkaPageSink(
handle.getTopicName(),
handle.getColumns(),
handle.topicName(),
handle.columns(),
keyEncoder,
messageEncoder,
producerFactory,
Expand All @@ -100,8 +100,8 @@ public ConnectorPageSink createPageSink(ConnectorTransactionHandle transactionHa
private static RowEncoderSpec toRowEncoderSpec(KafkaTableHandle handle, List<EncoderColumnHandle> columns, KafkaFieldType kafkaFieldType)
{
return switch (kafkaFieldType) {
case KEY -> new RowEncoderSpec(handle.getKeyDataFormat(), getDataSchema(handle.getKeyDataSchemaLocation()), columns, handle.getTopicName(), kafkaFieldType);
case MESSAGE -> new RowEncoderSpec(handle.getMessageDataFormat(), getDataSchema(handle.getMessageDataSchemaLocation()), columns, handle.getTopicName(), kafkaFieldType);
case KEY -> new RowEncoderSpec(handle.keyDataFormat(), getDataSchema(handle.keyDataSchemaLocation()), columns, handle.topicName(), kafkaFieldType);
case MESSAGE -> new RowEncoderSpec(handle.messageDataFormat(), getDataSchema(handle.messageDataSchemaLocation()), columns, handle.topicName(), kafkaFieldType);
};
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -122,7 +122,7 @@ private KafkaRecordCursor()
topicPartition = new TopicPartition(split.getTopicName(), split.getPartitionId());
kafkaConsumer = consumerFactory.create(connectorSession);
kafkaConsumer.assign(ImmutableList.of(topicPartition));
kafkaConsumer.seek(topicPartition, split.getMessagesRange().getBegin());
kafkaConsumer.seek(topicPartition, split.getMessagesRange().begin());
}

@Override
Expand Down Expand Up @@ -150,7 +150,7 @@ public boolean advanceNextPosition()
if (records.hasNext()) {
return nextRow(records.next());
}
if (kafkaConsumer.position(topicPartition) >= split.getMessagesRange().getEnd()) {
if (kafkaConsumer.position(topicPartition) >= split.getMessagesRange().end()) {
return false;
}
records = kafkaConsumer.poll(Duration.ofMillis(CONSUMER_POLL_TIMEOUT)).iterator();
Expand All @@ -161,7 +161,7 @@ private boolean nextRow(ConsumerRecord<byte[], byte[]> message)
{
requireNonNull(message, "message is null");

if (message.offset() >= split.getMessagesRange().getEnd()) {
if (message.offset() >= split.getMessagesRange().end()) {
return false;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -135,7 +135,7 @@ public long getRetainedSizeInBytes()
+ estimatedSizeOf(messageDataFormat)
+ sizeOf(keyDataSchemaContents, SizeOf::estimatedSizeOf)
+ sizeOf(messageDataSchemaContents, SizeOf::estimatedSizeOf)
+ messagesRange.getRetainedSizeInBytes()
+ messagesRange.retainedSizeInBytes()
+ leader.getRetainedSizeInBytes();
}

Expand Down
Loading