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

Add Kafka headers as column #4462

Merged
merged 3 commits into from Sep 10, 2020
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
31 changes: 16 additions & 15 deletions presto-docs/src/main/sphinx/connector/kafka.rst
Original file line number Diff line number Diff line change
Expand Up @@ -136,21 +136,22 @@ Internal Columns

For each defined table, the connector maintains the following columns:

======================= ========= =============================
Column name Type Description
======================= ========= =============================
``_partition_id`` BIGINT ID of the Kafka partition which contains this row.
``_partition_offset`` BIGINT Offset within the Kafka partition for this row.
``_segment_start`` BIGINT Lowest offset in the segment (inclusive) which contains this row. This offset is partition specific.
``_segment_end`` BIGINT Highest offset in the segment (exclusive) which contains this row. The offset is partition specific. This is the same value as ``_segment_start`` of the next segment (if it exists).
``_segment_count`` BIGINT Running count for the current row within the segment. For an uncompacted topic, ``_segment_start + _segment_count`` is equal to ``_partition_offset``.
``_message_corrupt`` BOOLEAN True if the decoder could not decode the message for this row. When true, data columns mapped from the message should be treated as invalid.
``_message`` VARCHAR Message bytes as an UTF-8 encoded string. This is only useful for a text topic.
``_message_length`` BIGINT Number of bytes in the message.
``_key_corrupt`` BOOLEAN True if the key decoder could not decode the key for this row. When true, data columns mapped from the key should be treated as invalid.
``_key`` VARCHAR Key bytes as an UTF-8 encoded string. This is only useful for textual keys.
``_key_length`` BIGINT Number of bytes in the key.
======================= ========= =============================
======================= =============================== =============================
Column name Type Description
======================= =============================== =============================
0xE282B0 marked this conversation as resolved.
Show resolved Hide resolved
``_partition_id`` BIGINT ID of the Kafka partition which contains this row.
``_partition_offset`` BIGINT Offset within the Kafka partition for this row.
``_segment_start`` BIGINT Lowest offset in the segment (inclusive) which contains this row. This offset is partition specific.
``_segment_end`` BIGINT Highest offset in the segment (exclusive) which contains this row. The offset is partition specific. This is the same value as ``_segment_start`` of the next segment (if it exists).
``_segment_count`` BIGINT Running count for the current row within the segment. For an uncompacted topic, ``_segment_start + _segment_count`` is equal to ``_partition_offset``.
``_message_corrupt`` BOOLEAN True if the decoder could not decode the message for this row. When true, data columns mapped from the message should be treated as invalid.
``_message`` VARCHAR Message bytes as an UTF-8 encoded string. This is only useful for a text topic.
``_message_length`` BIGINT Number of bytes in the message.
``_headers`` map(VARCHAR, array(VARBINARY)) Headers of the message where values with the same key are grouped as array.
``_key_corrupt`` BOOLEAN True if the key decoder could not decode the key for this row. When true, data columns mapped from the key should be treated as invalid.
``_key`` VARCHAR Key bytes as an UTF-8 encoded string. This is only useful for textual keys.
``_key_length`` BIGINT Number of bytes in the key.
======================= =============================== =============================

For tables without a table definition file, the ``_key_corrupt`` and
``_message_corrupt`` columns will always be ``false``.
Expand Down
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,182 @@
/*
* 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.TypeSignature.arrayType;
import static io.prestosql.spi.type.TypeSignature.mapType;
import static io.prestosql.spi.type.VarbinaryType.VARBINARY;
import static io.prestosql.spi.type.VarcharType.VARCHAR;
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>_headers</tt> - The header fields of the Kafka message. Key is a UTF-8 String and values an array of byte[].
*/
public static final String HEADERS_FIELD = "_headers";

/**
* <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)
{
Type varcharMapType = typeManager.getType(mapType(VARCHAR.getTypeSignature(), arrayType(VARBINARY.getTypeSignature())));

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(HEADERS_FIELD, new InternalField(
HEADERS_FIELD,
"Headers of the message as map",
varcharMapType))
.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;
}
}
Loading