Skip to content

Commit

Permalink
Refactor KafkaInternalFieldDescription to KafkaInternalFieldManager
Browse files Browse the repository at this point in the history
The KafkaInternalFieldManager creates the internalFields map in the
consructor where the TypeManager can be used.

Signed-off-by: Sven Pfennig <[email protected]>
  • Loading branch information
Sven Pfennig authored and losipiuk committed Sep 10, 2020
1 parent 8b3f1fe commit a8345e2
Show file tree
Hide file tree
Showing 5 changed files with 186 additions and 146 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -54,6 +54,7 @@ public void configure(Binder binder)
binder.bind(ConnectorPageSinkProvider.class).annotatedWith(ForClassLoaderSafe.class).to(KafkaPageSinkProvider.class).in(Scopes.SINGLETON);
binder.bind(ConnectorPageSinkProvider.class).to(ClassLoaderSafeConnectorPageSinkProvider.class).in(Scopes.SINGLETON);
binder.bind(KafkaConnector.class).in(Scopes.SINGLETON);
binder.bind(KafkaInternalFieldManager.class).in(Scopes.SINGLETON);

configBinder(binder).bindConfig(KafkaConfig.class);
newSetBinder(binder, TableDescriptionSupplier.class).addBinding().toProvider(KafkaTableDescriptionSupplier.class).in(Scopes.SINGLETON);
Expand Down

This file was deleted.

Original file line number Diff line number Diff line change
@@ -0,0 +1,167 @@
/*
* Licensed 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 io.prestosql.plugin.kafka;

import com.google.common.collect.ImmutableMap;
import com.google.inject.Inject;
import io.prestosql.spi.connector.ColumnMetadata;
import io.prestosql.spi.type.BigintType;
import io.prestosql.spi.type.BooleanType;
import io.prestosql.spi.type.Type;
import io.prestosql.spi.type.TypeManager;

import java.util.Map;
import java.util.Optional;

import static io.prestosql.spi.type.VarcharType.createUnboundedVarcharType;
import static java.util.Objects.requireNonNull;

public class KafkaInternalFieldManager
{
/**
* <tt>_partition_id</tt> - Kafka partition id.
*/
public static final String PARTITION_ID_FIELD = "_partition_id";

/**
* <tt>_partition_offset</tt> - The current offset of the message in the partition.
*/
public static final String PARTITION_OFFSET_FIELD = "_partition_offset";

/**
* <tt>_message_corrupt</tt> - True if the row converter could not read the a message. May be null if the row converter does not set a value (e.g. the dummy row converter does not).
*/
public static final String MESSAGE_CORRUPT_FIELD = "_message_corrupt";

/**
* <tt>_message</tt> - Represents the full topic as a text column. Format is UTF-8 which may be wrong for some topics. TODO: make charset configurable.
*/
public static final String MESSAGE_FIELD = "_message";

/**
* <tt>_message_length</tt> - length in bytes of the message.
*/
public static final String MESSAGE_LENGTH_FIELD = "_message_length";

/**
* <tt>_key_corrupt</tt> - True if the row converter could not read the a key. May be null if the row converter does not set a value (e.g. the dummy row converter does not).
*/
public static final String KEY_CORRUPT_FIELD = "_key_corrupt";

/**
* <tt>_key</tt> - Represents the key as a text column. Format is UTF-8 which may be wrong for topics. TODO: make charset configurable.
*/
public static final String KEY_FIELD = "_key";

/**
* <tt>_key_length</tt> - length in bytes of the key.
*/
public static final String KEY_LENGTH_FIELD = "_key_length";

public static class InternalField
{
private final String columnName;
private final String comment;
private final Type type;

InternalField(String columnName, String comment, Type type)
{
this.columnName = requireNonNull(columnName, "columnName is null");
this.comment = requireNonNull(comment, "comment is null");
this.type = requireNonNull(type, "type is null");
}

public String getColumnName()
{
return columnName;
}

private Type getType()
{
return type;
}

KafkaColumnHandle getColumnHandle(int index, boolean hidden)
{
return new KafkaColumnHandle(
getColumnName(),
getType(),
null,
null,
null,
false,
hidden,
true);
}

ColumnMetadata getColumnMetadata(boolean hidden)
{
return ColumnMetadata.builder()
.setName(columnName)
.setType(type)
.setComment(Optional.ofNullable(columnName))
.setHidden(hidden)
.build();
}
}

private final Map<String, InternalField> internalFields;

@Inject
public KafkaInternalFieldManager(TypeManager typeManager)
{
internalFields = new ImmutableMap.Builder<String, InternalField>()
.put(PARTITION_ID_FIELD, new InternalField(
PARTITION_ID_FIELD,
"Partition Id",
BigintType.BIGINT))
.put(PARTITION_OFFSET_FIELD, new InternalField(
PARTITION_OFFSET_FIELD,
"Offset for the message within the partition",
BigintType.BIGINT))
.put(MESSAGE_CORRUPT_FIELD, new InternalField(
MESSAGE_CORRUPT_FIELD,
"Message data is corrupt",
BooleanType.BOOLEAN))
.put(MESSAGE_FIELD, new InternalField(
MESSAGE_FIELD,
"Message text",
createUnboundedVarcharType()))
.put(MESSAGE_LENGTH_FIELD, new InternalField(
MESSAGE_LENGTH_FIELD,
"Total number of message bytes",
BigintType.BIGINT))
.put(KEY_CORRUPT_FIELD, new InternalField(
KEY_CORRUPT_FIELD,
"Key data is corrupt",
BooleanType.BOOLEAN))
.put(KEY_FIELD, new InternalField(
KEY_FIELD,
"Key text",
createUnboundedVarcharType()))
.put(KEY_LENGTH_FIELD, new InternalField(
KEY_LENGTH_FIELD,
"Total number of key bytes",
BigintType.BIGINT))
.build();
}

/**
* @return Map of {@link InternalField} for each internal field.
*/
public Map<String, InternalField> getInternalFields()
{
return internalFields;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -48,23 +48,26 @@

/**
* Manages the Kafka connector specific metadata information. The Connector provides an additional set of columns
* for each table that are created as hidden columns. See {@link KafkaInternalFieldDescription} for a list
* for each table that are created as hidden columns. See {@link KafkaInternalFieldManager} for a list
* of per-topic additional columns.
*/
public class KafkaMetadata
implements ConnectorMetadata
{
private final boolean hideInternalColumns;
private final Set<TableDescriptionSupplier> tableDescriptions;
private final KafkaInternalFieldManager kafkaInternalFieldManager;

@Inject
public KafkaMetadata(
KafkaConfig kafkaConfig,
Set<TableDescriptionSupplier> tableDescriptions)
Set<TableDescriptionSupplier> tableDescriptions,
KafkaInternalFieldManager kafkaInternalFieldManager)
{
requireNonNull(kafkaConfig, "kafkaConfig is null");
this.hideInternalColumns = kafkaConfig.isHideInternalColumns();
this.tableDescriptions = requireNonNull(tableDescriptions, "tableDescriptions is null");
this.kafkaInternalFieldManager = requireNonNull(kafkaInternalFieldManager, "kafkaInternalFieldDescription is null");
}

@Override
Expand Down Expand Up @@ -149,8 +152,8 @@ private Map<String, ColumnHandle> getColumnHandles(SchemaTableName schemaTableNa
}
});

for (KafkaInternalFieldDescription kafkaInternalFieldDescription : KafkaInternalFieldDescription.values()) {
columnHandles.put(kafkaInternalFieldDescription.getColumnName(), kafkaInternalFieldDescription.getColumnHandle(index.getAndIncrement(), hideInternalColumns));
for (KafkaInternalFieldManager.InternalField kafkaInternalField : kafkaInternalFieldManager.getInternalFields().values()) {
columnHandles.put(kafkaInternalField.getColumnName(), kafkaInternalField.getColumnHandle(index.getAndIncrement(), hideInternalColumns));
}

return columnHandles.build();
Expand Down Expand Up @@ -213,7 +216,7 @@ private ConnectorTableMetadata getTableMetadata(SchemaTableName schemaTableName)
}
});

for (KafkaInternalFieldDescription fieldDescription : KafkaInternalFieldDescription.values()) {
for (KafkaInternalFieldManager.InternalField fieldDescription : kafkaInternalFieldManager.getInternalFields().values()) {
builder.add(fieldDescription.getColumnMetadata(hideInternalColumns));
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,14 @@
import static io.prestosql.decoder.FieldValueProviders.booleanValueProvider;
import static io.prestosql.decoder.FieldValueProviders.bytesValueProvider;
import static io.prestosql.decoder.FieldValueProviders.longValueProvider;
import static io.prestosql.plugin.kafka.KafkaInternalFieldManager.KEY_CORRUPT_FIELD;
import static io.prestosql.plugin.kafka.KafkaInternalFieldManager.KEY_FIELD;
import static io.prestosql.plugin.kafka.KafkaInternalFieldManager.KEY_LENGTH_FIELD;
import static io.prestosql.plugin.kafka.KafkaInternalFieldManager.MESSAGE_CORRUPT_FIELD;
import static io.prestosql.plugin.kafka.KafkaInternalFieldManager.MESSAGE_FIELD;
import static io.prestosql.plugin.kafka.KafkaInternalFieldManager.MESSAGE_LENGTH_FIELD;
import static io.prestosql.plugin.kafka.KafkaInternalFieldManager.PARTITION_ID_FIELD;
import static io.prestosql.plugin.kafka.KafkaInternalFieldManager.PARTITION_OFFSET_FIELD;
import static java.lang.Math.max;
import static java.util.Collections.emptyIterator;
import static java.util.Objects.requireNonNull;
Expand Down Expand Up @@ -170,8 +178,7 @@ private boolean nextRow(ConsumerRecord<byte[], byte[]> message)

for (DecoderColumnHandle columnHandle : columnHandles) {
if (columnHandle.isInternal()) {
KafkaInternalFieldDescription fieldDescription = KafkaInternalFieldDescription.forColumnName(columnHandle.getName());
switch (fieldDescription) {
switch (columnHandle.getName()) {
case PARTITION_OFFSET_FIELD:
currentRowValuesMap.put(columnHandle, longValueProvider(message.offset()));
break;
Expand All @@ -197,7 +204,7 @@ private boolean nextRow(ConsumerRecord<byte[], byte[]> message)
currentRowValuesMap.put(columnHandle, longValueProvider(message.partition()));
break;
default:
throw new IllegalArgumentException("unknown internal field " + fieldDescription);
throw new IllegalArgumentException("unknown internal field " + columnHandle.getName());
}
}
}
Expand Down

0 comments on commit a8345e2

Please sign in to comment.