From 9b1839d41c1e3cdc5833fc9661f2347c40275e2a Mon Sep 17 00:00:00 2001 From: XHLY Date: Sun, 13 Oct 2019 02:08:18 +0800 Subject: [PATCH 1/8] Improve naming of TypeSignatureParamter.of() methods # Conflicts: both of them --- .../io/prestosql/spi/type/TypeSignature.java | 63 +++++++++++++++++-- .../spi/type/TypeSignatureParameter.java | 10 +++ 2 files changed, 67 insertions(+), 6 deletions(-) diff --git a/presto-spi/src/main/java/io/prestosql/spi/type/TypeSignature.java b/presto-spi/src/main/java/io/prestosql/spi/type/TypeSignature.java index b1e77fd8e..512fa2977 100644 --- a/presto-spi/src/main/java/io/prestosql/spi/type/TypeSignature.java +++ b/presto-spi/src/main/java/io/prestosql/spi/type/TypeSignature.java @@ -17,6 +17,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonValue; import java.util.ArrayList; +import java.util.Arrays; import java.util.HashSet; import java.util.List; import java.util.Locale; @@ -25,8 +26,9 @@ import java.util.Optional; import java.util.Set; import java.util.TreeSet; import java.util.regex.Pattern; +import java.util.stream.Collectors; -import static io.prestosql.spi.type.TypeSignatureParameter.typeParameter; +import static io.prestosql.spi.type.TypeSignatureParameter.of; import static java.lang.Character.isDigit; import static java.lang.String.format; import static java.util.Arrays.asList; @@ -50,11 +52,6 @@ public class TypeSignature SIMPLE_TYPE_WITH_SPACES.add("double precision"); } - public static TypeSignature mapType(TypeSignature keyType, TypeSignature valueType) - { - return new TypeSignature(StandardTypes.MAP, typeParameter(keyType), typeParameter(valueType)); - } - public TypeSignature(String base, TypeSignatureParameter... parameters) { this(base, asList(parameters)); @@ -100,7 +97,11 @@ public class TypeSignature return calculated; } + /** + * @deprecated Build TypeSignature programmatically instead + */ @JsonCreator + @Deprecated public static TypeSignature parseTypeSignature(String signature) { return parseTypeSignature(signature, new HashSet<>()); @@ -394,4 +395,54 @@ public class TypeSignature { return Objects.hash(base.toLowerCase(Locale.ENGLISH), parameters); } + + // Type signature constructors for common types + + public static TypeSignature arrayType(TypeSignature elementType) + { + return new TypeSignature(StandardTypes.ARRAY, of(elementType)); + } + + public static TypeSignature arrayType(TypeSignatureParameter elementType) + { + return new TypeSignature(StandardTypes.ARRAY, elementType); + } + + public static TypeSignature mapType(TypeSignature keyType, TypeSignature valueType) + { + return new TypeSignature(StandardTypes.MAP, of(keyType), of(valueType)); + } + + public static TypeSignature parametricType(String name, TypeSignature... parameters) + { + return new TypeSignature( + name, + Arrays.asList(parameters).stream() + .map(TypeSignatureParameter::of) + .collect(Collectors.toList())); + } + + public static TypeSignature functionType(TypeSignature first, TypeSignature... rest) + { + List parameters = new ArrayList<>(); + parameters.add(of(first)); + + Arrays.asList(rest).stream() + .map(TypeSignatureParameter::of) + .forEach(parameters::add); + + return new TypeSignature("function", parameters); + } + + public static TypeSignature rowType(TypeSignatureParameter... fields) + { + return rowType(Arrays.asList(fields)); + } + + public static TypeSignature rowType(List fields) + { + checkArgument(fields.stream().allMatch(parameter -> parameter.getKind() == ParameterKind.NAMED_TYPE), "Parameters for ROW type must be NAMED_TYPE parameters"); + + return new TypeSignature(StandardTypes.ROW, fields); + } } diff --git a/presto-spi/src/main/java/io/prestosql/spi/type/TypeSignatureParameter.java b/presto-spi/src/main/java/io/prestosql/spi/type/TypeSignatureParameter.java index 1580fc2c0..6f5b25c5e 100644 --- a/presto-spi/src/main/java/io/prestosql/spi/type/TypeSignatureParameter.java +++ b/presto-spi/src/main/java/io/prestosql/spi/type/TypeSignatureParameter.java @@ -44,6 +44,16 @@ public class TypeSignatureParameter return new TypeSignatureParameter(ParameterKind.NAMED_TYPE, namedTypeSignature); } + public static TypeSignatureParameter of(String name, TypeSignature type) + { + return new TypeSignatureParameter(ParameterKind.NAMED_TYPE, new NamedTypeSignature(Optional.of(new RowFieldName(name, false)), type)); + } + + /*public static TypeSignatureParameter anonymousField(TypeSignature type) + { + return new TypeSignatureParameter(ParameterKind.NAMED_TYPE, new NamedTypeSignature(Optional.empty(), type)); + }*/ + public static TypeSignatureParameter of(String variable) { return new TypeSignatureParameter(ParameterKind.VARIABLE, variable); -- Gitee From 50471b5c5034963bed3f157821eaef6d8f264359 Mon Sep 17 00:00:00 2001 From: XHLY Date: Mon, 30 Dec 2019 21:25:04 +0800 Subject: [PATCH 2/8] Use new Kafka API for consuming messages Previously we created one Presto split per one segment file in each Kafka partitions. New Kafka API does not expose information about segments. Because of that: - internal columns: _segment_start, _segment_end and _segment_count were removed - new configuration property was added `kafka.messages-per-split` to process single topic partition in parallel in Presto --- .../prestosql/plugin/kafka/KafkaConfig.java | 171 ++++++++++++++++++ .../plugin/kafka/KafkaConnector.java | 15 +- .../plugin/kafka/KafkaConnectorFactory.java | 55 +++--- .../plugin/kafka/KafkaConnectorModule.java | 4 +- .../plugin/kafka/KafkaConsumerFactory.java | 62 +++++++ .../kafka/KafkaInternalFieldDescription.java | 15 -- .../plugin/kafka/KafkaRecordSet.java | 163 ++++------------- .../plugin/kafka/KafkaRecordSetProvider.java | 10 +- .../io/prestosql/plugin/kafka/KafkaSplit.java | 39 ++-- .../plugin/kafka/KafkaSplitManager.java | 115 ++++-------- .../java/io/prestosql/plugin/kafka/Range.java | 68 +++++++ .../plugin/kafka/TestKafkaConfig.java | 68 +++++++ .../plugin/kafka/util/JsonSerializer.java | 47 +++++ .../plugin/kafka/util/NumberPartitioner.java | 20 +- .../plugin/kafka/util/TestingKafka.java | 92 ++++++++++ 15 files changed, 647 insertions(+), 297 deletions(-) create mode 100644 presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaConfig.java create mode 100644 presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaConsumerFactory.java create mode 100644 presto-kafka/src/main/java/io/prestosql/plugin/kafka/Range.java create mode 100644 presto-kafka/src/test/java/io/prestosql/plugin/kafka/TestKafkaConfig.java create mode 100644 presto-kafka/src/test/java/io/prestosql/plugin/kafka/util/JsonSerializer.java create mode 100644 presto-kafka/src/test/java/io/prestosql/plugin/kafka/util/TestingKafka.java diff --git a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaConfig.java b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaConfig.java new file mode 100644 index 000000000..0adfceed8 --- /dev/null +++ b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaConfig.java @@ -0,0 +1,171 @@ +/* + * 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.base.Splitter; +import com.google.common.collect.ImmutableSet; +import io.airlift.configuration.Config; +import io.airlift.configuration.ConfigDescription; +import io.airlift.units.DataSize; +import io.airlift.units.DataSize.Unit; +import io.airlift.units.Duration; +import io.airlift.units.MinDuration; +import io.prestosql.spi.HostAddress; + +import javax.validation.constraints.Min; +import javax.validation.constraints.NotNull; +import javax.validation.constraints.Size; + +import java.io.File; +import java.util.Set; +import java.util.stream.StreamSupport; + +import static com.google.common.collect.ImmutableSet.toImmutableSet; + +public class KafkaConfig +{ + private static final int KAFKA_DEFAULT_PORT = 9092; + + private Set nodes = ImmutableSet.of(); + private Duration kafkaConnectTimeout = Duration.valueOf("10s"); + private DataSize kafkaBufferSize = new DataSize(64, Unit.KILOBYTE); + private String defaultSchema = "default"; + private Set tableNames = ImmutableSet.of(); + private File tableDescriptionDir = new File("etc/kafka/"); + private boolean hideInternalColumns = true; + private int messagesPerSplit = 100_000; + + @Size(min = 1) + public Set getNodes() + { + return nodes; + } + + @Config("kafka.nodes") + @ConfigDescription("Seed nodes for Kafka cluster. At least one must exist") + public KafkaConfig setNodes(String nodes) + { + this.nodes = (nodes == null) ? null : parseNodes(nodes); + return this; + } + + @MinDuration("1s") + public Duration getKafkaConnectTimeout() + { + return kafkaConnectTimeout; + } + + @Config("kafka.connect-timeout") + @ConfigDescription("Kafka connection timeout") + public KafkaConfig setKafkaConnectTimeout(String kafkaConnectTimeout) + { + this.kafkaConnectTimeout = Duration.valueOf(kafkaConnectTimeout); + return this; + } + + public DataSize getKafkaBufferSize() + { + return kafkaBufferSize; + } + + @Config("kafka.buffer-size") + @ConfigDescription("Kafka message consumer buffer size") + public KafkaConfig setKafkaBufferSize(String kafkaBufferSize) + { + this.kafkaBufferSize = DataSize.valueOf(kafkaBufferSize); + return this; + } + + @NotNull + public String getDefaultSchema() + { + return defaultSchema; + } + + @Config("kafka.default-schema") + @ConfigDescription("Schema name to use in the connector") + public KafkaConfig setDefaultSchema(String defaultSchema) + { + this.defaultSchema = defaultSchema; + return this; + } + + @NotNull + public Set getTableNames() + { + return tableNames; + } + + @Config("kafka.table-names") + @ConfigDescription("Set of tables known to this connector") + public KafkaConfig setTableNames(String tableNames) + { + this.tableNames = ImmutableSet.copyOf(Splitter.on(',').omitEmptyStrings().trimResults().split(tableNames)); + return this; + } + + public boolean isHideInternalColumns() + { + return hideInternalColumns; + } + + @Config("kafka.hide-internal-columns") + @ConfigDescription("Whether internal columns are shown in table metadata or not. Default is no") + public KafkaConfig setHideInternalColumns(boolean hideInternalColumns) + { + this.hideInternalColumns = hideInternalColumns; + return this; + } + + @NotNull + public File getTableDescriptionDir() + { + return tableDescriptionDir; + } + + @Config("kafka.table-description-dir") + @ConfigDescription("Folder holding JSON description files for Kafka topics") + public KafkaConfig setTableDescriptionDir(File tableDescriptionDir) + { + this.tableDescriptionDir = tableDescriptionDir; + return this; + } + + private static ImmutableSet parseNodes(String nodes) + { + Splitter splitter = Splitter.on(',').omitEmptyStrings().trimResults(); + return StreamSupport.stream(splitter.split(nodes).spliterator(), false) + .map(KafkaConfig::toHostAddress) + .collect(toImmutableSet()); + } + + private static HostAddress toHostAddress(String value) + { + return HostAddress.fromString(value).withDefaultPort(KAFKA_DEFAULT_PORT); + } + + @Min(1) + public int getMessagesPerSplit() + { + return messagesPerSplit; + } + + @Config("kafka.messages-per-split") + @ConfigDescription("Count of Kafka messages to be processed by single Presto Kafka connector split") + public KafkaConfig setMessagesPerSplit(int messagesPerSplit) + { + this.messagesPerSplit = messagesPerSplit; + return this; + } +} diff --git a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaConnector.java b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaConnector.java index 610cdf0f4..3e0e46310 100644 --- a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaConnector.java +++ b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaConnector.java @@ -22,8 +22,6 @@ import io.prestosql.spi.connector.ConnectorSplitManager; import io.prestosql.spi.connector.ConnectorTransactionHandle; import io.prestosql.spi.transaction.IsolationLevel; -import javax.inject.Inject; - import static io.prestosql.spi.transaction.IsolationLevel.READ_COMMITTED; import static io.prestosql.spi.transaction.IsolationLevel.checkConnectorSupports; import static java.util.Objects.requireNonNull; @@ -37,16 +35,15 @@ public class KafkaConnector private static final Logger log = Logger.get(KafkaConnector.class); private final LifeCycleManager lifeCycleManager; - private final KafkaMetadata metadata; - private final KafkaSplitManager splitManager; - private final KafkaRecordSetProvider recordSetProvider; + private final ConnectorMetadata metadata; + private final ConnectorSplitManager splitManager; + private final ConnectorRecordSetProvider recordSetProvider; - @Inject public KafkaConnector( LifeCycleManager lifeCycleManager, - KafkaMetadata metadata, - KafkaSplitManager splitManager, - KafkaRecordSetProvider recordSetProvider) + ConnectorMetadata metadata, + ConnectorSplitManager splitManager, + ConnectorRecordSetProvider recordSetProvider) { this.lifeCycleManager = requireNonNull(lifeCycleManager, "lifeCycleManager is null"); this.metadata = requireNonNull(metadata, "metadata is null"); diff --git a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaConnectorFactory.java b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaConnectorFactory.java index 2668963ad..6a79d6814 100644 --- a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaConnectorFactory.java +++ b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaConnectorFactory.java @@ -17,6 +17,7 @@ import com.google.inject.Injector; import com.google.inject.Scopes; import com.google.inject.TypeLiteral; import io.airlift.bootstrap.Bootstrap; +import io.airlift.bootstrap.LifeCycleManager; import io.airlift.json.JsonModule; import io.prestosql.spi.NodeManager; import io.prestosql.spi.connector.Connector; @@ -24,18 +25,16 @@ import io.prestosql.spi.connector.ConnectorContext; import io.prestosql.spi.connector.ConnectorFactory; import io.prestosql.spi.connector.ConnectorHandleResolver; import io.prestosql.spi.connector.SchemaTableName; +import io.prestosql.spi.connector.classloader.ClassLoaderSafeConnectorRecordSetProvider; +import io.prestosql.spi.connector.classloader.ClassLoaderSafeConnectorSplitManager; import io.prestosql.spi.type.TypeManager; import java.util.Map; import java.util.Optional; import java.util.function.Supplier; -import static com.google.common.base.Throwables.throwIfUnchecked; import static java.util.Objects.requireNonNull; -/** - * Creates Kafka Connectors based off catalogName and specific configuration. - */ public class KafkaConnectorFactory implements ConnectorFactory { @@ -64,32 +63,32 @@ public class KafkaConnectorFactory requireNonNull(catalogName, "catalogName is null"); requireNonNull(config, "config is null"); - try { - Bootstrap app = new Bootstrap( - new JsonModule(), - new KafkaConnectorModule(), - binder -> { - binder.bind(TypeManager.class).toInstance(context.getTypeManager()); - binder.bind(NodeManager.class).toInstance(context.getNodeManager()); + Bootstrap app = new Bootstrap( + new JsonModule(), + new KafkaConnectorModule(), + binder -> { + binder.bind(TypeManager.class).toInstance(context.getTypeManager()); + binder.bind(NodeManager.class).toInstance(context.getNodeManager()); - if (tableDescriptionSupplier.isPresent()) { - binder.bind(new TypeLiteral>>() {}).toInstance(tableDescriptionSupplier.get()); - } - else { - binder.bind(new TypeLiteral>>() {}).to(KafkaTableDescriptionSupplier.class).in(Scopes.SINGLETON); - } - }); + if (tableDescriptionSupplier.isPresent()) { + binder.bind(new TypeLiteral>>() {}).toInstance(tableDescriptionSupplier.get()); + } + else { + binder.bind(new TypeLiteral>>() {}).to(KafkaTableDescriptionSupplier.class).in(Scopes.SINGLETON); + } + }); - Injector injector = app.strictConfig() - .doNotInitializeLogging() - .setRequiredConfigurationProperties(config) - .initialize(); + Injector injector = app + .strictConfig() + .doNotInitializeLogging() + .setRequiredConfigurationProperties(config) + .initialize(); - return injector.getInstance(KafkaConnector.class); - } - catch (Exception e) { - throwIfUnchecked(e); - throw new RuntimeException(e); - } + ClassLoader classLoader = KafkaConnectorFactory.class.getClassLoader(); + return new KafkaConnector( + injector.getInstance(LifeCycleManager.class), + injector.getInstance(KafkaMetadata.class), + new ClassLoaderSafeConnectorSplitManager(injector.getInstance(KafkaSplitManager.class), classLoader), + new ClassLoaderSafeConnectorRecordSetProvider(injector.getInstance(KafkaRecordSetProvider.class), classLoader)); } } diff --git a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaConnectorModule.java b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaConnectorModule.java index 20e0b3012..756bd400e 100644 --- a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaConnectorModule.java +++ b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaConnectorModule.java @@ -39,13 +39,11 @@ public class KafkaConnectorModule @Override public void configure(Binder binder) { - binder.bind(KafkaConnector.class).in(Scopes.SINGLETON); - binder.bind(KafkaMetadata.class).in(Scopes.SINGLETON); binder.bind(KafkaSplitManager.class).in(Scopes.SINGLETON); binder.bind(KafkaRecordSetProvider.class).in(Scopes.SINGLETON); - binder.bind(KafkaSimpleConsumerManager.class).in(Scopes.SINGLETON); + binder.bind(KafkaConsumerFactory.class).in(Scopes.SINGLETON); configBinder(binder).bindConfig(KafkaConnectorConfig.class); diff --git a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaConsumerFactory.java b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaConsumerFactory.java new file mode 100644 index 000000000..b6009697c --- /dev/null +++ b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaConsumerFactory.java @@ -0,0 +1,62 @@ +/* + * 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 io.airlift.units.DataSize; +import io.prestosql.spi.HostAddress; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.common.serialization.ByteArrayDeserializer; + +import javax.inject.Inject; + +import java.util.Properties; +import java.util.Set; + +import static java.util.Objects.requireNonNull; +import static java.util.stream.Collectors.joining; +import static org.apache.kafka.clients.consumer.ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG; +import static org.apache.kafka.clients.consumer.ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG; +import static org.apache.kafka.clients.consumer.ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG; +import static org.apache.kafka.clients.consumer.ConsumerConfig.RECEIVE_BUFFER_CONFIG; +import static org.apache.kafka.clients.consumer.ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG; + +public class KafkaConsumerFactory +{ + private final Set nodes; + private final DataSize kafkaBufferSize; + + @Inject + public KafkaConsumerFactory(KafkaConfig kafkaConfig) + { + requireNonNull(kafkaConfig, "kafkaConfig is null"); + + nodes = kafkaConfig.getNodes(); + kafkaBufferSize = kafkaConfig.getKafkaBufferSize(); + } + + public KafkaConsumer create() + { + Properties properties = new Properties(); + properties.put( + BOOTSTRAP_SERVERS_CONFIG, + nodes.stream() + .map(HostAddress::toString) + .collect(joining(","))); + properties.put(KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName()); + properties.put(VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName()); + properties.put(RECEIVE_BUFFER_CONFIG, Long.toString(kafkaBufferSize.toBytes())); + properties.put(ENABLE_AUTO_COMMIT_CONFIG, Boolean.toString(false)); + return new KafkaConsumer(properties); + } +} diff --git a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaInternalFieldDescription.java b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaInternalFieldDescription.java index a95c7a39b..6cd5872ab 100644 --- a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaInternalFieldDescription.java +++ b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaInternalFieldDescription.java @@ -46,21 +46,6 @@ public enum KafkaInternalFieldDescription */ PARTITION_OFFSET_FIELD("_partition_offset", BigintType.BIGINT, "Offset for the message within the partition"), - /** - * _segment_start - Kafka start offset for the segment which contains the current message. This is per-partition. - */ - SEGMENT_START_FIELD("_segment_start", BigintType.BIGINT, "Segment start offset"), - - /** - * _segment_end - Kafka end offset for the segment which contains the current message. This is per-partition. The end offset is the first offset that is *not* in the segment. - */ - SEGMENT_END_FIELD("_segment_end", BigintType.BIGINT, "Segment end offset"), - - /** - * _segment_count - Running count of messages in a segment. - */ - SEGMENT_COUNT_FIELD("_segment_count", BigintType.BIGINT, "Running message count per segment"), - /** * _message_corrupt - 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). */ diff --git a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaRecordSet.java b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaRecordSet.java index 30b1e86d1..fc1804e5a 100644 --- a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaRecordSet.java +++ b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaRecordSet.java @@ -14,70 +14,57 @@ package io.prestosql.plugin.kafka; import com.google.common.collect.ImmutableList; -import io.airlift.log.Logger; import io.airlift.slice.Slice; import io.prestosql.decoder.DecoderColumnHandle; import io.prestosql.decoder.FieldValueProvider; import io.prestosql.decoder.RowDecoder; -import io.prestosql.spi.PrestoException; import io.prestosql.spi.block.Block; import io.prestosql.spi.connector.ColumnHandle; import io.prestosql.spi.connector.RecordCursor; import io.prestosql.spi.connector.RecordSet; import io.prestosql.spi.type.Type; -import kafka.api.FetchRequest; -import kafka.api.FetchRequestBuilder; import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.clients.consumer.KafkaConsumer; import org.apache.kafka.common.TopicPartition; -import java.nio.ByteBuffer; -import java.util.Collections; import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Optional; -import java.util.concurrent.atomic.AtomicBoolean; import static com.google.common.base.Preconditions.checkArgument; 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.KafkaErrorCode.KAFKA_SPLIT_ERROR; -import static java.lang.String.format; +import static java.lang.Math.max; +import static java.util.Collections.emptyIterator; import static java.util.Objects.requireNonNull; -/** - * Kafka specific record set. Returns a cursor for a topic which iterates over a Kafka partition segment. - */ public class KafkaRecordSet implements RecordSet { - private static final Logger log = Logger.get(KafkaRecordSet.class); - - private static final int KAFKA_READ_BUFFER_SIZE = 100_000; private static final byte[] EMPTY_BYTE_ARRAY = new byte[0]; + private static final int CONSUMER_POLL_TIMEOUT = 100; private final KafkaSplit split; - private final KafkaSimpleConsumerManager consumerManager; + private final KafkaConsumerFactory consumerFactory; private final RowDecoder keyDecoder; private final RowDecoder messageDecoder; private final List columnHandles; private final List columnTypes; - KafkaRecordSet(KafkaSplit split, - KafkaSimpleConsumerManager consumerManager, + KafkaRecordSet( + KafkaSplit split, + KafkaConsumerFactory consumerFactory, List columnHandles, RowDecoder keyDecoder, RowDecoder messageDecoder) { this.split = requireNonNull(split, "split is null"); - - this.consumerManager = requireNonNull(consumerManager, "consumerManager is null"); + this.consumerFactory = requireNonNull(consumerFactory, "consumerManager is null"); this.keyDecoder = requireNonNull(keyDecoder, "rowDecoder is null"); this.messageDecoder = requireNonNull(messageDecoder, "rowDecoder is null"); @@ -105,25 +92,28 @@ public class KafkaRecordSet return new KafkaRecordCursor(); } - public class KafkaRecordCursor + private class KafkaRecordCursor implements RecordCursor { - private long totalBytes; - private long totalMessages; - private long cursorOffset = split.getStart(); - private Iterator> recordIterator; - private final AtomicBoolean reported = new AtomicBoolean(); - private KafkaConsumer leaderKafkaConsumer; + private final TopicPartition topicPartition; + private final KafkaConsumer kafkaConsumer; + private Iterator> records = emptyIterator(); + private long completedBytes; + private final FieldValueProvider[] currentRowValues = new FieldValueProvider[columnHandles.size()]; - KafkaRecordCursor() + private KafkaRecordCursor() { + topicPartition = new TopicPartition(split.getTopicName(), split.getPartitionId()); + kafkaConsumer = consumerFactory.create(); + kafkaConsumer.assign(ImmutableList.of(topicPartition)); + kafkaConsumer.seek(topicPartition, split.getMessagesRange().getBegin()); } @Override public long getCompletedBytes() { - return totalBytes; + return completedBytes; } @Override @@ -142,57 +132,30 @@ public class KafkaRecordSet @Override public boolean advanceNextPosition() { - while (true) { - if (cursorOffset >= split.getEnd()) { - return endOfData(); // Split end is exclusive. - } - // Create a fetch request - openFetchRequest(); - - while (recordIterator.hasNext()) { - ConsumerRecord record = recordIterator.next(); - long messageOffset = record.offset(); - - if (messageOffset >= split.getEnd()) { - return endOfData(); // Past our split end. Bail. - } - - if (messageOffset >= cursorOffset) { - return nextRow(record); - } + if (!records.hasNext()) { + if (kafkaConsumer.position(topicPartition) >= split.getMessagesRange().getEnd()) { + return false; } - recordIterator = null; - } - } - - private boolean endOfData() - { - if (!reported.getAndSet(true)) { - log.debug("Found a total of %d messages with %d bytes (%d messages expected). Last Offset: %d (%d, %d)", - totalMessages, totalBytes, split.getEnd() - split.getStart(), - cursorOffset, split.getStart(), split.getEnd()); + records = kafkaConsumer.poll(CONSUMER_POLL_TIMEOUT).iterator(); + return advanceNextPosition(); } - return false; + nextRow(records.next()); + return true; } - private boolean nextRow(ConsumerRecord record) + private boolean nextRow(ConsumerRecord message) { - cursorOffset = record.offset() + 1; // Cursor now points to the next message. - totalBytes += record.serializedValueSize(); - totalMessages++; + requireNonNull(message, "message is null"); + completedBytes += max(message.serializedKeySize(), 0) + max(message.serializedValueSize(), 0); byte[] keyData = EMPTY_BYTE_ARRAY; - byte[] messageData = EMPTY_BYTE_ARRAY; - ByteBuffer key = record.key(); - if (key != null) { - keyData = new byte[key.remaining()]; - key.get(keyData); + if (message.key() != null) { + keyData = message.key(); } - ByteBuffer message = record.value(); - if (message != null) { - messageData = new byte[message.remaining()]; - message.get(messageData); + byte[] messageData = EMPTY_BYTE_ARRAY; + if (message.value() != null) { + messageData = message.value(); } Map currentRowValuesMap = new HashMap<>(); @@ -204,11 +167,8 @@ public class KafkaRecordSet if (columnHandle.isInternal()) { KafkaInternalFieldDescription fieldDescription = KafkaInternalFieldDescription.forColumnName(columnHandle.getName()); switch (fieldDescription) { - case SEGMENT_COUNT_FIELD: - currentRowValuesMap.put(columnHandle, longValueProvider(totalMessages)); - break; case PARTITION_OFFSET_FIELD: - currentRowValuesMap.put(columnHandle, longValueProvider(record.offset())); + currentRowValuesMap.put(columnHandle, longValueProvider(message.offset())); break; case MESSAGE_FIELD: currentRowValuesMap.put(columnHandle, bytesValueProvider(messageData)); @@ -229,13 +189,7 @@ public class KafkaRecordSet currentRowValuesMap.put(columnHandle, booleanValueProvider(!decodedValue.isPresent())); break; case PARTITION_ID_FIELD: - currentRowValuesMap.put(columnHandle, longValueProvider(split.getPartitionId())); - break; - case SEGMENT_START_FIELD: - currentRowValuesMap.put(columnHandle, longValueProvider(split.getStart())); - break; - case SEGMENT_END_FIELD: - currentRowValuesMap.put(columnHandle, longValueProvider(split.getEnd())); + currentRowValuesMap.put(columnHandle, longValueProvider(message.partition())); break; default: throw new IllegalArgumentException("unknown internal field " + fieldDescription); @@ -307,48 +261,7 @@ public class KafkaRecordSet @Override public void close() { - if (leaderKafkaConsumer != null) { - leaderKafkaConsumer.close(); - } - } - - private void openFetchRequest() - { - try { - if (recordIterator == null) { - log.debug("Fetching %d bytes from offset %d (%d - %d). %d messages read so far", KAFKA_READ_BUFFER_SIZE, cursorOffset, split.getStart(), split.getEnd(), totalMessages); - FetchRequest req = new FetchRequestBuilder() - .clientId("presto-worker-" + Thread.currentThread().getName()) - .addFetch(split.getTopicName(), split.getPartitionId(), cursorOffset, KAFKA_READ_BUFFER_SIZE) - .build(); - - // TODO - this should look at the actual node this is running on and prefer - // that copy if running locally. - look into NodeInfo - if (leaderKafkaConsumer == null) { - leaderKafkaConsumer = consumerManager.getSaslConsumer(split.getLeader()); - } - TopicPartition topicPartition = new TopicPartition(split.getTopicName(), split.getPartitionId()); - leaderKafkaConsumer.assign(Collections.singletonList(topicPartition)); - leaderKafkaConsumer.seek(topicPartition, cursorOffset); - ConsumerRecords records = leaderKafkaConsumer.poll(500); - recordIterator = records.records(topicPartition).iterator(); - } - } - catch (Exception e) { // Catch all exceptions because Kafka library is written in scala and checked exceptions are not declared in method signature. - if (e instanceof PrestoException) { - throw e; - } - throw new PrestoException( - KAFKA_SPLIT_ERROR, - format( - "Cannot read data from topic '%s', partition '%s', startOffset %s, endOffset %s, leader %s ", - split.getTopicName(), - split.getPartitionId(), - split.getStart(), - split.getEnd(), - split.getLeader()), - e); - } + kafkaConsumer.close(); } } } diff --git a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaRecordSetProvider.java b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaRecordSetProvider.java index 566fd6d5e..b58b9486c 100644 --- a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaRecordSetProvider.java +++ b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaRecordSetProvider.java @@ -41,14 +41,14 @@ import static java.util.Objects.requireNonNull; public class KafkaRecordSetProvider implements ConnectorRecordSetProvider { - private DispatchingRowDecoderFactory decoderFactory; - private final KafkaSimpleConsumerManager consumerManager; + private final DispatchingRowDecoderFactory decoderFactory; + private final KafkaConsumerFactory consumerFactory; @Inject - public KafkaRecordSetProvider(DispatchingRowDecoderFactory decoderFactory, KafkaSimpleConsumerManager consumerManager) + public KafkaRecordSetProvider(DispatchingRowDecoderFactory decoderFactory, KafkaConsumerFactory consumerFactory) { this.decoderFactory = requireNonNull(decoderFactory, "decoderFactory is null"); - this.consumerManager = requireNonNull(consumerManager, "consumerManager is null"); + this.consumerFactory = requireNonNull(consumerFactory, "consumerManager is null"); } @Override @@ -76,7 +76,7 @@ public class KafkaRecordSetProvider .filter(col -> !col.isKeyDecoder()) .collect(toImmutableSet())); - return new KafkaRecordSet(kafkaSplit, consumerManager, kafkaColumns, keyDecoder, messageDecoder); + return new KafkaRecordSet(kafkaSplit, consumerFactory, kafkaColumns, keyDecoder, messageDecoder); } private Map getDecoderParameters(Optional dataSchema) diff --git a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaSplit.java b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaSplit.java index a416ba139..ef8ffabef 100644 --- a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaSplit.java +++ b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaSplit.java @@ -25,13 +25,6 @@ import java.util.Optional; import static com.google.common.base.MoreObjects.toStringHelper; import static java.util.Objects.requireNonNull; -/** - * Represents a kafka specific {@link ConnectorSplit}. Each split is mapped to a segment file on disk (based off the segment offset start() and end() values) so that - * a partition can be processed by reading segment files from partition leader. Otherwise, a Kafka topic could only be processed along partition boundaries. - *

- * When planning to process a Kafka topic with Presto, using smaller than the recommended segment size (default is 1G) allows Presto to optimize early and process a topic - * with more workers in parallel. - */ public class KafkaSplit implements ConnectorSplit { @@ -41,8 +34,7 @@ public class KafkaSplit private final Optional keyDataSchemaContents; private final Optional messageDataSchemaContents; private final int partitionId; - private final long start; - private final long end; + private final Range messagesRange; private final HostAddress leader; @JsonCreator @@ -53,8 +45,7 @@ public class KafkaSplit @JsonProperty("keyDataSchemaContents") Optional keyDataSchemaContents, @JsonProperty("messageDataSchemaContents") Optional messageDataSchemaContents, @JsonProperty("partitionId") int partitionId, - @JsonProperty("start") long start, - @JsonProperty("end") long end, + @JsonProperty("messagesRange") Range messagesRange, @JsonProperty("leader") HostAddress leader) { this.topicName = requireNonNull(topicName, "topicName is null"); @@ -63,21 +54,8 @@ public class KafkaSplit this.keyDataSchemaContents = keyDataSchemaContents; this.messageDataSchemaContents = messageDataSchemaContents; this.partitionId = partitionId; - this.start = start; - this.end = end; - this.leader = requireNonNull(leader, "leader address is null"); - } - - @JsonProperty - public long getStart() - { - return start; - } - - @JsonProperty - public long getEnd() - { - return end; + this.messagesRange = requireNonNull(messagesRange, "messagesRange is null"); + this.leader = requireNonNull(leader, "leader is null"); } @JsonProperty @@ -116,6 +94,12 @@ public class KafkaSplit return partitionId; } + @JsonProperty + public Range getMessagesRange() + { + return messagesRange; + } + @JsonProperty public HostAddress getLeader() { @@ -150,8 +134,7 @@ public class KafkaSplit .add("keyDataSchemaContents", keyDataSchemaContents) .add("messageDataSchemaContents", messageDataSchemaContents) .add("partitionId", partitionId) - .add("start", start) - .add("end", end) + .add("messagesRange", messagesRange) .add("leader", leader) .toString(); } diff --git a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaSplitManager.java b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaSplitManager.java index eb480cbcc..b47e11fc1 100644 --- a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaSplitManager.java +++ b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaSplitManager.java @@ -14,26 +14,16 @@ package io.prestosql.plugin.kafka; import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.ImmutableSet; import com.google.common.io.CharStreams; -import io.airlift.log.Logger; import io.prestosql.spi.HostAddress; import io.prestosql.spi.PrestoException; import io.prestosql.spi.connector.ConnectorSession; -import io.prestosql.spi.connector.ConnectorSplit; import io.prestosql.spi.connector.ConnectorSplitManager; import io.prestosql.spi.connector.ConnectorSplitSource; import io.prestosql.spi.connector.ConnectorTableHandle; import io.prestosql.spi.connector.ConnectorTransactionHandle; import io.prestosql.spi.connector.FixedSplitSource; -import kafka.api.PartitionOffsetRequestInfo; -import kafka.common.TopicAndPartition; -import kafka.javaapi.OffsetRequest; -import kafka.javaapi.OffsetResponse; -import kafka.javaapi.consumer.SimpleConsumer; import org.apache.kafka.clients.consumer.KafkaConsumer; -import org.apache.kafka.common.Node; import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; @@ -46,11 +36,11 @@ import java.io.InputStreamReader; import java.net.MalformedURLException; import java.net.URI; import java.net.URL; -import java.nio.ByteBuffer; import java.util.List; -import java.util.Set; -import java.util.concurrent.ThreadLocalRandom; +import java.util.Map; +import java.util.Optional; +import static com.google.common.collect.ImmutableList.toImmutableList; import static io.prestosql.plugin.kafka.KafkaErrorCode.KAFKA_SPLIT_ERROR; import static io.prestosql.spi.StandardErrorCode.GENERIC_INTERNAL_ERROR; import static java.lang.String.format; @@ -58,58 +48,54 @@ import static java.nio.charset.StandardCharsets.UTF_8; import static java.util.Locale.ENGLISH; import static java.util.Objects.requireNonNull; -/** - * Kafka specific implementation of {@link ConnectorSplitManager}. - */ public class KafkaSplitManager implements ConnectorSplitManager { - private static final Logger log = Logger.get(KafkaSplitManager.class); - - private final KafkaSimpleConsumerManager consumerManager; - private final Set nodes; + private final KafkaConsumerFactory consumerFactory; + private final int messagesPerSplit; @Inject - public KafkaSplitManager( - KafkaConnectorConfig kafkaConnectorConfig, - KafkaSimpleConsumerManager consumerManager) + public KafkaSplitManager(KafkaConsumerFactory consumerFactory, KafkaConfig kafkaConfig) { - this.consumerManager = requireNonNull(consumerManager, "consumerManager is null"); - - requireNonNull(kafkaConnectorConfig, "kafkaConfig is null"); - this.nodes = ImmutableSet.copyOf(kafkaConnectorConfig.getNodes()); + this.consumerFactory = requireNonNull(consumerFactory, "consumerManager is null"); + messagesPerSplit = requireNonNull(kafkaConfig, "kafkaConfig is null").getMessagesPerSplit(); } @Override public ConnectorSplitSource getSplits(ConnectorTransactionHandle transaction, ConnectorSession session, ConnectorTableHandle table, SplitSchedulingStrategy splitSchedulingStrategy) { KafkaTableHandle kafkaTableHandle = (KafkaTableHandle) table; - try (KafkaConsumer kafkaConsumer = consumerManager.getSaslConsumer(selectRandom(nodes))) { + try (KafkaConsumer kafkaConsumer = consumerFactory.create()) { List partitionInfos = kafkaConsumer.partitionsFor(kafkaTableHandle.getTopicName()); - ImmutableList.Builder splits = ImmutableList.builder(); + List topicPartitions = partitionInfos.stream() + .map(KafkaSplitManager::toTopicPartition) + .collect(toImmutableList()); + Map partitionBeginOffsets = kafkaConsumer.beginningOffsets(topicPartitions); + Map partitionEndOffsets = kafkaConsumer.endOffsets(topicPartitions); + + ImmutableList.Builder splits = ImmutableList.builder(); + Optional keyDataSchemaContents = kafkaTableHandle.getKeyDataSchemaLocation() + .map(KafkaSplitManager::readSchema); + Optional messageDataSchemaContents = kafkaTableHandle.getMessageDataSchemaLocation() + .map(KafkaSplitManager::readSchema); for (PartitionInfo partitionInfo : partitionInfos) { - log.debug("Adding Partition %s/%s", partitionInfo.topic(), partitionInfo.partition()); - Node leader = partitionInfo.leader(); - HostAddress partitionLeader = HostAddress.fromParts(leader.host(), leader.port()); - TopicPartition topicPartition = new TopicPartition(partitionInfo.topic(), partitionInfo.partition()); - kafkaConsumer.assign(ImmutableList.of(topicPartition)); - long beginOffset = kafkaConsumer.beginningOffsets(ImmutableList.of(topicPartition)).values().iterator().next(); - long endOffset = kafkaConsumer.endOffsets(ImmutableList.of(topicPartition)).values().iterator().next(); - KafkaSplit split = new KafkaSplit( - topicPartition.topic(), - kafkaTableHandle.getKeyDataFormat(), - kafkaTableHandle.getMessageDataFormat(), - kafkaTableHandle.getKeyDataSchemaLocation().map(KafkaSplitManager::readSchema), - kafkaTableHandle.getMessageDataSchemaLocation().map(KafkaSplitManager::readSchema), - topicPartition.partition(), - beginOffset, - endOffset, - partitionLeader); - splits.add(split); + TopicPartition topicPartition = toTopicPartition(partitionInfo); + HostAddress leader = HostAddress.fromParts(partitionInfo.leader().host(), partitionInfo.leader().port()); + new Range(partitionBeginOffsets.get(topicPartition), partitionEndOffsets.get(topicPartition)) + .partition(messagesPerSplit).stream() + .map(range -> new KafkaSplit( + kafkaTableHandle.getTopicName(), + kafkaTableHandle.getKeyDataFormat(), + kafkaTableHandle.getMessageDataFormat(), + keyDataSchemaContents, + messageDataSchemaContents, + partitionInfo.partition(), + range, + leader)) + .forEach(splits::add); } - return new FixedSplitSource(splits.build()); } catch (Exception e) { // Catch all exceptions because Kafka library is written in scala and checked exceptions are not declared in method signature. @@ -120,6 +106,11 @@ public class KafkaSplitManager } } + private static TopicPartition toTopicPartition(PartitionInfo partitionInfo) + { + return new TopicPartition(partitionInfo.topic(), partitionInfo.partition()); + } + private static String readSchema(String dataSchemaLocation) { InputStream inputStream = null; @@ -160,6 +151,7 @@ public class KafkaSplitManager private static boolean isURI(String location) { try { + //noinspection ResultOfMethodCallIgnored URI.create(location); } catch (Exception e) { @@ -167,31 +159,4 @@ public class KafkaSplitManager } return true; } - - private static long[] findAllOffsets(SimpleConsumer consumer, String topicName, int partitionId) - { - TopicAndPartition topicAndPartition = new TopicAndPartition(topicName, partitionId); - - // The API implies that this will always return all of the offsets. So it seems a partition can not have - // more than Integer.MAX_VALUE-1 segments. - // - // This also assumes that the lowest value returned will be the first segment available. So if segments have been dropped off, this value - // should not be 0. - PartitionOffsetRequestInfo partitionOffsetRequestInfo = new PartitionOffsetRequestInfo(kafka.api.OffsetRequest.LatestTime(), Integer.MAX_VALUE); - OffsetRequest offsetRequest = new OffsetRequest(ImmutableMap.of(topicAndPartition, partitionOffsetRequestInfo), kafka.api.OffsetRequest.CurrentVersion(), consumer.clientId()); - OffsetResponse offsetResponse = consumer.getOffsetsBefore(offsetRequest); - - if (offsetResponse.hasError()) { - short errorCode = offsetResponse.errorCode(topicName, partitionId); - throw new RuntimeException("could not fetch data from Kafka, error code is '" + errorCode + "'"); - } - - return offsetResponse.offsets(topicName, partitionId); - } - - private static T selectRandom(Iterable iterable) - { - List list = ImmutableList.copyOf(iterable); - return list.get(ThreadLocalRandom.current().nextInt(list.size())); - } } diff --git a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/Range.java b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/Range.java new file mode 100644 index 000000000..426c7969d --- /dev/null +++ b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/Range.java @@ -0,0 +1,68 @@ +/* + * 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.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.ImmutableList; + +import java.util.List; + +import static com.google.common.base.MoreObjects.toStringHelper; +import static java.lang.Math.min; + +public class Range +{ + private final long begin; // inclusive + private final long end; // exclusive + + @JsonCreator + public Range(@JsonProperty long begin, @JsonProperty long end) + { + this.begin = begin; + this.end = end; + } + + @JsonProperty + public long getBegin() + { + return begin; + } + + @JsonProperty + public long getEnd() + { + return end; + } + + public List partition(int partitionSize) + { + ImmutableList.Builder partitions = ImmutableList.builder(); + long position = begin; + while (position <= end) { + partitions.add(new Range(position, min(position + partitionSize, end))); + position += partitionSize; + } + return partitions.build(); + } + + @Override + public String toString() + { + return toStringHelper(this) + .add("begin", begin) + .add("end", end) + .toString(); + } +} diff --git a/presto-kafka/src/test/java/io/prestosql/plugin/kafka/TestKafkaConfig.java b/presto-kafka/src/test/java/io/prestosql/plugin/kafka/TestKafkaConfig.java new file mode 100644 index 000000000..cae9beded --- /dev/null +++ b/presto-kafka/src/test/java/io/prestosql/plugin/kafka/TestKafkaConfig.java @@ -0,0 +1,68 @@ +/* + * 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 org.testng.annotations.Test; + +import java.io.File; +import java.util.Map; + +import static io.airlift.configuration.testing.ConfigAssertions.assertFullMapping; +import static io.airlift.configuration.testing.ConfigAssertions.assertRecordedDefaults; +import static io.airlift.configuration.testing.ConfigAssertions.recordDefaults; + +public class TestKafkaConfig +{ + @Test + public void testDefaults() + { + assertRecordedDefaults(recordDefaults(KafkaConfig.class) + .setNodes("") + .setKafkaConnectTimeout("10s") + .setKafkaBufferSize("64kB") + .setDefaultSchema("default") + .setTableNames("") + .setTableDescriptionDir(new File("etc/kafka/")) + .setHideInternalColumns(true) + .setMessagesPerSplit(100_000)); + } + + @Test + public void testExplicitPropertyMappings() + { + Map properties = new ImmutableMap.Builder() + .put("kafka.table-description-dir", "/var/lib/kafka") + .put("kafka.table-names", "table1, table2, table3") + .put("kafka.default-schema", "kafka") + .put("kafka.nodes", "localhost:12345,localhost:23456") + .put("kafka.connect-timeout", "1h") + .put("kafka.buffer-size", "1MB") + .put("kafka.hide-internal-columns", "false") + .put("kafka.messages-per-split", "1") + .build(); + + KafkaConfig expected = new KafkaConfig() + .setTableDescriptionDir(new File("/var/lib/kafka")) + .setTableNames("table1, table2, table3") + .setDefaultSchema("kafka") + .setNodes("localhost:12345, localhost:23456") + .setKafkaConnectTimeout("1h") + .setKafkaBufferSize("1MB") + .setHideInternalColumns(false) + .setMessagesPerSplit(1); + + assertFullMapping(properties, expected); + } +} diff --git a/presto-kafka/src/test/java/io/prestosql/plugin/kafka/util/JsonSerializer.java b/presto-kafka/src/test/java/io/prestosql/plugin/kafka/util/JsonSerializer.java new file mode 100644 index 000000000..46f8eafa6 --- /dev/null +++ b/presto-kafka/src/test/java/io/prestosql/plugin/kafka/util/JsonSerializer.java @@ -0,0 +1,47 @@ +/* + * 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.util; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.kafka.common.serialization.Serializer; + +import java.util.Map; + +public class JsonSerializer + implements Serializer +{ + private final ObjectMapper objectMapper = new ObjectMapper(); + + @Override + public void configure(Map props, boolean isKey) + { + } + + @Override + public byte[] serialize(String topic, Object data) + { + try { + return objectMapper.writeValueAsBytes(data); + } + catch (JsonProcessingException e) { + throw new RuntimeException(e); + } + } + + @Override + public void close() + { + } +} diff --git a/presto-kafka/src/test/java/io/prestosql/plugin/kafka/util/NumberPartitioner.java b/presto-kafka/src/test/java/io/prestosql/plugin/kafka/util/NumberPartitioner.java index db7ed465a..63fd9feff 100644 --- a/presto-kafka/src/test/java/io/prestosql/plugin/kafka/util/NumberPartitioner.java +++ b/presto-kafka/src/test/java/io/prestosql/plugin/kafka/util/NumberPartitioner.java @@ -13,26 +13,28 @@ */ package io.prestosql.plugin.kafka.util; -import kafka.producer.Partitioner; -import kafka.utils.VerifiableProperties; +import org.apache.kafka.clients.producer.Partitioner; +import org.apache.kafka.common.Cluster; + +import java.util.Map; import static java.lang.Math.toIntExact; public class NumberPartitioner implements Partitioner { - @SuppressWarnings("UnusedParameters") - public NumberPartitioner(VerifiableProperties properties) - { - // constructor required by Kafka - } + @Override + public void configure(Map configs) {} @Override - public int partition(Object key, int numPartitions) + public int partition(String topic, Object key, byte[] keyBytes, Object value, byte[] valueBytes, Cluster cluster) { if (key instanceof Number) { - return toIntExact(((Number) key).longValue() % numPartitions); + return toIntExact(((Number) key).longValue() % cluster.partitionCountForTopic(topic)); } return 0; } + + @Override + public void close() {} } diff --git a/presto-kafka/src/test/java/io/prestosql/plugin/kafka/util/TestingKafka.java b/presto-kafka/src/test/java/io/prestosql/plugin/kafka/util/TestingKafka.java new file mode 100644 index 000000000..a7a0674bf --- /dev/null +++ b/presto-kafka/src/test/java/io/prestosql/plugin/kafka/util/TestingKafka.java @@ -0,0 +1,92 @@ +/* + * 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.util; + +import com.google.common.collect.ImmutableMap; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.common.serialization.LongSerializer; +import org.testcontainers.containers.KafkaContainer; + +import java.io.Closeable; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +import static io.prestosql.plugin.kafka.util.TestUtils.toProperties; +import static org.testcontainers.containers.KafkaContainer.KAFKA_PORT; + +public class TestingKafka + implements Closeable +{ + private final KafkaContainer container; + + public TestingKafka() + { + container = new KafkaContainer("5.2.1"); + } + + public void start() + { + container.start(); + } + + @Override + public void close() + { + container.close(); + } + + public void createTopics(String topic) + { + createTopics(2, 1, topic); + } + + private void createTopics(int partitions, int replication, String topic) + { + try { + List command = new ArrayList<>(); + command.add("kafka-topics"); + command.add("--partitions"); + command.add(Integer.toString(partitions)); + command.add("--replication-factor"); + command.add(Integer.toString(replication)); + command.add("--topic"); + command.add(topic); + + container.execInContainer(command.toArray(new String[0])); + } + catch (Exception e) { + throw new RuntimeException(e); + } + } + + public String getConnectString() + { + return container.getContainerIpAddress() + ":" + container.getMappedPort(KAFKA_PORT); + } + + public KafkaProducer createProducer() + { + Map properties = ImmutableMap.builder() + .put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, getConnectString()) + .put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, LongSerializer.class.getName()) + .put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, JsonSerializer.class.getName()) + .put(ProducerConfig.PARTITIONER_CLASS_CONFIG, NumberPartitioner.class.getName()) + .put(ProducerConfig.ACKS_CONFIG, "1") + .build(); + + return new KafkaProducer<>(toProperties(properties)); + } +} -- Gitee From a381d5704cfdc0b26992e30fe81ad5d521f5c326 Mon Sep 17 00:00:00 2001 From: XHLY Date: Wed, 22 Jan 2020 02:09:29 +0800 Subject: [PATCH 3/8] Add predicate of internal Kafka columns supporting push-down. MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Add push-down for following Kafka internal columns * _timestamp * _partition_offset * _partition_id If predicate specifies lower bound on _timestamp column (_timestamp >     XXXX), it is always pushed down. The upper bound predicate is pushed down only for topics using ``LogAppendTime`` mode. For topics using ``CreateTime`` mode, upper bound pushdown must be explicitly     allowed via ``kafka.timestamp-upper-bound-force-push-down-enabled`` config property     or ``timestamp_upper_bound_force_push_down_enabled`` session property. one more thing: Allow to extend Kafka connector. Implement inserts for Kafka connector. --- .../CarbondataConnectorFactory.java | 8 +- .../core/plugin/mpp/MppConnectorFactory.java | 8 +- pom.xml | 7 + .../prestosql/plugin/hive/HiveConnector.java | 2 +- .../plugin/hive/HiveConnectorFactory.java | 8 +- .../plugin/hive/TestHiveConnectorFactory.java | 4 +- presto-kafka/pom.xml | 75 ++-- .../plugin/kafka/KafkaAdminFactory.java | 54 +++ .../plugin/kafka/KafkaColumnHandle.java | 56 ++- .../prestosql/plugin/kafka/KafkaConfig.java | 35 +- .../plugin/kafka/KafkaConnector.java | 40 ++- .../plugin/kafka/KafkaConnectorFactory.java | 31 +- .../plugin/kafka/KafkaConnectorModule.java | 36 +- .../plugin/kafka/KafkaConsumerFactory.java | 43 +-- .../plugin/kafka/KafkaConsumerModule.java | 28 ++ .../plugin/kafka/KafkaErrorCode.java | 6 +- .../plugin/kafka/KafkaFilterManager.java | 334 ++++++++++++++++++ .../plugin/kafka/KafkaFilteringResult.java | 53 +++ .../plugin/kafka/KafkaHandleResolver.java | 7 + .../kafka/KafkaInternalFieldDescription.java | 133 ------- .../kafka/KafkaInternalFieldManager.java | 192 ++++++++++ .../prestosql/plugin/kafka/KafkaMetadata.java | 183 +++++++--- .../prestosql/plugin/kafka/KafkaPageSink.java | 129 +++++++ .../plugin/kafka/KafkaPageSinkProvider.java | 109 ++++++ .../prestosql/plugin/kafka/KafkaPlugin.java | 42 +-- .../plugin/kafka/KafkaProducerModule.java | 28 ++ .../plugin/kafka/KafkaRecordSet.java | 130 ++++++- .../plugin/kafka/KafkaRecordSetProvider.java | 4 +- .../plugin/kafka/KafkaSessionProperties.java | 54 +++ .../plugin/kafka/KafkaSplitManager.java | 20 +- .../kafka/KafkaTableDescriptionSupplier.java | 27 +- .../plugin/kafka/KafkaTableHandle.java | 44 ++- .../kafka/KafkaTopicFieldDescription.java | 5 +- .../MapBasedTableDescriptionSupplier.java | 47 +++ .../kafka/PlainTextKafkaConsumerFactory.java | 63 ++++ .../kafka/PlainTextKafkaProducerFactory.java | 58 +++ .../kafka/TableDescriptionSupplier.java | 27 ++ .../kafka/encoder/AbstractRowEncoder.java | 215 +++++++++++ .../encoder/DispatchingRowEncoderFactory.java | 43 +++ .../kafka/encoder/EncoderColumnHandle.java | 33 ++ .../plugin/kafka/encoder/EncoderModule.java | 45 +++ .../plugin/kafka/encoder/RowEncoder.java | 32 ++ .../kafka/encoder/RowEncoderFactory.java | 24 ++ .../kafka/encoder/avro/AvroRowEncoder.java | 160 +++++++++ .../encoder/avro/AvroRowEncoderFactory.java | 38 ++ .../kafka/encoder/csv/CsvRowEncoder.java | 142 ++++++++ .../encoder/csv/CsvRowEncoderFactory.java | 32 ++ .../kafka/encoder/json/JsonRowEncoder.java | 150 ++++++++ .../encoder/json/JsonRowEncoderFactory.java | 45 +++ .../kafka/encoder/raw/RawRowEncoder.java | 329 +++++++++++++++++ .../encoder/raw/RawRowEncoderFactory.java | 32 ++ .../kafka/ConfigurationAwareModules.java | 46 +++ .../plugin/kafka/TestKafkaConfig.java | 10 +- .../kafka/encoder/json/TestJsonEncoder.java | 71 ++++ .../encoder/raw/TestRawEncoderMapping.java | 77 ++++ .../write_test/all_datatypes_avro.json | 42 +++ .../all_datatypes_avro_key_schema.avsc | 14 + .../all_datatypes_avro_message_schema.avsc | 29 ++ .../write_test/all_datatypes_csv.json | 55 +++ .../write_test/all_datatypes_json.json | 55 +++ .../write_test/all_datatypes_raw.json | 63 ++++ .../java/io/prestosql/event/QueryMonitor.java | 2 +- presto-plugin-toolkit/pom.xml | 5 + ...ClassLoaderSafeConnectorAccessControl.java | 315 +++++++++++++++++ .../ClassLoaderSafeConnectorMetadata.java | 7 +- .../ClassLoaderSafeConnectorPageSink.java | 7 +- ...ssLoaderSafeConnectorPageSinkProvider.java | 7 +- ...LoaderSafeConnectorPageSourceProvider.java | 7 +- ...sLoaderSafeConnectorRecordSetProvider.java | 60 ++++ .../ClassLoaderSafeConnectorSplitManager.java | 7 +- ...assLoaderSafeNodePartitioningProvider.java | 7 +- .../classloader/ClassLoaderSafeRecordSet.java | 56 +++ .../ClassLoaderSafeConnectorMetadataTest.java | 2 +- ...aderSafeConnectorPageSinkProviderTest.java | 2 +- .../ClassLoaderSafeConnectorPageSinkTest.java | 2 +- ...erSafeConnectorPageSourceProviderTest.java | 2 +- ...ssLoaderSafeConnectorSplitManagerTest.java | 2 +- ...oaderSafeNodePartitioningProviderTest.java | 2 +- .../TestClassLoaderSafeWrappers.java | 2 +- .../spi/connector/ConnectorMetadata.java | 25 +- .../io/prestosql/spi/type/TypeSignature.java | 7 +- 81 files changed, 3931 insertions(+), 507 deletions(-) create mode 100644 presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaAdminFactory.java create mode 100644 presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaConsumerModule.java create mode 100644 presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaFilterManager.java create mode 100644 presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaFilteringResult.java delete mode 100644 presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaInternalFieldDescription.java create mode 100644 presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaInternalFieldManager.java create mode 100644 presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaPageSink.java create mode 100644 presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaPageSinkProvider.java create mode 100644 presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaProducerModule.java create mode 100644 presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaSessionProperties.java create mode 100644 presto-kafka/src/main/java/io/prestosql/plugin/kafka/MapBasedTableDescriptionSupplier.java create mode 100644 presto-kafka/src/main/java/io/prestosql/plugin/kafka/PlainTextKafkaConsumerFactory.java create mode 100644 presto-kafka/src/main/java/io/prestosql/plugin/kafka/PlainTextKafkaProducerFactory.java create mode 100644 presto-kafka/src/main/java/io/prestosql/plugin/kafka/TableDescriptionSupplier.java create mode 100644 presto-kafka/src/main/java/io/prestosql/plugin/kafka/encoder/AbstractRowEncoder.java create mode 100644 presto-kafka/src/main/java/io/prestosql/plugin/kafka/encoder/DispatchingRowEncoderFactory.java create mode 100644 presto-kafka/src/main/java/io/prestosql/plugin/kafka/encoder/EncoderColumnHandle.java create mode 100644 presto-kafka/src/main/java/io/prestosql/plugin/kafka/encoder/EncoderModule.java create mode 100644 presto-kafka/src/main/java/io/prestosql/plugin/kafka/encoder/RowEncoder.java create mode 100644 presto-kafka/src/main/java/io/prestosql/plugin/kafka/encoder/RowEncoderFactory.java create mode 100644 presto-kafka/src/main/java/io/prestosql/plugin/kafka/encoder/avro/AvroRowEncoder.java create mode 100644 presto-kafka/src/main/java/io/prestosql/plugin/kafka/encoder/avro/AvroRowEncoderFactory.java create mode 100644 presto-kafka/src/main/java/io/prestosql/plugin/kafka/encoder/csv/CsvRowEncoder.java create mode 100644 presto-kafka/src/main/java/io/prestosql/plugin/kafka/encoder/csv/CsvRowEncoderFactory.java create mode 100644 presto-kafka/src/main/java/io/prestosql/plugin/kafka/encoder/json/JsonRowEncoder.java create mode 100644 presto-kafka/src/main/java/io/prestosql/plugin/kafka/encoder/json/JsonRowEncoderFactory.java create mode 100644 presto-kafka/src/main/java/io/prestosql/plugin/kafka/encoder/raw/RawRowEncoder.java create mode 100644 presto-kafka/src/main/java/io/prestosql/plugin/kafka/encoder/raw/RawRowEncoderFactory.java create mode 100644 presto-kafka/src/test/java/io/prestosql/plugin/kafka/ConfigurationAwareModules.java create mode 100644 presto-kafka/src/test/java/io/prestosql/plugin/kafka/encoder/json/TestJsonEncoder.java create mode 100644 presto-kafka/src/test/java/io/prestosql/plugin/kafka/encoder/raw/TestRawEncoderMapping.java create mode 100644 presto-kafka/src/test/resources/write_test/all_datatypes_avro.json create mode 100644 presto-kafka/src/test/resources/write_test/all_datatypes_avro_key_schema.avsc create mode 100644 presto-kafka/src/test/resources/write_test/all_datatypes_avro_message_schema.avsc create mode 100644 presto-kafka/src/test/resources/write_test/all_datatypes_csv.json create mode 100644 presto-kafka/src/test/resources/write_test/all_datatypes_json.json create mode 100644 presto-kafka/src/test/resources/write_test/all_datatypes_raw.json create mode 100644 presto-plugin-toolkit/src/main/java/io/prestosql/plugin/base/classloader/ClassLoaderSafeConnectorAccessControl.java rename {presto-spi/src/main/java/io/prestosql/spi/connector => presto-plugin-toolkit/src/main/java/io/prestosql/plugin/base}/classloader/ClassLoaderSafeConnectorMetadata.java (99%) rename {presto-spi/src/main/java/io/prestosql/spi/connector => presto-plugin-toolkit/src/main/java/io/prestosql/plugin/base}/classloader/ClassLoaderSafeConnectorPageSink.java (95%) rename {presto-spi/src/main/java/io/prestosql/spi/connector => presto-plugin-toolkit/src/main/java/io/prestosql/plugin/base}/classloader/ClassLoaderSafeConnectorPageSinkProvider.java (95%) rename {presto-spi/src/main/java/io/prestosql/spi/connector => presto-plugin-toolkit/src/main/java/io/prestosql/plugin/base}/classloader/ClassLoaderSafeConnectorPageSourceProvider.java (92%) create mode 100644 presto-plugin-toolkit/src/main/java/io/prestosql/plugin/base/classloader/ClassLoaderSafeConnectorRecordSetProvider.java rename {presto-spi/src/main/java/io/prestosql/spi/connector => presto-plugin-toolkit/src/main/java/io/prestosql/plugin/base}/classloader/ClassLoaderSafeConnectorSplitManager.java (94%) rename {presto-spi/src/main/java/io/prestosql/spi/connector => presto-plugin-toolkit/src/main/java/io/prestosql/plugin/base}/classloader/ClassLoaderSafeNodePartitioningProvider.java (93%) create mode 100644 presto-plugin-toolkit/src/main/java/io/prestosql/plugin/base/classloader/ClassLoaderSafeRecordSet.java rename {presto-spi/src/test/java/io/prestosql/spi/connector => presto-plugin-toolkit/src/test/java/io/prestosql/plugin/base}/classloader/ClassLoaderSafeConnectorMetadataTest.java (99%) rename {presto-spi/src/test/java/io/prestosql/spi/connector => presto-plugin-toolkit/src/test/java/io/prestosql/plugin/base}/classloader/ClassLoaderSafeConnectorPageSinkProviderTest.java (99%) rename {presto-spi/src/test/java/io/prestosql/spi/connector => presto-plugin-toolkit/src/test/java/io/prestosql/plugin/base}/classloader/ClassLoaderSafeConnectorPageSinkTest.java (99%) rename {presto-spi/src/test/java/io/prestosql/spi/connector => presto-plugin-toolkit/src/test/java/io/prestosql/plugin/base}/classloader/ClassLoaderSafeConnectorPageSourceProviderTest.java (99%) rename {presto-spi/src/test/java/io/prestosql/spi/connector => presto-plugin-toolkit/src/test/java/io/prestosql/plugin/base}/classloader/ClassLoaderSafeConnectorSplitManagerTest.java (99%) rename {presto-spi/src/test/java/io/prestosql/spi/connector => presto-plugin-toolkit/src/test/java/io/prestosql/plugin/base}/classloader/ClassLoaderSafeNodePartitioningProviderTest.java (99%) rename {presto-spi/src/test/java/io/prestosql/spi/connector => presto-plugin-toolkit/src/test/java/io/prestosql/plugin/base}/classloader/TestClassLoaderSafeWrappers.java (97%) diff --git a/hetu-carbondata/src/main/java/io/hetu/core/plugin/carbondata/CarbondataConnectorFactory.java b/hetu-carbondata/src/main/java/io/hetu/core/plugin/carbondata/CarbondataConnectorFactory.java index 98cc6deb3..dae2f8069 100755 --- a/hetu-carbondata/src/main/java/io/hetu/core/plugin/carbondata/CarbondataConnectorFactory.java +++ b/hetu-carbondata/src/main/java/io/hetu/core/plugin/carbondata/CarbondataConnectorFactory.java @@ -23,6 +23,10 @@ import io.airlift.event.client.EventModule; import io.airlift.json.JsonModule; import io.airlift.units.DataSize; import io.hetu.core.plugin.carbondata.impl.CarbondataTableConfig; +import io.prestosql.plugin.base.classloader.ClassLoaderSafeConnectorPageSinkProvider; +import io.prestosql.plugin.base.classloader.ClassLoaderSafeConnectorPageSourceProvider; +import io.prestosql.plugin.base.classloader.ClassLoaderSafeConnectorSplitManager; +import io.prestosql.plugin.base.classloader.ClassLoaderSafeNodePartitioningProvider; import io.prestosql.plugin.base.jmx.MBeanServerModule; import io.prestosql.plugin.hive.ConnectorObjectNameGeneratorModule; import io.prestosql.plugin.hive.HiveAnalyzeProperties; @@ -55,10 +59,6 @@ import io.prestosql.spi.connector.ConnectorNodePartitioningProvider; import io.prestosql.spi.connector.ConnectorPageSinkProvider; import io.prestosql.spi.connector.ConnectorPageSourceProvider; import io.prestosql.spi.connector.ConnectorSplitManager; -import io.prestosql.spi.connector.classloader.ClassLoaderSafeConnectorPageSinkProvider; -import io.prestosql.spi.connector.classloader.ClassLoaderSafeConnectorPageSourceProvider; -import io.prestosql.spi.connector.classloader.ClassLoaderSafeConnectorSplitManager; -import io.prestosql.spi.connector.classloader.ClassLoaderSafeNodePartitioningProvider; import io.prestosql.spi.procedure.Procedure; import io.prestosql.spi.type.TypeManager; import org.apache.carbondata.hive.CarbonHiveSerDe; diff --git a/hetu-mpp/src/main/java/io/hetu/core/plugin/mpp/MppConnectorFactory.java b/hetu-mpp/src/main/java/io/hetu/core/plugin/mpp/MppConnectorFactory.java index 6d1435785..fc3fd2d4d 100644 --- a/hetu-mpp/src/main/java/io/hetu/core/plugin/mpp/MppConnectorFactory.java +++ b/hetu-mpp/src/main/java/io/hetu/core/plugin/mpp/MppConnectorFactory.java @@ -21,6 +21,10 @@ import io.airlift.bootstrap.Bootstrap; import io.airlift.bootstrap.LifeCycleManager; import io.airlift.event.client.EventModule; import io.airlift.json.JsonModule; +import io.prestosql.plugin.base.classloader.ClassLoaderSafeConnectorPageSinkProvider; +import io.prestosql.plugin.base.classloader.ClassLoaderSafeConnectorPageSourceProvider; +import io.prestosql.plugin.base.classloader.ClassLoaderSafeConnectorSplitManager; +import io.prestosql.plugin.base.classloader.ClassLoaderSafeNodePartitioningProvider; import io.prestosql.plugin.base.jmx.MBeanServerModule; import io.prestosql.plugin.hive.ConnectorObjectNameGeneratorModule; import io.prestosql.plugin.hive.HiveAnalyzeProperties; @@ -53,10 +57,6 @@ import io.prestosql.spi.connector.ConnectorNodePartitioningProvider; import io.prestosql.spi.connector.ConnectorPageSinkProvider; import io.prestosql.spi.connector.ConnectorPageSourceProvider; import io.prestosql.spi.connector.ConnectorSplitManager; -import io.prestosql.spi.connector.classloader.ClassLoaderSafeConnectorPageSinkProvider; -import io.prestosql.spi.connector.classloader.ClassLoaderSafeConnectorPageSourceProvider; -import io.prestosql.spi.connector.classloader.ClassLoaderSafeConnectorSplitManager; -import io.prestosql.spi.connector.classloader.ClassLoaderSafeNodePartitioningProvider; import io.prestosql.spi.heuristicindex.IndexClient; import io.prestosql.spi.procedure.Procedure; import io.prestosql.spi.type.TypeManager; diff --git a/pom.xml b/pom.xml index 84168c5d5..513b834e0 100644 --- a/pom.xml +++ b/pom.xml @@ -63,6 +63,7 @@ 1.66 0.7.0 5.0.3 + 2.4.1 io.hetu.core @@ -176,6 +161,11 @@ + + org.assertj + assertj-core + + io.airlift log-manager @@ -194,31 +184,6 @@ - - com.101tec - zkclient - 0.10 - runtime - - - log4j - log4j - - - org.slf4j - slf4j-log4j12 - - - zookeeper - org.apache.zookeeper - - - netty - io.netty - - - - org.testng diff --git a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaAdminFactory.java b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaAdminFactory.java new file mode 100644 index 000000000..98554486c --- /dev/null +++ b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaAdminFactory.java @@ -0,0 +1,54 @@ +/* + * 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 io.prestosql.spi.HostAddress; +import org.apache.kafka.clients.admin.AdminClient; +import org.apache.kafka.clients.admin.KafkaAdminClient; + +import javax.inject.Inject; + +import java.util.Properties; +import java.util.Set; + +import static java.util.Objects.requireNonNull; +import static java.util.stream.Collectors.joining; +import static org.apache.kafka.clients.consumer.ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG; + +public class KafkaAdminFactory +{ + private final Set nodes; + + @Inject + public KafkaAdminFactory(KafkaConfig kafkaConfig) + { + requireNonNull(kafkaConfig, "kafkaConfig is null"); + nodes = kafkaConfig.getNodes(); + } + + public AdminClient create() + { + return KafkaAdminClient.create(configure()); + } + + public Properties configure() + { + Properties properties = new Properties(); + properties.setProperty(BOOTSTRAP_SERVERS_CONFIG, nodes.stream() + .map(HostAddress::toString) + .collect(joining(","))); + return properties; + } +} diff --git a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaColumnHandle.java b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaColumnHandle.java index 64a53c8bf..98a685d31 100644 --- a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaColumnHandle.java +++ b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaColumnHandle.java @@ -16,6 +16,7 @@ package io.prestosql.plugin.kafka; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import io.prestosql.decoder.DecoderColumnHandle; +import io.prestosql.plugin.kafka.encoder.EncoderColumnHandle; import io.prestosql.spi.connector.ColumnMetadata; import io.prestosql.spi.type.Type; @@ -24,14 +25,9 @@ import java.util.Objects; import static com.google.common.base.MoreObjects.toStringHelper; import static java.util.Objects.requireNonNull; -/** - * Kafka specific connector column handle. - */ public final class KafkaColumnHandle - implements DecoderColumnHandle, Comparable + implements EncoderColumnHandle, DecoderColumnHandle { - private final int ordinalPosition; - /** * Column Name */ @@ -43,24 +39,24 @@ public final class KafkaColumnHandle private final Type type; /** - * Mapping hint for the decoder. Can be null. + * Mapping hint for the codec. Can be null. */ private final String mapping; /** - * Data format to use (selects the decoder). Can be null. + * Data format to use (selects the codec). Can be null. */ private final String dataFormat; /** - * Additional format hint for the selected decoder. Selects a decoder subtype (e.g. which timestamp decoder). + * Additional format hint for the selected codec. Selects a codec subtype (e.g. which timestamp codec). */ private final String formatHint; /** - * True if the key decoder should be used, false if the message decoder should be used. + * True if the key codec should be used, false if the message codec should be used. */ - private final boolean keyDecoder; + private final boolean keyCodec; /** * True if the column should be hidden. @@ -74,33 +70,25 @@ public final class KafkaColumnHandle @JsonCreator public KafkaColumnHandle( - @JsonProperty("ordinalPosition") int ordinalPosition, @JsonProperty("name") String name, @JsonProperty("type") Type type, @JsonProperty("mapping") String mapping, @JsonProperty("dataFormat") String dataFormat, @JsonProperty("formatHint") String formatHint, - @JsonProperty("keyDecoder") boolean keyDecoder, + @JsonProperty("keyCodec") boolean keyCodec, @JsonProperty("hidden") boolean hidden, @JsonProperty("internal") boolean internal) { - this.ordinalPosition = ordinalPosition; this.name = requireNonNull(name, "name is null"); this.type = requireNonNull(type, "type is null"); this.mapping = mapping; this.dataFormat = dataFormat; this.formatHint = formatHint; - this.keyDecoder = keyDecoder; + this.keyCodec = keyCodec; this.hidden = hidden; this.internal = internal; } - @JsonProperty - public int getOrdinalPosition() - { - return ordinalPosition; - } - @Override @JsonProperty public String getName() @@ -137,9 +125,9 @@ public final class KafkaColumnHandle } @JsonProperty - public boolean isKeyDecoder() + public boolean isKeyCodec() { - return keyDecoder; + return keyCodec; } @JsonProperty @@ -157,13 +145,17 @@ public final class KafkaColumnHandle ColumnMetadata getColumnMetadata() { - return new ColumnMetadata(name, type, null, hidden); + return ColumnMetadata.builder() + .setName(name) + .setType(type) + .setHidden(hidden) + .build(); } @Override public int hashCode() { - return Objects.hash(ordinalPosition, name, type, mapping, dataFormat, formatHint, keyDecoder, hidden, internal); + return Objects.hash(name, type, mapping, dataFormat, formatHint, keyCodec, hidden, internal); } @Override @@ -177,34 +169,26 @@ public final class KafkaColumnHandle } KafkaColumnHandle other = (KafkaColumnHandle) obj; - return Objects.equals(this.ordinalPosition, other.ordinalPosition) && - Objects.equals(this.name, other.name) && + return Objects.equals(this.name, other.name) && Objects.equals(this.type, other.type) && Objects.equals(this.mapping, other.mapping) && Objects.equals(this.dataFormat, other.dataFormat) && Objects.equals(this.formatHint, other.formatHint) && - Objects.equals(this.keyDecoder, other.keyDecoder) && + Objects.equals(this.keyCodec, other.keyCodec) && Objects.equals(this.hidden, other.hidden) && Objects.equals(this.internal, other.internal); } - @Override - public int compareTo(KafkaColumnHandle otherHandle) - { - return Integer.compare(this.getOrdinalPosition(), otherHandle.getOrdinalPosition()); - } - @Override public String toString() { return toStringHelper(this) - .add("ordinalPosition", ordinalPosition) .add("name", name) .add("type", type) .add("mapping", mapping) .add("dataFormat", dataFormat) .add("formatHint", formatHint) - .add("keyDecoder", keyDecoder) + .add("keyCodec", keyCodec) .add("hidden", hidden) .add("internal", internal) .toString(); diff --git a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaConfig.java b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaConfig.java index 0adfceed8..900ff5c6f 100644 --- a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaConfig.java +++ b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaConfig.java @@ -17,10 +17,9 @@ import com.google.common.base.Splitter; import com.google.common.collect.ImmutableSet; import io.airlift.configuration.Config; import io.airlift.configuration.ConfigDescription; +import io.airlift.configuration.DefunctConfig; import io.airlift.units.DataSize; import io.airlift.units.DataSize.Unit; -import io.airlift.units.Duration; -import io.airlift.units.MinDuration; import io.prestosql.spi.HostAddress; import javax.validation.constraints.Min; @@ -33,18 +32,19 @@ import java.util.stream.StreamSupport; import static com.google.common.collect.ImmutableSet.toImmutableSet; +@DefunctConfig("kafka.connect-timeout") public class KafkaConfig { private static final int KAFKA_DEFAULT_PORT = 9092; private Set nodes = ImmutableSet.of(); - private Duration kafkaConnectTimeout = Duration.valueOf("10s"); - private DataSize kafkaBufferSize = new DataSize(64, Unit.KILOBYTE); + private DataSize kafkaBufferSize = DataSize.of(64, Unit.KILOBYTE); private String defaultSchema = "default"; private Set tableNames = ImmutableSet.of(); private File tableDescriptionDir = new File("etc/kafka/"); private boolean hideInternalColumns = true; private int messagesPerSplit = 100_000; + private boolean timestampUpperBoundPushDownEnabled; @Size(min = 1) public Set getNodes() @@ -60,20 +60,6 @@ public class KafkaConfig return this; } - @MinDuration("1s") - public Duration getKafkaConnectTimeout() - { - return kafkaConnectTimeout; - } - - @Config("kafka.connect-timeout") - @ConfigDescription("Kafka connection timeout") - public KafkaConfig setKafkaConnectTimeout(String kafkaConnectTimeout) - { - this.kafkaConnectTimeout = Duration.valueOf(kafkaConnectTimeout); - return this; - } - public DataSize getKafkaBufferSize() { return kafkaBufferSize; @@ -168,4 +154,17 @@ public class KafkaConfig this.messagesPerSplit = messagesPerSplit; return this; } + + public boolean isTimestampUpperBoundPushDownEnabled() + { + return timestampUpperBoundPushDownEnabled; + } + + @Config("kafka.timestamp-upper-bound-force-push-down-enabled") + @ConfigDescription("timestamp upper bound force pushing down enabled") + public KafkaConfig setTimestampUpperBoundPushDownEnabled(boolean timestampUpperBoundPushDownEnabled) + { + this.timestampUpperBoundPushDownEnabled = timestampUpperBoundPushDownEnabled; + return this; + } } diff --git a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaConnector.java b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaConnector.java index 3e0e46310..6241416f6 100644 --- a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaConnector.java +++ b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaConnector.java @@ -14,41 +14,48 @@ package io.prestosql.plugin.kafka; import io.airlift.bootstrap.LifeCycleManager; -import io.airlift.log.Logger; import io.prestosql.spi.connector.Connector; import io.prestosql.spi.connector.ConnectorMetadata; +import io.prestosql.spi.connector.ConnectorPageSinkProvider; import io.prestosql.spi.connector.ConnectorRecordSetProvider; import io.prestosql.spi.connector.ConnectorSplitManager; import io.prestosql.spi.connector.ConnectorTransactionHandle; +import io.prestosql.spi.session.PropertyMetadata; import io.prestosql.spi.transaction.IsolationLevel; +import javax.inject.Inject; + +import java.util.List; + import static io.prestosql.spi.transaction.IsolationLevel.READ_COMMITTED; import static io.prestosql.spi.transaction.IsolationLevel.checkConnectorSupports; import static java.util.Objects.requireNonNull; -/** - * Kafka specific implementation of the Presto Connector SPI. This is a read only connector. - */ public class KafkaConnector implements Connector { - private static final Logger log = Logger.get(KafkaConnector.class); - private final LifeCycleManager lifeCycleManager; private final ConnectorMetadata metadata; private final ConnectorSplitManager splitManager; private final ConnectorRecordSetProvider recordSetProvider; + private final ConnectorPageSinkProvider pageSinkProvider; + private final KafkaSessionProperties sessionProperties; + @Inject public KafkaConnector( LifeCycleManager lifeCycleManager, ConnectorMetadata metadata, ConnectorSplitManager splitManager, - ConnectorRecordSetProvider recordSetProvider) + ConnectorRecordSetProvider recordSetProvider, + ConnectorPageSinkProvider pageSinkProvider, + KafkaSessionProperties sessionProperties) { this.lifeCycleManager = requireNonNull(lifeCycleManager, "lifeCycleManager is null"); this.metadata = requireNonNull(metadata, "metadata is null"); this.splitManager = requireNonNull(splitManager, "splitManager is null"); this.recordSetProvider = requireNonNull(recordSetProvider, "recordSetProvider is null"); + this.pageSinkProvider = requireNonNull(pageSinkProvider, "pageSinkProvider is null"); + this.sessionProperties = requireNonNull(sessionProperties, "sessionProperties is null"); } @Override @@ -76,14 +83,21 @@ public class KafkaConnector return recordSetProvider; } + @Override + public ConnectorPageSinkProvider getPageSinkProvider() + { + return pageSinkProvider; + } + + @Override + public List> getSessionProperties() + { + return sessionProperties.getSessionProperties(); + } + @Override public final void shutdown() { - try { - lifeCycleManager.stop(); - } - catch (Exception e) { - log.error(e, "Error shutting down connector"); - } + lifeCycleManager.stop(); } } diff --git a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaConnectorFactory.java b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaConnectorFactory.java index 6a79d6814..82072a7f6 100644 --- a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaConnectorFactory.java +++ b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaConnectorFactory.java @@ -14,35 +14,28 @@ package io.prestosql.plugin.kafka; import com.google.inject.Injector; -import com.google.inject.Scopes; -import com.google.inject.TypeLiteral; +import com.google.inject.Module; import io.airlift.bootstrap.Bootstrap; -import io.airlift.bootstrap.LifeCycleManager; import io.airlift.json.JsonModule; import io.prestosql.spi.NodeManager; import io.prestosql.spi.connector.Connector; import io.prestosql.spi.connector.ConnectorContext; import io.prestosql.spi.connector.ConnectorFactory; import io.prestosql.spi.connector.ConnectorHandleResolver; -import io.prestosql.spi.connector.SchemaTableName; -import io.prestosql.spi.connector.classloader.ClassLoaderSafeConnectorRecordSetProvider; -import io.prestosql.spi.connector.classloader.ClassLoaderSafeConnectorSplitManager; import io.prestosql.spi.type.TypeManager; import java.util.Map; -import java.util.Optional; -import java.util.function.Supplier; import static java.util.Objects.requireNonNull; public class KafkaConnectorFactory implements ConnectorFactory { - private final Optional>> tableDescriptionSupplier; + private final Module extension; - KafkaConnectorFactory(Optional>> tableDescriptionSupplier) + KafkaConnectorFactory(Module extension) { - this.tableDescriptionSupplier = requireNonNull(tableDescriptionSupplier, "tableDescriptionSupplier is null"); + this.extension = requireNonNull(extension, "extension is null"); } @Override @@ -66,16 +59,11 @@ public class KafkaConnectorFactory Bootstrap app = new Bootstrap( new JsonModule(), new KafkaConnectorModule(), + extension, binder -> { + binder.bind(ClassLoader.class).toInstance(KafkaConnectorFactory.class.getClassLoader()); binder.bind(TypeManager.class).toInstance(context.getTypeManager()); binder.bind(NodeManager.class).toInstance(context.getNodeManager()); - - if (tableDescriptionSupplier.isPresent()) { - binder.bind(new TypeLiteral>>() {}).toInstance(tableDescriptionSupplier.get()); - } - else { - binder.bind(new TypeLiteral>>() {}).to(KafkaTableDescriptionSupplier.class).in(Scopes.SINGLETON); - } }); Injector injector = app @@ -84,11 +72,6 @@ public class KafkaConnectorFactory .setRequiredConfigurationProperties(config) .initialize(); - ClassLoader classLoader = KafkaConnectorFactory.class.getClassLoader(); - return new KafkaConnector( - injector.getInstance(LifeCycleManager.class), - injector.getInstance(KafkaMetadata.class), - new ClassLoaderSafeConnectorSplitManager(injector.getInstance(KafkaSplitManager.class), classLoader), - new ClassLoaderSafeConnectorRecordSetProvider(injector.getInstance(KafkaRecordSetProvider.class), classLoader)); + return injector.getInstance(KafkaConnector.class); } } diff --git a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaConnectorModule.java b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaConnectorModule.java index 756bd400e..f055d6331 100644 --- a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaConnectorModule.java +++ b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaConnectorModule.java @@ -19,38 +19,58 @@ import com.google.inject.Binder; import com.google.inject.Module; import com.google.inject.Scopes; import io.prestosql.decoder.DecoderModule; +import io.prestosql.plugin.base.classloader.ClassLoaderSafeConnectorPageSinkProvider; +import io.prestosql.plugin.base.classloader.ClassLoaderSafeConnectorRecordSetProvider; +import io.prestosql.plugin.base.classloader.ClassLoaderSafeConnectorSplitManager; +import io.prestosql.plugin.base.classloader.ForClassLoaderSafe; +import io.prestosql.plugin.kafka.encoder.EncoderModule; +import io.prestosql.spi.connector.ConnectorMetadata; +import io.prestosql.spi.connector.ConnectorPageSinkProvider; +import io.prestosql.spi.connector.ConnectorRecordSetProvider; +import io.prestosql.spi.connector.ConnectorSplitManager; import io.prestosql.spi.type.Type; +import io.prestosql.spi.type.TypeId; import io.prestosql.spi.type.TypeManager; import javax.inject.Inject; +import static com.google.inject.multibindings.Multibinder.newSetBinder; import static io.airlift.configuration.ConfigBinder.configBinder; import static io.airlift.json.JsonBinder.jsonBinder; import static io.airlift.json.JsonCodecBinder.jsonCodecBinder; import static io.prestosql.spi.type.TypeSignature.parseTypeSignature; import static java.util.Objects.requireNonNull; -/** - * Guice module for the Apache Kafka connector. - */ public class KafkaConnectorModule implements Module { @Override public void configure(Binder binder) { - binder.bind(KafkaMetadata.class).in(Scopes.SINGLETON); - binder.bind(KafkaSplitManager.class).in(Scopes.SINGLETON); - binder.bind(KafkaRecordSetProvider.class).in(Scopes.SINGLETON); - - binder.bind(KafkaConsumerFactory.class).in(Scopes.SINGLETON); + binder.bind(ConnectorMetadata.class).to(KafkaMetadata.class).in(Scopes.SINGLETON); + binder.bind(ConnectorSplitManager.class).annotatedWith(ForClassLoaderSafe.class).to(KafkaSplitManager.class).in(Scopes.SINGLETON); + binder.bind(ConnectorSplitManager.class).to(ClassLoaderSafeConnectorSplitManager.class).in(Scopes.SINGLETON); + binder.bind(ConnectorRecordSetProvider.class).annotatedWith(ForClassLoaderSafe.class).to(KafkaRecordSetProvider.class).in(Scopes.SINGLETON); + binder.bind(ConnectorRecordSetProvider.class).to(ClassLoaderSafeConnectorRecordSetProvider.class).in(Scopes.SINGLETON); + 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); + binder.bind(KafkaSessionProperties.class).in(Scopes.SINGLETON); + binder.bind(KafkaAdminFactory.class).in(Scopes.SINGLETON); + binder.bind(KafkaFilterManager.class).in(Scopes.SINGLETON); + binder.bind(KafkaSimpleConsumerManager.class).in(Scopes.SINGLETON); configBinder(binder).bindConfig(KafkaConnectorConfig.class); + configBinder(binder).bindConfig(KafkaConfig.class); + newSetBinder(binder, TableDescriptionSupplier.class).addBinding().toProvider(KafkaTableDescriptionSupplier.class).in(Scopes.SINGLETON); jsonBinder(binder).addDeserializerBinding(Type.class).to(TypeDeserializer.class); jsonCodecBinder(binder).bindJsonCodec(KafkaTopicDescription.class); binder.install(new DecoderModule()); + binder.install(new EncoderModule()); + binder.install(new KafkaProducerModule()); } public static final class TypeDeserializer diff --git a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaConsumerFactory.java b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaConsumerFactory.java index b6009697c..9872e7809 100644 --- a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaConsumerFactory.java +++ b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaConsumerFactory.java @@ -11,52 +11,19 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package io.prestosql.plugin.kafka; -import io.airlift.units.DataSize; -import io.prestosql.spi.HostAddress; import org.apache.kafka.clients.consumer.KafkaConsumer; -import org.apache.kafka.common.serialization.ByteArrayDeserializer; - -import javax.inject.Inject; import java.util.Properties; -import java.util.Set; -import static java.util.Objects.requireNonNull; -import static java.util.stream.Collectors.joining; -import static org.apache.kafka.clients.consumer.ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG; -import static org.apache.kafka.clients.consumer.ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG; -import static org.apache.kafka.clients.consumer.ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG; -import static org.apache.kafka.clients.consumer.ConsumerConfig.RECEIVE_BUFFER_CONFIG; -import static org.apache.kafka.clients.consumer.ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG; - -public class KafkaConsumerFactory +public interface KafkaConsumerFactory { - private final Set nodes; - private final DataSize kafkaBufferSize; - - @Inject - public KafkaConsumerFactory(KafkaConfig kafkaConfig) + default KafkaConsumer create() { - requireNonNull(kafkaConfig, "kafkaConfig is null"); - - nodes = kafkaConfig.getNodes(); - kafkaBufferSize = kafkaConfig.getKafkaBufferSize(); + return new KafkaConsumer<>(configure()); } - public KafkaConsumer create() - { - Properties properties = new Properties(); - properties.put( - BOOTSTRAP_SERVERS_CONFIG, - nodes.stream() - .map(HostAddress::toString) - .collect(joining(","))); - properties.put(KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName()); - properties.put(VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName()); - properties.put(RECEIVE_BUFFER_CONFIG, Long.toString(kafkaBufferSize.toBytes())); - properties.put(ENABLE_AUTO_COMMIT_CONFIG, Boolean.toString(false)); - return new KafkaConsumer(properties); - } + Properties configure(); } diff --git a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaConsumerModule.java b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaConsumerModule.java new file mode 100644 index 000000000..0759a716e --- /dev/null +++ b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaConsumerModule.java @@ -0,0 +1,28 @@ +/* + * 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.inject.Binder; +import com.google.inject.Module; +import com.google.inject.Scopes; + +public class KafkaConsumerModule + implements Module +{ + @Override + public void configure(Binder binder) + { + binder.bind(KafkaConsumerFactory.class).to(PlainTextKafkaConsumerFactory.class).in(Scopes.SINGLETON); + } +} diff --git a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaErrorCode.java b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaErrorCode.java index 006a70a52..44a883464 100644 --- a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaErrorCode.java +++ b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaErrorCode.java @@ -18,6 +18,7 @@ import io.prestosql.spi.ErrorCodeSupplier; import io.prestosql.spi.ErrorType; import static io.prestosql.spi.ErrorType.EXTERNAL; +import static io.prestosql.spi.ErrorType.INTERNAL_ERROR; /** * Kafka connector specific error codes. @@ -25,7 +26,10 @@ import static io.prestosql.spi.ErrorType.EXTERNAL; public enum KafkaErrorCode implements ErrorCodeSupplier { - KAFKA_SPLIT_ERROR(0, EXTERNAL); + KAFKA_SPLIT_ERROR(0, EXTERNAL), + KAFKA_SCHEMA_ERROR(1, EXTERNAL), + KAFKA_PRODUCER_ERROR(2, INTERNAL_ERROR) + /**/; private final ErrorCode errorCode; diff --git a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaFilterManager.java b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaFilterManager.java new file mode 100644 index 000000000..d4688a30c --- /dev/null +++ b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaFilterManager.java @@ -0,0 +1,334 @@ +/* + * 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.annotations.VisibleForTesting; +import com.google.common.collect.ImmutableMap; +import io.prestosql.spi.PrestoException; +import io.prestosql.spi.connector.ColumnHandle; +import io.prestosql.spi.connector.ConnectorSession; +import io.prestosql.spi.predicate.Domain; +import io.prestosql.spi.predicate.Marker; +import io.prestosql.spi.predicate.Ranges; +import io.prestosql.spi.predicate.SortedRangeSet; +import io.prestosql.spi.predicate.TupleDomain; +import io.prestosql.spi.predicate.ValueSet; +import org.apache.kafka.clients.admin.AdminClient; +import org.apache.kafka.clients.admin.Config; +import org.apache.kafka.clients.admin.DescribeConfigsResult; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.consumer.OffsetAndTimestamp; +import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.config.ConfigResource; + +import javax.inject.Inject; + +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.function.Function; + +import static com.google.common.base.Verify.verify; +import static com.google.common.collect.ImmutableList.toImmutableList; +import static com.google.common.collect.ImmutableSet.toImmutableSet; +import static com.google.common.collect.Iterables.getOnlyElement; +import static io.prestosql.plugin.kafka.KafkaErrorCode.KAFKA_SPLIT_ERROR; +import static io.prestosql.plugin.kafka.KafkaInternalFieldManager.OFFSET_TIMESTAMP_FIELD; +import static io.prestosql.plugin.kafka.KafkaInternalFieldManager.PARTITION_ID_FIELD; +import static io.prestosql.plugin.kafka.KafkaInternalFieldManager.PARTITION_OFFSET_FIELD; +import static io.prestosql.spi.type.Timestamps.MICROSECONDS_PER_MILLISECOND; +import static java.lang.Math.floorDiv; +import static java.lang.String.format; +import static java.util.Objects.requireNonNull; + +public class KafkaFilterManager +{ + private static final long INVALID_KAFKA_RANGE_INDEX = -1; + private static final String TOPIC_CONFIG_TIMESTAMP_KEY = "message.timestamp.type"; + private static final String TOPIC_CONFIG_TIMESTAMP_VALUE_LOG_APPEND_TIME = "LogAppendTime"; + + private final KafkaConsumerFactory consumerFactory; + private final KafkaAdminFactory adminFactory; + + @Inject + public KafkaFilterManager(KafkaConsumerFactory consumerFactory, KafkaAdminFactory adminFactory) + { + this.consumerFactory = requireNonNull(consumerFactory, "consumerManager is null"); + this.adminFactory = requireNonNull(adminFactory, "adminFactory is null"); + } + + public KafkaFilteringResult getKafkaFilterResult( + ConnectorSession session, + KafkaTableHandle kafkaTableHandle, + List partitionInfos, + Map partitionBeginOffsets, + Map partitionEndOffsets) + { + requireNonNull(session, "session is null"); + requireNonNull(kafkaTableHandle, "kafkaTableHandle is null"); + requireNonNull(partitionInfos, "partitionInfos is null"); + requireNonNull(partitionBeginOffsets, "partitionBeginOffsets is null"); + requireNonNull(partitionEndOffsets, "partitionEndOffsets is null"); + + TupleDomain constraint = kafkaTableHandle.getConstraint(); + verify(!constraint.isNone(), "constraint is none"); + + if (!constraint.isAll()) { + Set partitionIds = partitionInfos.stream().map(partitionInfo -> (long) partitionInfo.partition()).collect(toImmutableSet()); + Optional offsetRanged = Optional.empty(); + Optional offsetTimestampRanged = Optional.empty(); + Set partitionIdsFiltered = partitionIds; + Optional> domains = constraint.getDomains(); + + for (Map.Entry entry : domains.get().entrySet()) { + KafkaColumnHandle columnHandle = (KafkaColumnHandle) entry.getKey(); + if (!columnHandle.isInternal()) { + continue; + } + switch (columnHandle.getName()) { + case PARTITION_OFFSET_FIELD: + offsetRanged = filterRangeByDomain(entry.getValue()); + break; + case PARTITION_ID_FIELD: + partitionIdsFiltered = filterValuesByDomain(entry.getValue(), partitionIds); + break; + case OFFSET_TIMESTAMP_FIELD: + offsetTimestampRanged = filterRangeByDomain(entry.getValue()); + break; + default: + break; + } + } + + // push down offset + if (offsetRanged.isPresent()) { + Range range = offsetRanged.get(); + partitionBeginOffsets = overridePartitionBeginOffsets(partitionBeginOffsets, + partition -> (range.getBegin() != INVALID_KAFKA_RANGE_INDEX) ? Optional.of(range.getBegin()) : Optional.empty()); + partitionEndOffsets = overridePartitionEndOffsets(partitionEndOffsets, + partition -> (range.getEnd() != INVALID_KAFKA_RANGE_INDEX) ? Optional.of(range.getEnd()) : Optional.empty()); + } + + // push down timestamp if possible + if (offsetTimestampRanged.isPresent()) { + try (KafkaConsumer kafkaConsumer = consumerFactory.create()) { + Optional finalOffsetTimestampRanged = offsetTimestampRanged; + // filter negative value to avoid java.lang.IllegalArgumentException when using KafkaConsumer offsetsForTimes + if (offsetTimestampRanged.get().getBegin() > INVALID_KAFKA_RANGE_INDEX) { + partitionBeginOffsets = overridePartitionBeginOffsets(partitionBeginOffsets, + partition -> findOffsetsForTimestampGreaterOrEqual(kafkaConsumer, partition, finalOffsetTimestampRanged.get().getBegin())); + } + if (isTimestampUpperBoundPushdownEnabled(session, kafkaTableHandle.getTopicName())) { + if (offsetTimestampRanged.get().getEnd() > INVALID_KAFKA_RANGE_INDEX) { + partitionEndOffsets = overridePartitionEndOffsets(partitionEndOffsets, + partition -> findOffsetsForTimestampGreaterOrEqual(kafkaConsumer, partition, finalOffsetTimestampRanged.get().getEnd())); + } + } + } + } + + // push down partitions + final Set finalPartitionIdsFiltered = partitionIdsFiltered; + List partitionFilteredInfos = partitionInfos.stream() + .filter(partitionInfo -> finalPartitionIdsFiltered.contains((long) partitionInfo.partition())) + .collect(toImmutableList()); + return new KafkaFilteringResult(partitionFilteredInfos, partitionBeginOffsets, partitionEndOffsets); + } + return new KafkaFilteringResult(partitionInfos, partitionBeginOffsets, partitionEndOffsets); + } + + private boolean isTimestampUpperBoundPushdownEnabled(ConnectorSession session, String topic) + { + try (AdminClient adminClient = adminFactory.create()) { + ConfigResource topicResource = new ConfigResource(ConfigResource.Type.TOPIC, topic); + + DescribeConfigsResult describeResult = adminClient.describeConfigs(Collections.singleton(topicResource)); + Map configMap = describeResult.all().get(); + + if (configMap != null) { + Config config = configMap.get(topicResource); + String timestampType = config.get(TOPIC_CONFIG_TIMESTAMP_KEY).value(); + if (TOPIC_CONFIG_TIMESTAMP_VALUE_LOG_APPEND_TIME.equals(timestampType)) { + return true; + } + } + } + catch (Exception e) { + throw new PrestoException(KAFKA_SPLIT_ERROR, format("Failed to get configuration for topic '%s'", topic), e); + } + return KafkaSessionProperties.isTimestampUpperBoundPushdownEnabled(session); + } + + private static Optional findOffsetsForTimestampGreaterOrEqual(KafkaConsumer kafkaConsumer, TopicPartition topicPartition, long timestamp) + { + final long transferTimestamp = floorDiv(timestamp, MICROSECONDS_PER_MILLISECOND); + Map topicPartitionOffsets = kafkaConsumer.offsetsForTimes(ImmutableMap.of(topicPartition, transferTimestamp)); + return Optional.ofNullable(getOnlyElement(topicPartitionOffsets.values(), null)).map(OffsetAndTimestamp::offset); + } + + private static Map overridePartitionBeginOffsets(Map partitionBeginOffsets, + Function> overrideFunction) + { + ImmutableMap.Builder partitionFilteredBeginOffsetsBuilder = ImmutableMap.builder(); + partitionBeginOffsets.forEach((partition, partitionIndex) -> { + Optional newOffset = overrideFunction.apply(partition); + partitionFilteredBeginOffsetsBuilder.put(partition, newOffset.map(index -> Long.max(partitionIndex, index)).orElse(partitionIndex)); + }); + return partitionFilteredBeginOffsetsBuilder.build(); + } + + private static Map overridePartitionEndOffsets(Map partitionEndOffsets, + Function> overrideFunction) + { + ImmutableMap.Builder partitionFilteredEndOffsetsBuilder = ImmutableMap.builder(); + partitionEndOffsets.forEach((partition, partitionIndex) -> { + Optional newOffset = overrideFunction.apply(partition); + partitionFilteredEndOffsetsBuilder.put(partition, newOffset.map(index -> Long.min(partitionIndex, index)).orElse(partitionIndex)); + }); + return partitionFilteredEndOffsetsBuilder.build(); + } + + @VisibleForTesting + public static Optional filterRangeByDomain(Domain domain) + { + Long low = INVALID_KAFKA_RANGE_INDEX; + Long high = INVALID_KAFKA_RANGE_INDEX; + if (domain.isSingleValue()) { + // still return range for single value case like (_partition_offset=XXX or _timestamp=XXX) + low = (long) domain.getSingleValue(); + high = (long) domain.getSingleValue(); + } + else { + ValueSet valueSet = domain.getValues(); + if (valueSet instanceof SortedRangeSet) { + // still return range for single value case like (_partition_offset in (XXX1,XXX2) or _timestamp in XXX1, XXX2) + Ranges ranges = ((SortedRangeSet) valueSet).getRanges(); + List rangeList = ranges.getOrderedRanges(); + if (rangeList.stream().allMatch(io.prestosql.spi.predicate.Range::isSingleValue)) { + List values = rangeList.stream() + .map(range -> (Long) range.getSingleValue()) + .collect(toImmutableList()); + low = Collections.min(values); + high = Collections.max(values); + } + else { + Marker lowMark = ranges.getSpan().getLow(); + low = getLowByLowMark(lowMark).orElse(low); + Marker highMark = ranges.getSpan().getHigh(); + high = getHighByHighMark(highMark).orElse(high); + } + } + } + if (high != INVALID_KAFKA_RANGE_INDEX) { + high = high + 1; + } + return Optional.of(new Range(low, high)); + } + + @VisibleForTesting + public static Set filterValuesByDomain(Domain domain, Set sourceValues) + { + requireNonNull(sourceValues, "sourceValues is none"); + if (domain.isSingleValue()) { + long singleValue = (long) domain.getSingleValue(); + return sourceValues.stream().filter(sourceValue -> sourceValue == singleValue).collect(toImmutableSet()); + } + else { + ValueSet valueSet = domain.getValues(); + if (valueSet instanceof SortedRangeSet) { + Ranges ranges = ((SortedRangeSet) valueSet).getRanges(); + List rangeList = ranges.getOrderedRanges(); + if (rangeList.stream().allMatch(io.prestosql.spi.predicate.Range::isSingleValue)) { + return rangeList.stream() + .map(range -> (Long) range.getSingleValue()) + .filter(sourceValues::contains) + .collect(toImmutableSet()); + } + else { + // still return values for range case like (_partition_id > 1) + long low = 0; + long high = Long.MAX_VALUE; + + Marker lowMark = ranges.getSpan().getLow(); + low = maxLow(low, lowMark); + Marker highMark = ranges.getSpan().getHigh(); + high = minHigh(high, highMark); + final long finalLow = low; + final long finalHigh = high; + return sourceValues.stream() + .filter(item -> item >= finalLow && item <= finalHigh) + .collect(toImmutableSet()); + } + } + } + return sourceValues; + } + + private static long minHigh(long high, Marker highMark) + { + Optional highByHighMark = getHighByHighMark(highMark); + if (highByHighMark.isPresent()) { + high = Long.min(highByHighMark.get(), high); + } + return high; + } + + private static long maxLow(long low, Marker lowMark) + { + Optional lowByLowMark = getLowByLowMark(lowMark); + if (lowByLowMark.isPresent()) { + low = Long.max(lowByLowMark.get(), low); + } + return low; + } + + private static Optional getHighByHighMark(Marker highMark) + { + if (!highMark.isUpperUnbounded()) { + long high = (Long) highMark.getValue(); + switch (highMark.getBound()) { + case EXACTLY: + break; + case BELOW: + high--; + break; + default: + throw new AssertionError("Unhandled bound: " + highMark.getBound()); + } + return Optional.of(high); + } + return Optional.empty(); + } + + private static Optional getLowByLowMark(Marker lowMark) + { + if (!lowMark.isLowerUnbounded()) { + long low = (Long) lowMark.getValue(); + switch (lowMark.getBound()) { + case EXACTLY: + break; + case ABOVE: + low++; + break; + default: + throw new AssertionError("Unhandled bound: " + lowMark.getBound()); + } + return Optional.of(low); + } + return Optional.empty(); + } +} diff --git a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaFilteringResult.java b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaFilteringResult.java new file mode 100644 index 000000000..5ea078648 --- /dev/null +++ b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaFilteringResult.java @@ -0,0 +1,53 @@ +/* + * 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.ImmutableList; +import com.google.common.collect.ImmutableMap; +import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.TopicPartition; + +import java.util.List; +import java.util.Map; + +public class KafkaFilteringResult +{ + private final List partitionInfos; + private final Map partitionBeginOffsets; + private final Map partitionEndOffsets; + + public KafkaFilteringResult(List partitionInfos, + Map partitionBeginOffsets, + Map partitionEndOffsets) + { + this.partitionInfos = ImmutableList.copyOf(partitionInfos); + this.partitionBeginOffsets = ImmutableMap.copyOf(partitionBeginOffsets); + this.partitionEndOffsets = ImmutableMap.copyOf(partitionEndOffsets); + } + + public List getPartitionInfos() + { + return partitionInfos; + } + + public Map getPartitionBeginOffsets() + { + return partitionBeginOffsets; + } + + public Map getPartitionEndOffsets() + { + return partitionEndOffsets; + } +} diff --git a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaHandleResolver.java b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaHandleResolver.java index fe3c404fa..9cce6d412 100644 --- a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaHandleResolver.java +++ b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaHandleResolver.java @@ -15,6 +15,7 @@ package io.prestosql.plugin.kafka; import io.prestosql.spi.connector.ColumnHandle; import io.prestosql.spi.connector.ConnectorHandleResolver; +import io.prestosql.spi.connector.ConnectorInsertTableHandle; import io.prestosql.spi.connector.ConnectorSplit; import io.prestosql.spi.connector.ConnectorTableHandle; import io.prestosql.spi.connector.ConnectorTransactionHandle; @@ -40,6 +41,12 @@ public class KafkaHandleResolver return KafkaColumnHandle.class; } + @Override + public Class getInsertTableHandleClass() + { + return KafkaTableHandle.class; + } + @Override public Class getSplitClass() { diff --git a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaInternalFieldDescription.java b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaInternalFieldDescription.java deleted file mode 100644 index 6cd5872ab..000000000 --- a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaInternalFieldDescription.java +++ /dev/null @@ -1,133 +0,0 @@ -/* - * 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 io.prestosql.spi.connector.ColumnMetadata; -import io.prestosql.spi.type.BigintType; -import io.prestosql.spi.type.BooleanType; -import io.prestosql.spi.type.Type; - -import java.util.Map; - -import static com.google.common.base.Preconditions.checkArgument; -import static com.google.common.base.Strings.isNullOrEmpty; -import static com.google.common.collect.ImmutableMap.toImmutableMap; -import static io.prestosql.spi.type.VarcharType.createUnboundedVarcharType; -import static java.util.Arrays.stream; -import static java.util.Objects.requireNonNull; -import static java.util.function.Function.identity; - -/** - * Describes an internal (managed by the connector) field which is added to each table row. The definition itself makes the row - * show up in the tables (the columns are hidden by default, so they must be explicitly selected) but unless the field is hooked in using the - * forBooleanValue/forLongValue/forBytesValue methods and the resulting FieldValueProvider is then passed into the appropriate row decoder, the fields - * will be null. Most values are assigned in the {@link io.prestosql.plugin.kafka.KafkaRecordSet}. - */ -public enum KafkaInternalFieldDescription -{ - /** - * _partition_id - Kafka partition id. - */ - PARTITION_ID_FIELD("_partition_id", BigintType.BIGINT, "Partition Id"), - - /** - * _partition_offset - The current offset of the message in the partition. - */ - PARTITION_OFFSET_FIELD("_partition_offset", BigintType.BIGINT, "Offset for the message within the partition"), - - /** - * _message_corrupt - 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). - */ - MESSAGE_CORRUPT_FIELD("_message_corrupt", BooleanType.BOOLEAN, "Message data is corrupt"), - - /** - * _message - Represents the full topic as a text column. Format is UTF-8 which may be wrong for some topics. TODO: make charset configurable. - */ - MESSAGE_FIELD("_message", createUnboundedVarcharType(), "Message text"), - - /** - * _message_length - length in bytes of the message. - */ - MESSAGE_LENGTH_FIELD("_message_length", BigintType.BIGINT, "Total number of message bytes"), - - /** - * _key_corrupt - 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). - */ - KEY_CORRUPT_FIELD("_key_corrupt", BooleanType.BOOLEAN, "Key data is corrupt"), - - /** - * _key - Represents the key as a text column. Format is UTF-8 which may be wrong for topics. TODO: make charset configurable. - */ - KEY_FIELD("_key", createUnboundedVarcharType(), "Key text"), - - /** - * _key_length - length in bytes of the key. - */ - KEY_LENGTH_FIELD("_key_length", BigintType.BIGINT, "Total number of key bytes"); - - private static final Map BY_COLUMN_NAME = - stream(KafkaInternalFieldDescription.values()) - .collect(toImmutableMap(KafkaInternalFieldDescription::getColumnName, identity())); - - public static KafkaInternalFieldDescription forColumnName(String columnName) - { - KafkaInternalFieldDescription description = BY_COLUMN_NAME.get(columnName); - checkArgument(description != null, "Unknown internal column name %s", columnName); - return description; - } - - private final String columnName; - private final Type type; - private final String comment; - - KafkaInternalFieldDescription( - String columnName, - Type type, - String comment) - { - checkArgument(!isNullOrEmpty(columnName), "name is null or is empty"); - this.columnName = columnName; - this.type = requireNonNull(type, "type is null"); - this.comment = requireNonNull(comment, "comment is null"); - } - - public String getColumnName() - { - return columnName; - } - - public Type getType() - { - return type; - } - - KafkaColumnHandle getColumnHandle(int index, boolean hidden) - { - return new KafkaColumnHandle( - index, - getColumnName(), - getType(), - null, - null, - null, - false, - hidden, - true); - } - - ColumnMetadata getColumnMetadata(boolean hidden) - { - return new ColumnMetadata(columnName, type, comment, hidden); - } -} diff --git a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaInternalFieldManager.java b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaInternalFieldManager.java new file mode 100644 index 000000000..7ffa1c085 --- /dev/null +++ b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaInternalFieldManager.java @@ -0,0 +1,192 @@ +/* + * 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.TimestampType.TIMESTAMP_MILLIS; +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 +{ + /** + * _partition_id - Kafka partition id. + */ + public static final String PARTITION_ID_FIELD = "_partition_id"; + + /** + * _partition_offset - The current offset of the message in the partition. + */ + public static final String PARTITION_OFFSET_FIELD = "_partition_offset"; + + /** + * _message_corrupt - 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"; + + /** + * _message - 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"; + + /** + * _message_length - length in bytes of the message. + */ + public static final String MESSAGE_LENGTH_FIELD = "_message_length"; + + /** + * _headers - 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"; + + /** + * _key_corrupt - 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"; + + /** + * _key - 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"; + + /** + * _key_length - length in bytes of the key. + */ + public static final String KEY_LENGTH_FIELD = "_key_length"; + + /** + * _timestamp - message timestamp + */ + public static final String OFFSET_TIMESTAMP_FIELD = "_timestamp"; + + 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 internalFields; + + @Inject + public KafkaInternalFieldManager(TypeManager typeManager) + { + Type varcharMapType = typeManager.getType(mapType(VARCHAR.getTypeSignature(), arrayType(VARBINARY.getTypeSignature()))); + + internalFields = new ImmutableMap.Builder() + .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)) + .put(OFFSET_TIMESTAMP_FIELD, new InternalField( + OFFSET_TIMESTAMP_FIELD, + "Message timestamp", + TIMESTAMP_MILLIS)) + .build(); + } + + /** + * @return Map of {@link InternalField} for each internal field. + */ + public Map getInternalFields() + { + return internalFields; + } +} diff --git a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaMetadata.java b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaMetadata.java index 6d21fbccc..f031b594a 100644 --- a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaMetadata.java +++ b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaMetadata.java @@ -15,82 +15,91 @@ package io.prestosql.plugin.kafka; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; -import com.google.common.collect.ImmutableSet; -import io.airlift.log.Logger; +import io.airlift.slice.Slice; import io.prestosql.decoder.dummy.DummyRowDecoder; import io.prestosql.spi.connector.ColumnHandle; import io.prestosql.spi.connector.ColumnMetadata; +import io.prestosql.spi.connector.ConnectorInsertTableHandle; import io.prestosql.spi.connector.ConnectorMetadata; +import io.prestosql.spi.connector.ConnectorOutputMetadata; import io.prestosql.spi.connector.ConnectorSession; import io.prestosql.spi.connector.ConnectorTableHandle; import io.prestosql.spi.connector.ConnectorTableMetadata; import io.prestosql.spi.connector.ConnectorTableProperties; +import io.prestosql.spi.connector.Constraint; +import io.prestosql.spi.connector.ConstraintApplicationResult; import io.prestosql.spi.connector.SchemaTableName; import io.prestosql.spi.connector.SchemaTablePrefix; import io.prestosql.spi.connector.TableNotFoundException; +import io.prestosql.spi.predicate.TupleDomain; +import io.prestosql.spi.statistics.ComputedStatistics; import javax.inject.Inject; +import java.util.Collection; import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.Set; import java.util.concurrent.atomic.AtomicInteger; -import java.util.function.Supplier; +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.collect.ImmutableList.toImmutableList; import static io.prestosql.plugin.kafka.KafkaHandleResolver.convertColumnHandle; import static io.prestosql.plugin.kafka.KafkaHandleResolver.convertTableHandle; import static java.util.Objects.requireNonNull; /** * 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 static final Logger LOG = Logger.get(KafkaMetadata.class); private final boolean hideInternalColumns; - private final Map tableDescriptions; + private final Set tableDescriptions; + private final KafkaInternalFieldManager kafkaInternalFieldManager; @Inject public KafkaMetadata( - KafkaConnectorConfig kafkaConnectorConfig, - Supplier> kafkaTableDescriptionSupplier) + KafkaConfig kafkaConfig, + Set tableDescriptions, + KafkaInternalFieldManager kafkaInternalFieldManager) { - requireNonNull(kafkaConnectorConfig, "kafkaConfig is null"); - this.hideInternalColumns = kafkaConnectorConfig.isHideInternalColumns(); - - requireNonNull(kafkaTableDescriptionSupplier, "kafkaTableDescriptionSupplier is null"); - this.tableDescriptions = kafkaTableDescriptionSupplier.get(); + requireNonNull(kafkaConfig, "kafkaConfig is null"); + this.hideInternalColumns = kafkaConfig.isHideInternalColumns(); + this.tableDescriptions = requireNonNull(tableDescriptions, "tableDescriptions is null"); + this.kafkaInternalFieldManager = requireNonNull(kafkaInternalFieldManager, "kafkaInternalFieldDescription is null"); } @Override public List listSchemaNames(ConnectorSession session) { - ImmutableSet.Builder builder = ImmutableSet.builder(); - for (SchemaTableName tableName : tableDescriptions.keySet()) { - builder.add(tableName.getSchemaName()); - } - return ImmutableList.copyOf(builder.build()); + return tableDescriptions.stream() + .map(TableDescriptionSupplier::listTables) + .flatMap(Set::stream) + .map(SchemaTableName::getSchemaName) + .collect(toImmutableList()); } @Override public KafkaTableHandle getTableHandle(ConnectorSession session, SchemaTableName schemaTableName) { - KafkaTopicDescription table = tableDescriptions.get(schemaTableName); - if (table == null) { - return null; - } - - return new KafkaTableHandle( - schemaTableName.getSchemaName(), - schemaTableName.getTableName(), - table.getTopicName(), - getDataFormat(table.getKey()), - getDataFormat(table.getMessage()), - table.getKey().flatMap(KafkaTopicFieldGroup::getDataSchema), - table.getMessage().flatMap(KafkaTopicFieldGroup::getDataSchema)); + return getTopicDescription(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), + getColumnHandles(schemaTableName).values().stream() + .map(KafkaColumnHandle.class::cast) + .collect(toImmutableList()), + TupleDomain.all())) + .orElse(null); } private static String getDataFormat(Optional fieldGroup) @@ -107,33 +116,29 @@ public class KafkaMetadata @Override public List listTables(ConnectorSession session, Optional schemaName) { - ImmutableList.Builder builder = ImmutableList.builder(); - for (SchemaTableName tableName : tableDescriptions.keySet()) { - if (schemaName.map(tableName.getSchemaName()::equals).orElse(true)) { - builder.add(tableName); - } - } - - return builder.build(); + return tableDescriptions.stream() + .map(TableDescriptionSupplier::listTables) + .flatMap(Set::stream) + .filter(tableName -> schemaName.map(tableName.getSchemaName()::equals).orElse(true)) + .collect(toImmutableList()); } - @SuppressWarnings("ValueOfIncrementOrDecrementUsed") @Override public Map getColumnHandles(ConnectorSession session, ConnectorTableHandle tableHandle) { KafkaTableHandle kafkaTableHandle = convertTableHandle(tableHandle); + return getColumnHandles(kafkaTableHandle.toSchemaTableName()); + } - KafkaTopicDescription kafkaTopicDescription = tableDescriptions.get(kafkaTableHandle.toSchemaTableName()); - if (kafkaTopicDescription == null) { - throw new TableNotFoundException(kafkaTableHandle.toSchemaTableName()); - } + private Map getColumnHandles(SchemaTableName schemaTableName) + { + KafkaTopicDescription kafkaTopicDescription = getRequiredTopicDescription(schemaTableName); ImmutableMap.Builder columnHandles = ImmutableMap.builder(); AtomicInteger index = new AtomicInteger(0); - kafkaTopicDescription.getKey().ifPresent(key -> - { + kafkaTopicDescription.getKey().ifPresent(key -> { List fields = key.getFields(); if (fields != null) { for (KafkaTopicFieldDescription kafkaTopicFieldDescription : fields) { @@ -142,8 +147,7 @@ public class KafkaMetadata } }); - kafkaTopicDescription.getMessage().ifPresent(message -> - { + kafkaTopicDescription.getMessage().ifPresent(message -> { List fields = message.getFields(); if (fields != null) { for (KafkaTopicFieldDescription kafkaTopicFieldDescription : fields) { @@ -152,8 +156,8 @@ public class KafkaMetadata } }); - 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(); @@ -167,7 +171,7 @@ public class KafkaMetadata ImmutableMap.Builder> columns = ImmutableMap.builder(); List tableNames; - if (!prefix.getTable().isPresent()) { + if (prefix.getTable().isEmpty()) { tableNames = listTables(session, prefix.getSchema()); } else { @@ -180,7 +184,6 @@ public class KafkaMetadata } catch (TableNotFoundException e) { // information_schema table or a system table - LOG.debug("Error message: " + e.getMessage()); } } return columns.build(); @@ -193,13 +196,9 @@ public class KafkaMetadata return convertColumnHandle(columnHandle).getColumnMetadata(); } - @SuppressWarnings("ValueOfIncrementOrDecrementUsed") private ConnectorTableMetadata getTableMetadata(SchemaTableName schemaTableName) { - KafkaTopicDescription table = tableDescriptions.get(schemaTableName); - if (table == null) { - throw new TableNotFoundException(schemaTableName); - } + KafkaTopicDescription table = getRequiredTopicDescription(schemaTableName); ImmutableList.Builder builder = ImmutableList.builder(); @@ -221,7 +220,7 @@ public class KafkaMetadata } }); - for (KafkaInternalFieldDescription fieldDescription : KafkaInternalFieldDescription.values()) { + for (KafkaInternalFieldManager.InternalField fieldDescription : kafkaInternalFieldManager.getInternalFields().values()) { builder.add(fieldDescription.getColumnMetadata(hideInternalColumns)); } @@ -239,4 +238,72 @@ public class KafkaMetadata { return new ConnectorTableProperties(); } + + @Override + public Optional> applyFilter(ConnectorSession session, ConnectorTableHandle table, Constraint constraint) + { + KafkaTableHandle handle = (KafkaTableHandle) table; + TupleDomain oldDomain = handle.getConstraint(); + TupleDomain 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.getColumns(), + newDomain); + + return Optional.of(new ConstraintApplicationResult<>(handle, constraint.getSummary())); + } + + private KafkaTopicDescription getRequiredTopicDescription(SchemaTableName schemaTableName) + { + return getTopicDescription(schemaTableName).orElseThrow(() -> new TableNotFoundException(schemaTableName)); + } + + private Optional getTopicDescription(SchemaTableName schemaTableName) + { + return tableDescriptions.stream() + .map(kafkaTableDescriptionSupplier -> kafkaTableDescriptionSupplier.getTopicDescription(schemaTableName)) + .filter(Optional::isPresent) + .map(Optional::get) + .findFirst(); + } + + @Override + public ConnectorInsertTableHandle beginInsert(ConnectorSession session, ConnectorTableHandle tableHandle, List columns) + { + // TODO: support transactional inserts https://github.com/prestosql/presto/issues/4303 + KafkaTableHandle table = (KafkaTableHandle) tableHandle; + List actualColumns = table.getColumns().stream() + .filter(col -> !col.isInternal()) + .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(), + actualColumns, + TupleDomain.none()); + } + + @Override + public Optional finishInsert(ConnectorSession session, ConnectorInsertTableHandle insertHandle, Collection fragments, Collection computedStatistics) + { + // TODO: support transactional inserts https://github.com/prestosql/presto/issues/4303 + return Optional.empty(); + } } diff --git a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaPageSink.java b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaPageSink.java new file mode 100644 index 000000000..0ebfd4866 --- /dev/null +++ b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaPageSink.java @@ -0,0 +1,129 @@ +/* + * 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.ImmutableList; +import io.airlift.slice.Slice; +import io.prestosql.plugin.kafka.encoder.RowEncoder; +import io.prestosql.spi.Page; +import io.prestosql.spi.PrestoException; +import io.prestosql.spi.connector.ConnectorPageSink; +import org.apache.kafka.clients.producer.Callback; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.clients.producer.RecordMetadata; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.Collection; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.atomic.AtomicLong; + +import static io.prestosql.plugin.kafka.KafkaErrorCode.KAFKA_PRODUCER_ERROR; +import static java.lang.String.format; +import static java.util.Objects.requireNonNull; +import static java.util.concurrent.CompletableFuture.completedFuture; + +public class KafkaPageSink + implements ConnectorPageSink +{ + private final String topicName; + private final List columns; + private final RowEncoder keyEncoder; + private final RowEncoder messageEncoder; + private final KafkaProducer producer; + private final ErrorCountingCallback errorCounter; + + public KafkaPageSink( + String topicName, + List columns, + RowEncoder keyEncoder, + RowEncoder messageEncoder, + PlainTextKafkaProducerFactory producerFactory) + { + this.topicName = requireNonNull(topicName, "topicName is null"); + this.columns = requireNonNull(ImmutableList.copyOf(columns), "columns is null"); + this.keyEncoder = requireNonNull(keyEncoder, "keyEncoder is null"); + this.messageEncoder = requireNonNull(messageEncoder, "messageEncoder is null"); + requireNonNull(producerFactory, "producerFactory is null"); + this.producer = producerFactory.create(); + this.errorCounter = new ErrorCountingCallback(); + } + + private static class ErrorCountingCallback + implements Callback + { + private final AtomicLong errorCounter; + + public ErrorCountingCallback() + { + this.errorCounter = new AtomicLong(0); + } + + @Override + public void onCompletion(RecordMetadata recordMetadata, Exception e) + { + if (e != null) { + errorCounter.incrementAndGet(); + } + } + + public long getErrorCount() + { + return errorCounter.get(); + } + } + + @Override + public CompletableFuture appendPage(Page page) + { + for (int position = 0; position < page.getPositionCount(); position++) { + for (int channel = 0; channel < page.getChannelCount(); channel++) { + if (columns.get(channel).isKeyCodec()) { + keyEncoder.appendColumnValue(page.getBlock(channel), position); + } + else { + messageEncoder.appendColumnValue(page.getBlock(channel), position); + } + } + producer.send(new ProducerRecord<>(topicName, keyEncoder.toByteArray(), messageEncoder.toByteArray()), errorCounter); + } + return NOT_BLOCKED; + } + + @Override + public CompletableFuture> finish() + { + producer.flush(); + producer.close(); + try { + keyEncoder.close(); + messageEncoder.close(); + } + catch (IOException e) { + throw new UncheckedIOException("Failed to close row encoders", e); + } + if (errorCounter.getErrorCount() > 0) { + throw new PrestoException(KAFKA_PRODUCER_ERROR, format("%d producer record('s) failed to send", errorCounter.getErrorCount())); + } + return completedFuture(ImmutableList.of()); + } + + @Override + public void abort() + { + producer.close(); + } +} diff --git a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaPageSinkProvider.java b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaPageSinkProvider.java new file mode 100644 index 000000000..d3aa1e59a --- /dev/null +++ b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaPageSinkProvider.java @@ -0,0 +1,109 @@ +/* + * 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.ImmutableList; +import io.prestosql.plugin.kafka.encoder.DispatchingRowEncoderFactory; +import io.prestosql.plugin.kafka.encoder.EncoderColumnHandle; +import io.prestosql.plugin.kafka.encoder.RowEncoder; +import io.prestosql.spi.PrestoException; +import io.prestosql.spi.connector.ConnectorInsertTableHandle; +import io.prestosql.spi.connector.ConnectorOutputTableHandle; +import io.prestosql.spi.connector.ConnectorPageSink; +import io.prestosql.spi.connector.ConnectorPageSinkProvider; +import io.prestosql.spi.connector.ConnectorSession; +import io.prestosql.spi.connector.ConnectorTransactionHandle; + +import javax.inject.Inject; + +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Paths; +import java.util.Optional; + +import static io.prestosql.plugin.kafka.KafkaErrorCode.KAFKA_SCHEMA_ERROR; +import static java.lang.String.format; +import static java.util.Objects.requireNonNull; + +public class KafkaPageSinkProvider + implements ConnectorPageSinkProvider +{ + private final DispatchingRowEncoderFactory encoderFactory; + private final PlainTextKafkaProducerFactory producerFactory; + + @Inject + public KafkaPageSinkProvider(DispatchingRowEncoderFactory encoderFactory, PlainTextKafkaProducerFactory producerFactory) + { + this.encoderFactory = requireNonNull(encoderFactory, "encoderFactory is null"); + this.producerFactory = requireNonNull(producerFactory, "producerFactory is null"); + } + + @Override + public ConnectorPageSink createPageSink(ConnectorTransactionHandle transactionHandle, ConnectorSession session, ConnectorOutputTableHandle tableHandle) + { + throw new UnsupportedOperationException("Table creation is not supported by the kafka connector"); + } + + @Override + public ConnectorPageSink createPageSink(ConnectorTransactionHandle transactionHandle, ConnectorSession session, ConnectorInsertTableHandle tableHandle) + { + requireNonNull(tableHandle, "tableHandle is null"); + KafkaTableHandle handle = (KafkaTableHandle) tableHandle; + + ImmutableList.Builder keyColumns = ImmutableList.builder(); + ImmutableList.Builder messageColumns = ImmutableList.builder(); + handle.getColumns().forEach(col -> { + if (col.isInternal()) { + throw new IllegalArgumentException(format("unexpected internal column '%s'", col.getName())); + } + if (col.isKeyCodec()) { + keyColumns.add(col); + } + else { + messageColumns.add(col); + } + }); + + RowEncoder keyEncoder = encoderFactory.create( + session, + handle.getKeyDataFormat(), + getDataSchema(handle.getKeyDataSchemaLocation()), + keyColumns.build()); + + RowEncoder messageEncoder = encoderFactory.create( + session, + handle.getMessageDataFormat(), + getDataSchema(handle.getMessageDataSchemaLocation()), + messageColumns.build()); + + return new KafkaPageSink( + handle.getTopicName(), + handle.getColumns(), + keyEncoder, + messageEncoder, + producerFactory); + } + + private Optional getDataSchema(Optional dataSchemaLocation) + { + return dataSchemaLocation.map(location -> { + try { + return Files.readString(Paths.get(location)); + } + catch (IOException e) { + throw new PrestoException(KAFKA_SCHEMA_ERROR, format("Unable to read data schema at '%s'", dataSchemaLocation.get()), e); + } + }); + } +} diff --git a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaPlugin.java b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaPlugin.java index e17b7b795..826b4fa8a 100644 --- a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaPlugin.java +++ b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaPlugin.java @@ -13,53 +13,35 @@ */ package io.prestosql.plugin.kafka; -import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.ImmutableList; +import com.google.inject.Module; import io.prestosql.spi.Plugin; import io.prestosql.spi.connector.ConnectorFactory; -import io.prestosql.spi.connector.SchemaTableName; -import io.prestosql.spi.function.ConnectorConfig; -import io.prestosql.spi.queryeditorui.ConnectorUtil; -import io.prestosql.spi.queryeditorui.ConnectorWithProperties; - -import java.util.Arrays; -import java.util.Map; -import java.util.Optional; -import java.util.function.Supplier; import static java.util.Objects.requireNonNull; -/** - * Presto plugin to use Apache Kafka as a data source. - */ -@ConnectorConfig(connectorLabel = "Kafka: Allow the use of Apache Kafka topics as tables in openLooKeng", - propertiesEnabled = true, - catalogConfigFilesEnabled = true, - globalConfigFilesEnabled = true, - docLink = "https://openlookeng.io/docs/docs/connector/kafka.html", - configLink = "https://openlookeng.io/docs/docs/connector/kafka.html#configuration") public class KafkaPlugin implements Plugin { - private Optional>> tableDescriptionSupplier = Optional.empty(); + public static final Module DEFAULT_EXTENSION = binder -> { + binder.install(new KafkaConsumerModule()); + }; + + private final Module extension; - @VisibleForTesting - public synchronized void setTableDescriptionSupplier(Supplier> tableDescriptionSupplier) + public KafkaPlugin() { - this.tableDescriptionSupplier = Optional.of(requireNonNull(tableDescriptionSupplier, "tableDescriptionSupplier is null")); + this(DEFAULT_EXTENSION); } - @Override - public synchronized Iterable getConnectorFactories() + public KafkaPlugin(Module extension) { - return ImmutableList.of(new KafkaConnectorFactory(tableDescriptionSupplier)); + this.extension = requireNonNull(extension, "extension is null"); } @Override - public Optional getConnectorWithProperties() + public synchronized Iterable getConnectorFactories() { - ConnectorConfig connectorConfig = KafkaPlugin.class.getAnnotation(ConnectorConfig.class); - return ConnectorUtil.assembleConnectorProperties(connectorConfig, - Arrays.asList(KafkaConnectorConfig.class.getDeclaredMethods())); + return ImmutableList.of(new KafkaConnectorFactory(extension)); } } diff --git a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaProducerModule.java b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaProducerModule.java new file mode 100644 index 000000000..72b7e4449 --- /dev/null +++ b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaProducerModule.java @@ -0,0 +1,28 @@ +/* + * 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.inject.Binder; +import com.google.inject.Module; +import com.google.inject.Scopes; + +public class KafkaProducerModule + implements Module +{ + @Override + public void configure(Binder binder) + { + binder.bind(PlainTextKafkaProducerFactory.class).in(Scopes.SINGLETON); + } +} diff --git a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaRecordSet.java b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaRecordSet.java index fc1804e5a..ad0e7cb52 100644 --- a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaRecordSet.java +++ b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaRecordSet.java @@ -13,20 +13,31 @@ */ package io.prestosql.plugin.kafka; +import com.google.common.collect.ArrayListMultimap; import com.google.common.collect.ImmutableList; +import com.google.common.collect.Multimap; import io.airlift.slice.Slice; import io.prestosql.decoder.DecoderColumnHandle; import io.prestosql.decoder.FieldValueProvider; import io.prestosql.decoder.RowDecoder; import io.prestosql.spi.block.Block; +import io.prestosql.spi.block.BlockBuilder; +import io.prestosql.spi.block.MapBlockBuilder; import io.prestosql.spi.connector.ColumnHandle; import io.prestosql.spi.connector.RecordCursor; import io.prestosql.spi.connector.RecordSet; +import io.prestosql.spi.type.ArrayType; +import io.prestosql.spi.type.MapType; import io.prestosql.spi.type.Type; +import io.prestosql.spi.type.VarbinaryType; +import io.prestosql.spi.type.VarcharType; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.KafkaConsumer; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.header.Header; +import org.apache.kafka.common.header.Headers; +import java.lang.invoke.MethodHandles; import java.util.HashMap; import java.util.Iterator; import java.util.List; @@ -37,7 +48,20 @@ import static com.google.common.base.Preconditions.checkArgument; 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.HEADERS_FIELD; +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.OFFSET_TIMESTAMP_FIELD; +import static io.prestosql.plugin.kafka.KafkaInternalFieldManager.PARTITION_ID_FIELD; +import static io.prestosql.plugin.kafka.KafkaInternalFieldManager.PARTITION_OFFSET_FIELD; +import static io.prestosql.spi.type.Timestamps.MICROSECONDS_PER_MILLISECOND; +import static io.prestosql.spi.type.TypeUtils.writeNativeValue; import static java.lang.Math.max; +import static java.lang.invoke.MethodType.methodType; import static java.util.Collections.emptyIterator; import static java.util.Objects.requireNonNull; @@ -46,6 +70,7 @@ public class KafkaRecordSet { private static final byte[] EMPTY_BYTE_ARRAY = new byte[0]; private static final int CONSUMER_POLL_TIMEOUT = 100; + private static final FieldValueProvider EMPTY_HEADERS_FIELD_PROVIDER = createEmptyHeadersFieldProvider(); private final KafkaSplit split; @@ -139,34 +164,35 @@ public class KafkaRecordSet records = kafkaConsumer.poll(CONSUMER_POLL_TIMEOUT).iterator(); return advanceNextPosition(); } - nextRow(records.next()); - return true; + + return nextRow(records.next()); } private boolean nextRow(ConsumerRecord message) { requireNonNull(message, "message is null"); - completedBytes += max(message.serializedKeySize(), 0) + max(message.serializedValueSize(), 0); - byte[] keyData = EMPTY_BYTE_ARRAY; - if (message.key() != null) { - keyData = message.key(); + if (message.offset() >= split.getMessagesRange().getEnd()) { + return false; } + completedBytes += max(message.serializedKeySize(), 0) + max(message.serializedValueSize(), 0); + + byte[] keyData = EMPTY_BYTE_ARRAY; byte[] messageData = EMPTY_BYTE_ARRAY; if (message.value() != null) { messageData = message.value(); } + long timeStamp = message.timestamp(); Map currentRowValuesMap = new HashMap<>(); - Optional> decodedKey = keyDecoder.decodeRow(keyData, null); - Optional> decodedValue = messageDecoder.decodeRow(messageData, null); + Optional> decodedKey = keyDecoder.decodeRow(keyData); + Optional> decodedValue = messageDecoder.decodeRow(messageData); 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; @@ -182,17 +208,24 @@ public class KafkaRecordSet case KEY_LENGTH_FIELD: currentRowValuesMap.put(columnHandle, longValueProvider(keyData.length)); break; + case OFFSET_TIMESTAMP_FIELD: + timeStamp *= MICROSECONDS_PER_MILLISECOND; + currentRowValuesMap.put(columnHandle, longValueProvider(timeStamp)); + break; case KEY_CORRUPT_FIELD: - currentRowValuesMap.put(columnHandle, booleanValueProvider(!decodedKey.isPresent())); + currentRowValuesMap.put(columnHandle, booleanValueProvider(decodedKey.isEmpty())); + break; + case HEADERS_FIELD: + currentRowValuesMap.put(columnHandle, headerMapValueProvider((MapType) columnHandle.getType(), message.headers())); break; case MESSAGE_CORRUPT_FIELD: - currentRowValuesMap.put(columnHandle, booleanValueProvider(!decodedValue.isPresent())); + currentRowValuesMap.put(columnHandle, booleanValueProvider(decodedValue.isEmpty())); break; case PARTITION_ID_FIELD: currentRowValuesMap.put(columnHandle, longValueProvider(message.partition())); break; default: - throw new IllegalArgumentException("unknown internal field " + fieldDescription); + throw new IllegalArgumentException("unknown internal field " + columnHandle.getName()); } } } @@ -264,4 +297,75 @@ public class KafkaRecordSet kafkaConsumer.close(); } } + + private static FieldValueProvider createEmptyHeadersFieldProvider() + { + MapType mapType = new MapType(VarcharType.VARCHAR, new ArrayType(VarbinaryType.VARBINARY), + MethodHandles.empty(methodType(Boolean.class, Block.class, int.class, long.class)), + MethodHandles.empty(methodType(Boolean.class, Block.class, int.class, Block.class, int.class)), + MethodHandles.empty(methodType(long.class, Object.class)), + MethodHandles.empty(methodType(long.class, Object.class))); + BlockBuilder mapBlockBuilder = new MapBlockBuilder(mapType, null, 0); + mapBlockBuilder.beginBlockEntry(); + mapBlockBuilder.closeEntry(); + Block emptyMapBlock = mapType.getObject(mapBlockBuilder, 0); + return new FieldValueProvider() { + @Override + public boolean isNull() + { + return false; + } + + @Override + public Block getBlock() + { + return emptyMapBlock; + } + }; + } + + public static FieldValueProvider headerMapValueProvider(MapType varcharMapType, Headers headers) + { + if (!headers.iterator().hasNext()) { + return EMPTY_HEADERS_FIELD_PROVIDER; + } + + Type keyType = varcharMapType.getTypeParameters().get(0); + Type valueArrayType = varcharMapType.getTypeParameters().get(1); + Type valueType = valueArrayType.getTypeParameters().get(0); + + BlockBuilder mapBlockBuilder = varcharMapType.createBlockBuilder(null, 1); + BlockBuilder builder = mapBlockBuilder.beginBlockEntry(); + + // Group by keys and collect values as array. + Multimap headerMap = ArrayListMultimap.create(); + for (Header header : headers) { + headerMap.put(header.key(), header.value()); + } + + for (String headerKey : headerMap.keySet()) { + writeNativeValue(keyType, builder, headerKey); + BlockBuilder arrayBuilder = builder.beginBlockEntry(); + for (byte[] value : headerMap.get(headerKey)) { + writeNativeValue(valueType, arrayBuilder, value); + } + builder.closeEntry(); + } + + mapBlockBuilder.closeEntry(); + + return new FieldValueProvider() { + @Override + public boolean isNull() + { + return false; + } + + @Override + public Block getBlock() + { + return varcharMapType.getObject(mapBlockBuilder, 0); + } + }; + } } diff --git a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaRecordSetProvider.java b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaRecordSetProvider.java index b58b9486c..dbe971c0a 100644 --- a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaRecordSetProvider.java +++ b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaRecordSetProvider.java @@ -65,7 +65,7 @@ public class KafkaRecordSetProvider getDecoderParameters(kafkaSplit.getKeyDataSchemaContents()), kafkaColumns.stream() .filter(col -> !col.isInternal()) - .filter(KafkaColumnHandle::isKeyDecoder) + .filter(KafkaColumnHandle::isKeyCodec) .collect(toImmutableSet())); RowDecoder messageDecoder = decoderFactory.create( @@ -73,7 +73,7 @@ public class KafkaRecordSetProvider getDecoderParameters(kafkaSplit.getMessageDataSchemaContents()), kafkaColumns.stream() .filter(col -> !col.isInternal()) - .filter(col -> !col.isKeyDecoder()) + .filter(col -> !col.isKeyCodec()) .collect(toImmutableSet())); return new KafkaRecordSet(kafkaSplit, consumerFactory, kafkaColumns, keyDecoder, messageDecoder); diff --git a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaSessionProperties.java b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaSessionProperties.java new file mode 100644 index 000000000..333d4dc4f --- /dev/null +++ b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaSessionProperties.java @@ -0,0 +1,54 @@ +/* + * 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.ImmutableList; +import io.prestosql.spi.connector.ConnectorSession; +import io.prestosql.spi.session.PropertyMetadata; + +import javax.inject.Inject; + +import java.util.List; + +public final class KafkaSessionProperties +{ + private static final String TIMESTAMP_UPPER_BOUND_FORCE_PUSH_DOWN_ENABLED = "timestamp_upper_bound_force_push_down_enabled"; + private final List> sessionProperties; + + @Inject + public KafkaSessionProperties(KafkaConfig kafkaConfig) + { + sessionProperties = ImmutableList.of(PropertyMetadata.booleanProperty( + TIMESTAMP_UPPER_BOUND_FORCE_PUSH_DOWN_ENABLED, + "Enable or disable timestamp upper bound push down for topic createTime mode", + kafkaConfig.isTimestampUpperBoundPushDownEnabled(), false)); + } + + public List> getSessionProperties() + { + return sessionProperties; + } + + /** + * If predicate specifies lower bound on _timestamp column (_timestamp > XXXX), it is always pushed down. + * The upper bound predicate is pushed down only for topics using ``LogAppendTime`` mode. + * For topics using ``CreateTime`` mode, upper bound push down must be explicitly + * allowed via ``kafka.timestamp-upper-bound-force-push-down-enabled`` config property + * or ``timestamp_upper_bound_force_push_down_enabled`` session property. + */ + public static boolean isTimestampUpperBoundPushdownEnabled(ConnectorSession session) + { + return session.getProperty(TIMESTAMP_UPPER_BOUND_FORCE_PUSH_DOWN_ENABLED, Boolean.class); + } +} diff --git a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaSplitManager.java b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaSplitManager.java index b47e11fc1..ea0276be7 100644 --- a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaSplitManager.java +++ b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaSplitManager.java @@ -22,6 +22,7 @@ import io.prestosql.spi.connector.ConnectorSplitManager; import io.prestosql.spi.connector.ConnectorSplitSource; import io.prestosql.spi.connector.ConnectorTableHandle; import io.prestosql.spi.connector.ConnectorTransactionHandle; +import io.prestosql.spi.connector.DynamicFilter; import io.prestosql.spi.connector.FixedSplitSource; import org.apache.kafka.clients.consumer.KafkaConsumer; import org.apache.kafka.common.PartitionInfo; @@ -52,17 +53,24 @@ public class KafkaSplitManager implements ConnectorSplitManager { private final KafkaConsumerFactory consumerFactory; + private final KafkaFilterManager kafkaFilterManager; private final int messagesPerSplit; @Inject - public KafkaSplitManager(KafkaConsumerFactory consumerFactory, KafkaConfig kafkaConfig) + public KafkaSplitManager(KafkaConsumerFactory consumerFactory, KafkaConfig kafkaConfig, KafkaFilterManager kafkaFilterManager) { this.consumerFactory = requireNonNull(consumerFactory, "consumerManager is null"); - messagesPerSplit = requireNonNull(kafkaConfig, "kafkaConfig is null").getMessagesPerSplit(); + this.messagesPerSplit = requireNonNull(kafkaConfig, "kafkaConfig is null").getMessagesPerSplit(); + this.kafkaFilterManager = requireNonNull(kafkaFilterManager, "kafkaFilterManager is null"); } @Override - public ConnectorSplitSource getSplits(ConnectorTransactionHandle transaction, ConnectorSession session, ConnectorTableHandle table, SplitSchedulingStrategy splitSchedulingStrategy) + public ConnectorSplitSource getSplits( + ConnectorTransactionHandle transaction, + ConnectorSession session, + ConnectorTableHandle table, + SplitSchedulingStrategy splitSchedulingStrategy, + DynamicFilter dynamicFilter) { KafkaTableHandle kafkaTableHandle = (KafkaTableHandle) table; try (KafkaConsumer kafkaConsumer = consumerFactory.create()) { @@ -74,12 +82,18 @@ public class KafkaSplitManager Map partitionBeginOffsets = kafkaConsumer.beginningOffsets(topicPartitions); Map partitionEndOffsets = kafkaConsumer.endOffsets(topicPartitions); + KafkaFilteringResult kafkaFilteringResult = kafkaFilterManager.getKafkaFilterResult(session, kafkaTableHandle, + partitionInfos, partitionBeginOffsets, partitionEndOffsets); + partitionInfos = kafkaFilteringResult.getPartitionInfos(); + partitionBeginOffsets = kafkaFilteringResult.getPartitionBeginOffsets(); + partitionEndOffsets = kafkaFilteringResult.getPartitionEndOffsets(); ImmutableList.Builder splits = ImmutableList.builder(); Optional keyDataSchemaContents = kafkaTableHandle.getKeyDataSchemaLocation() .map(KafkaSplitManager::readSchema); Optional messageDataSchemaContents = kafkaTableHandle.getMessageDataSchemaLocation() .map(KafkaSplitManager::readSchema); + for (PartitionInfo partitionInfo : partitionInfos) { TopicPartition topicPartition = toTopicPartition(partitionInfo); HostAddress leader = HostAddress.fromParts(partitionInfo.leader().host(), partitionInfo.leader().port()); diff --git a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaTableDescriptionSupplier.java b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaTableDescriptionSupplier.java index a2c07bfe9..ac9f8de37 100644 --- a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaTableDescriptionSupplier.java +++ b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaTableDescriptionSupplier.java @@ -23,6 +23,7 @@ import io.prestosql.decoder.dummy.DummyRowDecoder; import io.prestosql.spi.connector.SchemaTableName; import javax.inject.Inject; +import javax.inject.Provider; import java.io.File; import java.io.IOException; @@ -31,7 +32,6 @@ import java.util.List; import java.util.Map; import java.util.Optional; import java.util.Set; -import java.util.function.Supplier; import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Strings.isNullOrEmpty; @@ -40,7 +40,7 @@ import static java.util.Arrays.asList; import static java.util.Objects.requireNonNull; public class KafkaTableDescriptionSupplier - implements Supplier> + implements Provider { private static final Logger log = Logger.get(KafkaTableDescriptionSupplier.class); @@ -50,24 +50,29 @@ public class KafkaTableDescriptionSupplier private final Set tableNames; @Inject - KafkaTableDescriptionSupplier(KafkaConnectorConfig kafkaConnectorConfig, - JsonCodec topicDescriptionCodec) + KafkaTableDescriptionSupplier(KafkaConfig kafkaConfig, JsonCodec topicDescriptionCodec) { this.topicDescriptionCodec = requireNonNull(topicDescriptionCodec, "topicDescriptionCodec is null"); - - requireNonNull(kafkaConnectorConfig, "kafkaConfig is null"); - this.tableDescriptionDir = kafkaConnectorConfig.getTableDescriptionDir(); - this.defaultSchema = kafkaConnectorConfig.getDefaultSchema(); - this.tableNames = ImmutableSet.copyOf(kafkaConnectorConfig.getTableNames()); + requireNonNull(kafkaConfig, "kafkaConfig is null"); + this.tableDescriptionDir = kafkaConfig.getTableDescriptionDir(); + this.defaultSchema = kafkaConfig.getDefaultSchema(); + this.tableNames = ImmutableSet.copyOf(kafkaConfig.getTableNames()); } @Override - public Map get() + public TableDescriptionSupplier get() + { + Map tables = populateTables(); + return new MapBasedTableDescriptionSupplier(tables); + } + + private Map populateTables() { ImmutableMap.Builder builder = ImmutableMap.builder(); + log.debug("Loading kafka table definitions from %s", tableDescriptionDir.getAbsolutePath()); + try { - log.debug("Loading kafka table definitions from %s", tableDescriptionDir.getCanonicalPath()); for (File file : listFiles(tableDescriptionDir)) { if (file.isFile() && file.getName().endsWith(".json")) { KafkaTopicDescription table = topicDescriptionCodec.fromJson(readAllBytes(file.toPath())); diff --git a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaTableHandle.java b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaTableHandle.java index 9d628f7b4..4b61539da 100644 --- a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaTableHandle.java +++ b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaTableHandle.java @@ -15,24 +15,26 @@ package io.prestosql.plugin.kafka; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.ImmutableList; +import io.prestosql.spi.connector.ColumnHandle; +import io.prestosql.spi.connector.ConnectorInsertTableHandle; import io.prestosql.spi.connector.ConnectorTableHandle; import io.prestosql.spi.connector.SchemaTableName; +import io.prestosql.spi.predicate.TupleDomain; +import java.util.List; import java.util.Objects; import java.util.Optional; import static com.google.common.base.MoreObjects.toStringHelper; import static java.util.Objects.requireNonNull; -/** - * Kafka specific {@link ConnectorTableHandle}. - */ public final class KafkaTableHandle - implements ConnectorTableHandle + implements ConnectorTableHandle, ConnectorInsertTableHandle { /** * The schema name for this table. Is set through configuration and read - * using {@link KafkaConnectorConfig#getDefaultSchema()}. Usually 'default'. + * using {@link KafkaConfig#getDefaultSchema()}. Usually 'default'. */ private final String schemaName; @@ -50,6 +52,8 @@ public final class KafkaTableHandle private final String messageDataFormat; private final Optional keyDataSchemaLocation; private final Optional messageDataSchemaLocation; + private final List columns; + private final TupleDomain constraint; @JsonCreator public KafkaTableHandle( @@ -59,15 +63,19 @@ public final class KafkaTableHandle @JsonProperty("keyDataFormat") String keyDataFormat, @JsonProperty("messageDataFormat") String messageDataFormat, @JsonProperty("keyDataSchemaLocation") Optional keyDataSchemaLocation, - @JsonProperty("messageDataSchemaLocation") Optional messageDataSchemaLocation) + @JsonProperty("messageDataSchemaLocation") Optional messageDataSchemaLocation, + @JsonProperty("columns") List columns, + @JsonProperty("constraint") TupleDomain constraint) { this.schemaName = requireNonNull(schemaName, "schemaName is null"); this.tableName = requireNonNull(tableName, "tableName is null"); this.topicName = requireNonNull(topicName, "topicName is null"); this.keyDataFormat = requireNonNull(keyDataFormat, "keyDataFormat is null"); this.messageDataFormat = requireNonNull(messageDataFormat, "messageDataFormat is null"); - this.keyDataSchemaLocation = keyDataSchemaLocation; - this.messageDataSchemaLocation = messageDataSchemaLocation; + this.keyDataSchemaLocation = requireNonNull(keyDataSchemaLocation, "keyDataSchemaLocation is null"); + this.messageDataSchemaLocation = requireNonNull(messageDataSchemaLocation, "messageDataSchemaLocation is null"); + this.columns = requireNonNull(ImmutableList.copyOf(columns), "columns is null"); + this.constraint = requireNonNull(constraint, "constraint is null"); } @JsonProperty @@ -112,6 +120,18 @@ public final class KafkaTableHandle return keyDataSchemaLocation; } + @JsonProperty + public List getColumns() + { + return columns; + } + + @JsonProperty + public TupleDomain getConstraint() + { + return constraint; + } + public SchemaTableName toSchemaTableName() { return new SchemaTableName(schemaName, tableName); @@ -120,7 +140,7 @@ public final class KafkaTableHandle @Override public int hashCode() { - return Objects.hash(schemaName, tableName, topicName, keyDataFormat, messageDataFormat, keyDataSchemaLocation, messageDataSchemaLocation); + return Objects.hash(schemaName, tableName, topicName, keyDataFormat, messageDataFormat, keyDataSchemaLocation, messageDataSchemaLocation, columns, constraint); } @Override @@ -140,7 +160,9 @@ public final class KafkaTableHandle && Objects.equals(this.keyDataFormat, other.keyDataFormat) && Objects.equals(this.messageDataFormat, other.messageDataFormat) && Objects.equals(this.keyDataSchemaLocation, other.keyDataSchemaLocation) - && Objects.equals(this.messageDataSchemaLocation, other.messageDataSchemaLocation); + && Objects.equals(this.messageDataSchemaLocation, other.messageDataSchemaLocation) + && Objects.equals(this.columns, other.columns) + && Objects.equals(this.constraint, other.constraint); } @Override @@ -154,6 +176,8 @@ public final class KafkaTableHandle .add("messageDataFormat", messageDataFormat) .add("keyDataSchemaLocation", keyDataSchemaLocation) .add("messageDataSchemaLocation", messageDataSchemaLocation) + .add("columns", columns) + .add("constraint", constraint) .toString(); } } diff --git a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaTopicFieldDescription.java b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaTopicFieldDescription.java index d174ee89d..b97e63b57 100644 --- a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaTopicFieldDescription.java +++ b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/KafkaTopicFieldDescription.java @@ -100,16 +100,15 @@ public final class KafkaTopicFieldDescription return hidden; } - KafkaColumnHandle getColumnHandle(boolean keyDecoder, int index) + KafkaColumnHandle getColumnHandle(boolean keyCodec, int index) { return new KafkaColumnHandle( - index, getName(), getType(), getMapping(), getDataFormat(), getFormatHint(), - keyDecoder, + keyCodec, isHidden(), false); } diff --git a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/MapBasedTableDescriptionSupplier.java b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/MapBasedTableDescriptionSupplier.java new file mode 100644 index 000000000..1b1a74415 --- /dev/null +++ b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/MapBasedTableDescriptionSupplier.java @@ -0,0 +1,47 @@ +/* + * 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 io.prestosql.spi.connector.SchemaTableName; + +import java.util.Map; +import java.util.Optional; +import java.util.Set; + +import static java.util.Objects.requireNonNull; + +public class MapBasedTableDescriptionSupplier + implements TableDescriptionSupplier +{ + private final Map map; + + public MapBasedTableDescriptionSupplier(Map map) + { + this.map = ImmutableMap.copyOf(requireNonNull(map, "map is null")); + } + + @Override + public Set listTables() + { + return map.keySet(); + } + + @Override + public Optional getTopicDescription(SchemaTableName schemaTableName) + { + return Optional.ofNullable(map.get(schemaTableName)); + } +} diff --git a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/PlainTextKafkaConsumerFactory.java b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/PlainTextKafkaConsumerFactory.java new file mode 100644 index 000000000..3fc383efa --- /dev/null +++ b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/PlainTextKafkaConsumerFactory.java @@ -0,0 +1,63 @@ +/* + * 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 io.airlift.units.DataSize; +import io.prestosql.spi.HostAddress; +import org.apache.kafka.common.serialization.ByteArrayDeserializer; + +import javax.inject.Inject; + +import java.util.Properties; +import java.util.Set; + +import static java.util.Objects.requireNonNull; +import static java.util.stream.Collectors.joining; +import static org.apache.kafka.clients.consumer.ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG; +import static org.apache.kafka.clients.consumer.ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG; +import static org.apache.kafka.clients.consumer.ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG; +import static org.apache.kafka.clients.consumer.ConsumerConfig.RECEIVE_BUFFER_CONFIG; +import static org.apache.kafka.clients.consumer.ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG; + +public class PlainTextKafkaConsumerFactory + implements KafkaConsumerFactory +{ + private final Set nodes; + private final DataSize kafkaBufferSize; + + @Inject + public PlainTextKafkaConsumerFactory(KafkaConfig kafkaConfig) + { + requireNonNull(kafkaConfig, "kafkaConfig is null"); + + nodes = kafkaConfig.getNodes(); + kafkaBufferSize = kafkaConfig.getKafkaBufferSize(); + } + + @Override + public Properties configure() + { + Properties properties = new Properties(); + properties.put( + BOOTSTRAP_SERVERS_CONFIG, + nodes.stream() + .map(HostAddress::toString) + .collect(joining(","))); + properties.put(KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName()); + properties.put(VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName()); + properties.put(RECEIVE_BUFFER_CONFIG, Long.toString(kafkaBufferSize.toBytes())); + properties.put(ENABLE_AUTO_COMMIT_CONFIG, Boolean.toString(false)); + return properties; + } +} diff --git a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/PlainTextKafkaProducerFactory.java b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/PlainTextKafkaProducerFactory.java new file mode 100644 index 000000000..d0637760e --- /dev/null +++ b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/PlainTextKafkaProducerFactory.java @@ -0,0 +1,58 @@ +/* + * 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.common.collect.ImmutableSet; +import io.prestosql.spi.HostAddress; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.common.serialization.ByteArraySerializer; + +import javax.inject.Inject; + +import java.util.Map; +import java.util.Set; + +import static java.util.Objects.requireNonNull; +import static java.util.stream.Collectors.joining; +import static org.apache.kafka.clients.producer.ProducerConfig.ACKS_CONFIG; +import static org.apache.kafka.clients.producer.ProducerConfig.BOOTSTRAP_SERVERS_CONFIG; +import static org.apache.kafka.clients.producer.ProducerConfig.LINGER_MS_CONFIG; + +public class PlainTextKafkaProducerFactory +{ + private final Map properties; + + @Inject + public PlainTextKafkaProducerFactory(KafkaConfig kafkaConfig) + { + requireNonNull(kafkaConfig, "kafkaConfig is null"); + Set nodes = ImmutableSet.copyOf(kafkaConfig.getNodes()); + properties = ImmutableMap.builder() + .put(BOOTSTRAP_SERVERS_CONFIG, nodes.stream() + .map(HostAddress::toString) + .collect(joining(","))) + .put(ACKS_CONFIG, "all") + .put(LINGER_MS_CONFIG, 5) + .build(); + } + + /** + * Creates a KafkaProducer with the properties set in the constructor. + */ + public KafkaProducer create() + { + return new KafkaProducer<>(properties, new ByteArraySerializer(), new ByteArraySerializer()); + } +} diff --git a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/TableDescriptionSupplier.java b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/TableDescriptionSupplier.java new file mode 100644 index 000000000..c91d22899 --- /dev/null +++ b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/TableDescriptionSupplier.java @@ -0,0 +1,27 @@ +/* + * 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 io.prestosql.spi.connector.SchemaTableName; + +import java.util.Optional; +import java.util.Set; + +public interface TableDescriptionSupplier +{ + Set listTables(); + + Optional getTopicDescription(SchemaTableName schemaTableName); +} diff --git a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/encoder/AbstractRowEncoder.java b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/encoder/AbstractRowEncoder.java new file mode 100644 index 000000000..962e011fc --- /dev/null +++ b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/encoder/AbstractRowEncoder.java @@ -0,0 +1,215 @@ +/* + * 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.encoder; + +import com.google.common.collect.ImmutableList; +import com.google.common.primitives.Shorts; +import com.google.common.primitives.SignedBytes; +import io.prestosql.spi.block.Block; +import io.prestosql.spi.connector.ConnectorSession; +import io.prestosql.spi.type.SqlDate; +import io.prestosql.spi.type.SqlTime; +import io.prestosql.spi.type.SqlTimeWithTimeZone; +import io.prestosql.spi.type.SqlTimestamp; +import io.prestosql.spi.type.SqlTimestampWithTimeZone; +import io.prestosql.spi.type.TimestampType; +import io.prestosql.spi.type.TimestampWithTimeZoneType; +import io.prestosql.spi.type.Type; + +import java.nio.ByteBuffer; +import java.util.List; + +import static com.google.common.base.Preconditions.checkArgument; +import static io.prestosql.spi.type.BigintType.BIGINT; +import static io.prestosql.spi.type.BooleanType.BOOLEAN; +import static io.prestosql.spi.type.DateType.DATE; +import static io.prestosql.spi.type.DoubleType.DOUBLE; +import static io.prestosql.spi.type.IntegerType.INTEGER; +import static io.prestosql.spi.type.RealType.REAL; +import static io.prestosql.spi.type.SmallintType.SMALLINT; +import static io.prestosql.spi.type.TimeType.TIME; +import static io.prestosql.spi.type.TimeWithTimeZoneType.TIME_WITH_TIME_ZONE; +import static io.prestosql.spi.type.TinyintType.TINYINT; +import static io.prestosql.spi.type.VarbinaryType.isVarbinaryType; +import static io.prestosql.spi.type.Varchars.isVarcharType; +import static java.lang.Float.intBitsToFloat; +import static java.lang.Math.toIntExact; +import static java.lang.String.format; +import static java.util.Objects.requireNonNull; + +public abstract class AbstractRowEncoder + implements RowEncoder +{ + protected final ConnectorSession session; + protected final List columnHandles; + + /** + * The current column index for appending values to the row encoder. + * Gets incremented by appendColumnValue and set back to zero when the encoder is reset. + */ + protected int currentColumnIndex; + + protected AbstractRowEncoder(ConnectorSession session, List columnHandles) + { + this.session = requireNonNull(session, "session is null"); + requireNonNull(columnHandles, "columnHandles is null"); + this.columnHandles = ImmutableList.copyOf(columnHandles); + this.currentColumnIndex = 0; + } + + @Override + public RowEncoder appendColumnValue(Block block, int position) + { + checkArgument(currentColumnIndex < columnHandles.size(), format("currentColumnIndex '%d' is greater than number of columns '%d'", currentColumnIndex, columnHandles.size())); + Type type = columnHandles.get(currentColumnIndex).getType(); + if (block.isNull(position)) { + appendNullValue(); + } + else if (type == BOOLEAN) { + appendBoolean(type.getBoolean(block, position)); + } + else if (type == BIGINT) { + appendLong(type.getLong(block, position)); + } + else if (type == INTEGER) { + appendInt(toIntExact(type.getLong(block, position))); + } + else if (type == SMALLINT) { + appendShort(Shorts.checkedCast(type.getLong(block, position))); + } + else if (type == TINYINT) { + appendByte(SignedBytes.checkedCast(type.getLong(block, position))); + } + else if (type == DOUBLE) { + appendDouble(type.getDouble(block, position)); + } + else if (type == REAL) { + appendFloat(intBitsToFloat(toIntExact(type.getLong(block, position)))); + } + else if (isVarcharType(type)) { + appendString(type.getSlice(block, position).toStringUtf8()); + } + else if (isVarbinaryType(type)) { + appendByteBuffer(type.getSlice(block, position).toByteBuffer()); + } + else if (type == DATE) { + appendSqlDate((SqlDate) type.getObjectValue(session, block, position)); + } + else if (type == TIME) { + appendSqlTime((SqlTime) type.getObjectValue(session, block, position)); + } + else if (type == TIME_WITH_TIME_ZONE) { + appendSqlTimeWithTimeZone((SqlTimeWithTimeZone) type.getObjectValue(session, block, position)); + } + else if (type instanceof TimestampType) { + appendSqlTimestamp((SqlTimestamp) type.getObjectValue(session, block, position)); + } + else if (type instanceof TimestampWithTimeZoneType) { + appendSqlTimestampWithTimeZone((SqlTimestampWithTimeZone) type.getObjectValue(session, block, position)); + } + else { + throw new UnsupportedOperationException(format("Column '%s' does not support 'null' value", columnHandles.get(currentColumnIndex).getName())); + } + currentColumnIndex++; + return this; + } + + // these append value methods should be overridden for each row encoder + // only the methods with types supported by the data format should be overridden + protected void appendNullValue() + { + throw new UnsupportedOperationException(format("Column '%s' does not support 'null' value", columnHandles.get(currentColumnIndex).getName())); + } + + protected void appendLong(long value) + { + throw new UnsupportedOperationException(format("Unsupported type '%s' for column '%s'", long.class.getName(), columnHandles.get(currentColumnIndex).getName())); + } + + protected void appendInt(int value) + { + throw new UnsupportedOperationException(format("Unsupported type '%s' for column '%s'", int.class.getName(), columnHandles.get(currentColumnIndex).getName())); + } + + protected void appendShort(short value) + { + throw new UnsupportedOperationException(format("Unsupported type '%s' for column '%s'", short.class.getName(), columnHandles.get(currentColumnIndex).getName())); + } + + protected void appendByte(byte value) + { + throw new UnsupportedOperationException(format("Unsupported type '%s' for column '%s'", byte.class.getName(), columnHandles.get(currentColumnIndex).getName())); + } + + protected void appendDouble(double value) + { + throw new UnsupportedOperationException(format("Unsupported type '%s' for column '%s'", double.class.getName(), columnHandles.get(currentColumnIndex).getName())); + } + + protected void appendFloat(float value) + { + throw new UnsupportedOperationException(format("Unsupported type '%s' for column '%s'", float.class.getName(), columnHandles.get(currentColumnIndex).getName())); + } + + protected void appendBoolean(boolean value) + { + throw new UnsupportedOperationException(format("Unsupported type '%s' for column '%s'", boolean.class.getName(), columnHandles.get(currentColumnIndex).getName())); + } + + protected void appendString(String value) + { + throw new UnsupportedOperationException(format("Unsupported type '%s' for column '%s'", value.getClass().getName(), columnHandles.get(currentColumnIndex).getName())); + } + + protected void appendByteBuffer(ByteBuffer value) + { + throw new UnsupportedOperationException(format("Unsupported type '%s' for column '%s'", value.getClass().getName(), columnHandles.get(currentColumnIndex).getName())); + } + + protected void appendSqlDate(SqlDate value) + { + throw new UnsupportedOperationException(format("Unsupported type '%s' for column '%s'", value.getClass().getName(), columnHandles.get(currentColumnIndex).getName())); + } + + protected void appendSqlTime(SqlTime value) + { + throw new UnsupportedOperationException(format("Unsupported type '%s' for column '%s'", value.getClass().getName(), columnHandles.get(currentColumnIndex).getName())); + } + + protected void appendSqlTimeWithTimeZone(SqlTimeWithTimeZone value) + { + throw new UnsupportedOperationException(format("Unsupported type '%s' for column '%s'", value.getClass().getName(), columnHandles.get(currentColumnIndex).getName())); + } + + protected void appendSqlTimestamp(SqlTimestamp value) + { + throw new UnsupportedOperationException(format("Unsupported type '%s' for column '%s'", value.getClass().getName(), columnHandles.get(currentColumnIndex).getName())); + } + + protected void appendSqlTimestampWithTimeZone(SqlTimestampWithTimeZone value) + { + throw new UnsupportedOperationException(format("Unsupported type '%s' for column '%s'", value.getClass().getName(), columnHandles.get(currentColumnIndex).getName())); + } + + protected void resetColumnIndex() + { + currentColumnIndex = 0; + } + + @Override + public void close() + { + // do nothing + } +} diff --git a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/encoder/DispatchingRowEncoderFactory.java b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/encoder/DispatchingRowEncoderFactory.java new file mode 100644 index 000000000..ffbb26576 --- /dev/null +++ b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/encoder/DispatchingRowEncoderFactory.java @@ -0,0 +1,43 @@ +/* + * 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.encoder; + +import com.google.common.collect.ImmutableMap; +import io.prestosql.spi.connector.ConnectorSession; + +import javax.inject.Inject; + +import java.util.List; +import java.util.Map; +import java.util.Optional; + +import static com.google.common.base.Preconditions.checkArgument; +import static java.util.Objects.requireNonNull; + +public class DispatchingRowEncoderFactory +{ + private final Map factories; + + @Inject + public DispatchingRowEncoderFactory(Map factories) + { + this.factories = ImmutableMap.copyOf(requireNonNull(factories, "factories is null")); + } + + public RowEncoder create(ConnectorSession session, String dataFormat, Optional dataSchema, List columnHandles) + { + checkArgument(factories.containsKey(dataFormat), "unknown data format '%s'", dataFormat); + return factories.get(dataFormat).create(session, dataSchema, columnHandles); + } +} diff --git a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/encoder/EncoderColumnHandle.java b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/encoder/EncoderColumnHandle.java new file mode 100644 index 000000000..293f1aa5d --- /dev/null +++ b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/encoder/EncoderColumnHandle.java @@ -0,0 +1,33 @@ +/* + * 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.encoder; + +import io.prestosql.spi.connector.ColumnHandle; +import io.prestosql.spi.type.Type; + +public interface EncoderColumnHandle + extends ColumnHandle +{ + boolean isInternal(); + + String getFormatHint(); + + Type getType(); + + String getName(); + + String getMapping(); + + String getDataFormat(); +} diff --git a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/encoder/EncoderModule.java b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/encoder/EncoderModule.java new file mode 100644 index 000000000..fa07ff69f --- /dev/null +++ b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/encoder/EncoderModule.java @@ -0,0 +1,45 @@ +/* + * 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.encoder; + +import com.google.inject.Binder; +import com.google.inject.Module; +import com.google.inject.multibindings.MapBinder; +import io.prestosql.plugin.kafka.encoder.avro.AvroRowEncoder; +import io.prestosql.plugin.kafka.encoder.avro.AvroRowEncoderFactory; +import io.prestosql.plugin.kafka.encoder.csv.CsvRowEncoder; +import io.prestosql.plugin.kafka.encoder.csv.CsvRowEncoderFactory; +import io.prestosql.plugin.kafka.encoder.json.JsonRowEncoder; +import io.prestosql.plugin.kafka.encoder.json.JsonRowEncoderFactory; +import io.prestosql.plugin.kafka.encoder.raw.RawRowEncoder; +import io.prestosql.plugin.kafka.encoder.raw.RawRowEncoderFactory; + +import static com.google.inject.Scopes.SINGLETON; + +public class EncoderModule + implements Module +{ + @Override + public void configure(Binder binder) + { + MapBinder encoderFactoriesByName = MapBinder.newMapBinder(binder, String.class, RowEncoderFactory.class); + + encoderFactoriesByName.addBinding(AvroRowEncoder.NAME).to(AvroRowEncoderFactory.class).in(SINGLETON); + encoderFactoriesByName.addBinding(CsvRowEncoder.NAME).to(CsvRowEncoderFactory.class).in(SINGLETON); + encoderFactoriesByName.addBinding(RawRowEncoder.NAME).to(RawRowEncoderFactory.class).in(SINGLETON); + encoderFactoriesByName.addBinding(JsonRowEncoder.NAME).to(JsonRowEncoderFactory.class).in(SINGLETON); + + binder.bind(DispatchingRowEncoderFactory.class).in(SINGLETON); + } +} diff --git a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/encoder/RowEncoder.java b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/encoder/RowEncoder.java new file mode 100644 index 000000000..99b274767 --- /dev/null +++ b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/encoder/RowEncoder.java @@ -0,0 +1,32 @@ +/* + * 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.encoder; + +import io.prestosql.spi.block.Block; + +import java.io.Closeable; + +public interface RowEncoder + extends Closeable +{ + /** + * Adds the value from the given block/position to the row being encoded + */ + RowEncoder appendColumnValue(Block block, int position); + + /** + * Returns the encoded values as a byte array, and resets any info needed to prepare for next row + */ + byte[] toByteArray(); +} diff --git a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/encoder/RowEncoderFactory.java b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/encoder/RowEncoderFactory.java new file mode 100644 index 000000000..5406ea434 --- /dev/null +++ b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/encoder/RowEncoderFactory.java @@ -0,0 +1,24 @@ +/* + * 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.encoder; + +import io.prestosql.spi.connector.ConnectorSession; + +import java.util.List; +import java.util.Optional; + +public interface RowEncoderFactory +{ + RowEncoder create(ConnectorSession session, Optional dataSchema, List columnHandles); +} diff --git a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/encoder/avro/AvroRowEncoder.java b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/encoder/avro/AvroRowEncoder.java new file mode 100644 index 000000000..72a4ef232 --- /dev/null +++ b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/encoder/avro/AvroRowEncoder.java @@ -0,0 +1,160 @@ +/* + * 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.encoder.avro; + +import com.google.common.collect.ImmutableSet; +import io.prestosql.plugin.kafka.encoder.AbstractRowEncoder; +import io.prestosql.plugin.kafka.encoder.EncoderColumnHandle; +import io.prestosql.spi.connector.ConnectorSession; +import io.prestosql.spi.type.Type; +import org.apache.avro.Schema; +import org.apache.avro.file.DataFileWriter; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericDatumWriter; +import org.apache.avro.generic.GenericRecord; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.List; +import java.util.Set; + +import static com.google.common.base.Preconditions.checkArgument; +import static io.prestosql.spi.type.BigintType.BIGINT; +import static io.prestosql.spi.type.BooleanType.BOOLEAN; +import static io.prestosql.spi.type.DoubleType.DOUBLE; +import static io.prestosql.spi.type.IntegerType.INTEGER; +import static io.prestosql.spi.type.RealType.REAL; +import static io.prestosql.spi.type.Varchars.isVarcharType; +import static java.lang.String.format; +import static java.util.Objects.requireNonNull; + +public class AvroRowEncoder + extends AbstractRowEncoder +{ + private static final Set SUPPORTED_PRIMITIVE_TYPES = ImmutableSet.of( + BOOLEAN, INTEGER, BIGINT, DOUBLE, REAL); + + public static final String NAME = "avro"; + + private final ByteArrayOutputStream byteArrayOutputStream; + private final Schema parsedSchema; + private final DataFileWriter dataFileWriter; + private final GenericRecord record; + + public AvroRowEncoder(ConnectorSession session, List columnHandles, Schema parsedSchema) + { + super(session, columnHandles); + for (EncoderColumnHandle columnHandle : this.columnHandles) { + checkArgument(columnHandle.getFormatHint() == null, "Unexpected format hint '%s' defined for column '%s'", columnHandle.getFormatHint(), columnHandle.getName()); + checkArgument(columnHandle.getDataFormat() == null, "Unexpected data format '%s' defined for column '%s'", columnHandle.getDataFormat(), columnHandle.getName()); + + checkArgument(isSupportedType(columnHandle.getType()), "Unsupported column type '%s' for column '%s'", columnHandle.getType(), columnHandle.getName()); + } + this.byteArrayOutputStream = new ByteArrayOutputStream(); + this.parsedSchema = requireNonNull(parsedSchema, "parsedSchema is null"); + this.dataFileWriter = new DataFileWriter<>(new GenericDatumWriter<>(this.parsedSchema)); + this.record = new GenericData.Record(this.parsedSchema); + } + + private boolean isSupportedType(Type type) + { + return isVarcharType(type) || SUPPORTED_PRIMITIVE_TYPES.contains(type); + } + + @Override + protected void appendNullValue() + { + record.put(columnHandles.get(currentColumnIndex).getName(), null); + } + + @Override + protected void appendLong(long value) + { + record.put(columnHandles.get(currentColumnIndex).getName(), value); + } + + @Override + protected void appendInt(int value) + { + record.put(columnHandles.get(currentColumnIndex).getName(), value); + } + + @Override + protected void appendShort(short value) + { + record.put(columnHandles.get(currentColumnIndex).getName(), value); + } + + @Override + protected void appendByte(byte value) + { + record.put(columnHandles.get(currentColumnIndex).getName(), value); + } + + @Override + protected void appendDouble(double value) + { + record.put(columnHandles.get(currentColumnIndex).getName(), value); + } + + @Override + protected void appendFloat(float value) + { + record.put(columnHandles.get(currentColumnIndex).getName(), value); + } + + @Override + protected void appendBoolean(boolean value) + { + record.put(columnHandles.get(currentColumnIndex).getName(), value); + } + + @Override + protected void appendString(String value) + { + record.put(columnHandles.get(currentColumnIndex).getName(), value); + } + + @Override + public byte[] toByteArray() + { + // make sure entire row has been updated with new values + checkArgument(currentColumnIndex == columnHandles.size(), format("Missing %d columns", columnHandles.size() - currentColumnIndex + 1)); + + try { + byteArrayOutputStream.reset(); + dataFileWriter.create(parsedSchema, byteArrayOutputStream); + dataFileWriter.append(record); + dataFileWriter.close(); + + resetColumnIndex(); // reset currentColumnIndex to prepare for next row + return byteArrayOutputStream.toByteArray(); + } + catch (IOException e) { + throw new UncheckedIOException("Failed to append record", e); + } + } + + @Override + public void close() + { + try { + byteArrayOutputStream.close(); + } + catch (IOException e) { + throw new UncheckedIOException("Failed to close ByteArrayOutputStream", e); + } + } +} diff --git a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/encoder/avro/AvroRowEncoderFactory.java b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/encoder/avro/AvroRowEncoderFactory.java new file mode 100644 index 000000000..594af4979 --- /dev/null +++ b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/encoder/avro/AvroRowEncoderFactory.java @@ -0,0 +1,38 @@ +/* + * 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.encoder.avro; + +import io.prestosql.plugin.kafka.encoder.EncoderColumnHandle; +import io.prestosql.plugin.kafka.encoder.RowEncoder; +import io.prestosql.plugin.kafka.encoder.RowEncoderFactory; +import io.prestosql.spi.connector.ConnectorSession; +import org.apache.avro.Schema; + +import java.util.List; +import java.util.Optional; + +import static com.google.common.base.Preconditions.checkArgument; +import static java.util.Objects.requireNonNull; + +public class AvroRowEncoderFactory + implements RowEncoderFactory +{ + @Override + public RowEncoder create(ConnectorSession session, Optional dataSchema, List columnHandles) + { + checkArgument(dataSchema.isPresent(), "dataSchema for Avro format is not present"); + Schema parsedSchema = new Schema.Parser().parse(requireNonNull(dataSchema.get(), "dataSchema is null")); + return new AvroRowEncoder(session, columnHandles, parsedSchema); + } +} diff --git a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/encoder/csv/CsvRowEncoder.java b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/encoder/csv/CsvRowEncoder.java new file mode 100644 index 000000000..b96b2e910 --- /dev/null +++ b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/encoder/csv/CsvRowEncoder.java @@ -0,0 +1,142 @@ +/* + * 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.encoder.csv; + +import au.com.bytecode.opencsv.CSVWriter; +import com.google.common.collect.ImmutableSet; +import io.prestosql.plugin.kafka.encoder.AbstractRowEncoder; +import io.prestosql.plugin.kafka.encoder.EncoderColumnHandle; +import io.prestosql.spi.connector.ConnectorSession; +import io.prestosql.spi.type.Type; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.OutputStreamWriter; +import java.io.UncheckedIOException; +import java.nio.charset.StandardCharsets; +import java.util.List; +import java.util.Set; + +import static com.google.common.base.Preconditions.checkArgument; +import static io.prestosql.spi.type.BigintType.BIGINT; +import static io.prestosql.spi.type.BooleanType.BOOLEAN; +import static io.prestosql.spi.type.DoubleType.DOUBLE; +import static io.prestosql.spi.type.IntegerType.INTEGER; +import static io.prestosql.spi.type.RealType.REAL; +import static io.prestosql.spi.type.SmallintType.SMALLINT; +import static io.prestosql.spi.type.TinyintType.TINYINT; +import static io.prestosql.spi.type.Varchars.isVarcharType; +import static java.lang.String.format; + +public class CsvRowEncoder + extends AbstractRowEncoder +{ + private static final Set SUPPORTED_PRIMITIVE_TYPES = ImmutableSet.of( + BOOLEAN, TINYINT, SMALLINT, INTEGER, BIGINT, DOUBLE, REAL); + + public static final String NAME = "csv"; + + private final String[] row; + + public CsvRowEncoder(ConnectorSession session, List columnHandles) + { + super(session, columnHandles); + for (EncoderColumnHandle columnHandle : this.columnHandles) { + checkArgument(columnHandle.getFormatHint() == null, "Unexpected format hint '%s' defined for column '%s'", columnHandle.getFormatHint(), columnHandle.getName()); + checkArgument(columnHandle.getDataFormat() == null, "Unexpected data format '%s' defined for column '%s'", columnHandle.getDataFormat(), columnHandle.getName()); + + checkArgument(isSupportedType(columnHandle.getType()), "Unsupported column type '%s' for column '%s'", columnHandle.getType(), columnHandle.getName()); + } + this.row = new String[this.columnHandles.size()]; + } + + private boolean isSupportedType(Type type) + { + return isVarcharType(type) || SUPPORTED_PRIMITIVE_TYPES.contains(type); + } + + @Override + protected void appendNullValue() + { + row[currentColumnIndex] = null; + } + + @Override + protected void appendLong(long value) + { + row[currentColumnIndex] = Long.toString(value); + } + + @Override + protected void appendInt(int value) + { + row[currentColumnIndex] = Integer.toString(value); + } + + @Override + protected void appendShort(short value) + { + row[currentColumnIndex] = Short.toString(value); + } + + @Override + protected void appendByte(byte value) + { + row[currentColumnIndex] = Byte.toString(value); + } + + @Override + protected void appendDouble(double value) + { + row[currentColumnIndex] = Double.toString(value); + } + + @Override + protected void appendFloat(float value) + { + row[currentColumnIndex] = Float.toString(value); + } + + @Override + protected void appendBoolean(boolean value) + { + row[currentColumnIndex] = Boolean.toString(value); + } + + @Override + protected void appendString(String value) + { + row[currentColumnIndex] = value; + } + + @Override + public byte[] toByteArray() + { + // make sure entire row has been updated with new values + checkArgument(currentColumnIndex == columnHandles.size(), format("Missing %d columns", columnHandles.size() - currentColumnIndex + 1)); + + try (ByteArrayOutputStream byteArrayOuts = new ByteArrayOutputStream(); + OutputStreamWriter outsWriter = new OutputStreamWriter(byteArrayOuts, StandardCharsets.UTF_8); + CSVWriter writer = new CSVWriter(outsWriter, ',', '"', "")) { + writer.writeNext(row); + writer.flush(); + + resetColumnIndex(); // reset currentColumnIndex to prepare for next row + return byteArrayOuts.toByteArray(); + } + catch (IOException e) { + throw new UncheckedIOException(e); + } + } +} diff --git a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/encoder/csv/CsvRowEncoderFactory.java b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/encoder/csv/CsvRowEncoderFactory.java new file mode 100644 index 000000000..327e26f6f --- /dev/null +++ b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/encoder/csv/CsvRowEncoderFactory.java @@ -0,0 +1,32 @@ +/* + * 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.encoder.csv; + +import io.prestosql.plugin.kafka.encoder.EncoderColumnHandle; +import io.prestosql.plugin.kafka.encoder.RowEncoder; +import io.prestosql.plugin.kafka.encoder.RowEncoderFactory; +import io.prestosql.spi.connector.ConnectorSession; + +import java.util.List; +import java.util.Optional; + +public class CsvRowEncoderFactory + implements RowEncoderFactory +{ + @Override + public RowEncoder create(ConnectorSession session, Optional dataSchema, List columnHandles) + { + return new CsvRowEncoder(session, columnHandles); + } +} diff --git a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/encoder/json/JsonRowEncoder.java b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/encoder/json/JsonRowEncoder.java new file mode 100644 index 000000000..b4ff756eb --- /dev/null +++ b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/encoder/json/JsonRowEncoder.java @@ -0,0 +1,150 @@ +/* + * 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.encoder.json; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.node.ObjectNode; +import com.google.common.collect.ImmutableSet; +import io.prestosql.plugin.kafka.encoder.AbstractRowEncoder; +import io.prestosql.plugin.kafka.encoder.EncoderColumnHandle; +import io.prestosql.spi.connector.ConnectorSession; +import io.prestosql.spi.type.Type; + +import java.nio.ByteBuffer; +import java.util.List; +import java.util.Set; + +import static com.google.common.base.Preconditions.checkArgument; +import static io.prestosql.spi.type.BigintType.BIGINT; +import static io.prestosql.spi.type.BooleanType.BOOLEAN; +import static io.prestosql.spi.type.DoubleType.DOUBLE; +import static io.prestosql.spi.type.IntegerType.INTEGER; +import static io.prestosql.spi.type.SmallintType.SMALLINT; +import static io.prestosql.spi.type.TinyintType.TINYINT; +import static io.prestosql.spi.type.Varchars.isVarcharType; +import static java.lang.String.format; +import static java.util.Objects.requireNonNull; + +public class JsonRowEncoder + extends AbstractRowEncoder +{ + private static final Set PRIMITIVE_SUPPORTED_TYPES = ImmutableSet.of( + BIGINT, INTEGER, SMALLINT, TINYINT, DOUBLE, BOOLEAN); + + public static final String NAME = "json"; + + private final ObjectMapper objectMapper; + private final ObjectNode node; + + JsonRowEncoder(ConnectorSession session, List columnHandles, ObjectMapper objectMapper) + { + super(session, columnHandles); + + for (EncoderColumnHandle columnHandle : this.columnHandles) { + checkArgument(isSupportedType(columnHandle.getType()), "Unsupported column type '%s' for column '%s'", columnHandle.getType(), columnHandle.getName()); + checkArgument(columnHandle.getFormatHint() == null, "Unexpected format hint '%s' defined for column '%s'", columnHandle.getFormatHint(), columnHandle.getName()); + checkArgument(columnHandle.getDataFormat() == null, "Unexpected data format '%s' defined for column '%s'", columnHandle.getDataFormat(), columnHandle.getName()); + } + + this.objectMapper = requireNonNull(objectMapper, "objectMapper is null"); + this.node = objectMapper.createObjectNode(); + } + + private boolean isSupportedType(Type type) + { + return isVarcharType(type) || + PRIMITIVE_SUPPORTED_TYPES.contains(type); + } + + private String currentColumnName() + { + return columnHandles.get(currentColumnIndex).getName(); + } + + @Override + protected void appendNullValue() + { + node.putNull(currentColumnName()); + } + + @Override + protected void appendLong(long value) + { + node.put(currentColumnName(), value); + } + + @Override + protected void appendInt(int value) + { + node.put(currentColumnName(), value); + } + + @Override + protected void appendShort(short value) + { + node.put(currentColumnName(), value); + } + + @Override + protected void appendByte(byte value) + { + node.put(currentColumnName(), value); + } + + @Override + protected void appendDouble(double value) + { + node.put(currentColumnName(), value); + } + + @Override + protected void appendFloat(float value) + { + node.put(currentColumnName(), value); + } + + @Override + protected void appendBoolean(boolean value) + { + node.put(currentColumnName(), value); + } + + @Override + protected void appendString(String value) + { + node.put(currentColumnName(), value); + } + + @Override + protected void appendByteBuffer(ByteBuffer value) + { + node.put(currentColumnName(), value.array()); + } + + @Override + public byte[] toByteArray() + { + // make sure entire row has been updated with new values + checkArgument(currentColumnIndex == columnHandles.size(), format("Missing %d columns", columnHandles.size() - currentColumnIndex + 1)); + + try { + resetColumnIndex(); // reset currentColumnIndex to prepare for next row + return objectMapper.writeValueAsBytes(node); + } + catch (JsonProcessingException e) { + throw new RuntimeException(e); + } + } +} diff --git a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/encoder/json/JsonRowEncoderFactory.java b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/encoder/json/JsonRowEncoderFactory.java new file mode 100644 index 000000000..0b1307e1b --- /dev/null +++ b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/encoder/json/JsonRowEncoderFactory.java @@ -0,0 +1,45 @@ +/* + * 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.encoder.json; + +import com.fasterxml.jackson.databind.ObjectMapper; +import io.prestosql.plugin.kafka.encoder.EncoderColumnHandle; +import io.prestosql.plugin.kafka.encoder.RowEncoder; +import io.prestosql.plugin.kafka.encoder.RowEncoderFactory; +import io.prestosql.spi.connector.ConnectorSession; + +import javax.inject.Inject; + +import java.util.List; +import java.util.Optional; + +import static java.util.Objects.requireNonNull; + +public class JsonRowEncoderFactory + implements RowEncoderFactory +{ + private final ObjectMapper objectMapper; + + @Inject + public JsonRowEncoderFactory(ObjectMapper objectMapper) + { + this.objectMapper = requireNonNull(objectMapper, "objectMapper is null"); + } + + @Override + public RowEncoder create(ConnectorSession session, Optional dataSchema, List columnHandles) + { + return new JsonRowEncoder(session, columnHandles, objectMapper); + } +} diff --git a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/encoder/raw/RawRowEncoder.java b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/encoder/raw/RawRowEncoder.java new file mode 100644 index 000000000..87f342062 --- /dev/null +++ b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/encoder/raw/RawRowEncoder.java @@ -0,0 +1,329 @@ +/* + * 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.encoder.raw; + +import com.google.common.base.Joiner; +import com.google.common.collect.ImmutableSet; +import io.prestosql.plugin.kafka.encoder.AbstractRowEncoder; +import io.prestosql.plugin.kafka.encoder.EncoderColumnHandle; +import io.prestosql.spi.connector.ConnectorSession; +import io.prestosql.spi.type.Type; + +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.util.Arrays; +import java.util.List; +import java.util.Locale; +import java.util.Optional; +import java.util.Set; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.collect.ImmutableList.toImmutableList; +import static io.prestosql.spi.type.BigintType.BIGINT; +import static io.prestosql.spi.type.BooleanType.BOOLEAN; +import static io.prestosql.spi.type.DoubleType.DOUBLE; +import static io.prestosql.spi.type.IntegerType.INTEGER; +import static io.prestosql.spi.type.RealType.REAL; +import static io.prestosql.spi.type.SmallintType.SMALLINT; +import static io.prestosql.spi.type.TinyintType.TINYINT; +import static io.prestosql.spi.type.Varchars.isVarcharType; +import static java.lang.Integer.parseInt; +import static java.lang.String.format; + +public class RawRowEncoder + extends AbstractRowEncoder +{ + private enum FieldType + { + BYTE(Byte.SIZE), + SHORT(Short.SIZE), + INT(Integer.SIZE), + LONG(Long.SIZE), + FLOAT(Float.SIZE), + DOUBLE(Double.SIZE); + + private final int size; + + FieldType(int bitSize) + { + this.size = bitSize / 8; + } + + public int getSize() + { + return size; + } + } + + private static final Pattern MAPPING_PATTERN = Pattern.compile("(\\d+)(?::(\\d+))?"); + private static final Set SUPPORTED_PRIMITIVE_TYPES = ImmutableSet.of( + BIGINT, INTEGER, SMALLINT, TINYINT, DOUBLE, REAL, BOOLEAN); + + public static final String NAME = "raw"; + + private final List columnMappings; + private final ByteBuffer buffer; + + public RawRowEncoder(ConnectorSession session, List columnHandles) + { + super(session, columnHandles); + + for (EncoderColumnHandle handle : this.columnHandles) { + checkArgument(isSupportedType(handle.getType()), "Unsupported column type '%s' for column '%s'", handle.getType().getDisplayName(), handle.getName()); + checkArgument(handle.getFormatHint() == null, "Unexpected format hint '%s' defined for column '%s'", handle.getFormatHint(), handle.getName()); + } + + // parse column mappings from column handles + this.columnMappings = this.columnHandles.stream().map(ColumnMapping::new).collect(toImmutableList()); + + for (ColumnMapping mapping : this.columnMappings) { + if (mapping.getLength() != mapping.getFieldType().getSize() && !isVarcharType(mapping.getType())) { + throw new IndexOutOfBoundsException(format( + "Mapping length '%s' is not equal to expected length '%s' for column '%s'", + mapping.getLength(), + mapping.getFieldType().getSize(), + mapping.getName())); + } + } + + // check that column mappings don't overlap and that there are no gaps + int position = 0; + for (ColumnMapping mapping : this.columnMappings) { + checkArgument(mapping.getStart() == position, format( + "Start mapping '%s' for column '%s' does not equal expected mapping '%s'", + mapping.getStart(), + mapping.getName(), + position)); + checkArgument(mapping.getEnd() > mapping.getStart(), format( + "End mapping '%s' for column '%s' is less than or equal to start '%s'", + mapping.getEnd(), + mapping.getName(), + mapping.getStart())); + position += mapping.getLength(); + } + + this.buffer = ByteBuffer.allocate(position); + } + + private static class ColumnMapping + { + private final String name; + private final Type type; + private final FieldType fieldType; + private final int start; + private final int end; + + public ColumnMapping(EncoderColumnHandle columnHandle) + { + this.name = columnHandle.getName(); + this.type = columnHandle.getType(); + + this.fieldType = parseFieldType(columnHandle.getDataFormat(), this.name); + checkFieldType(this.name, this.type, this.fieldType); + + Optional mapping = Optional.ofNullable(columnHandle.getMapping()); + if (mapping.isPresent()) { + Matcher mappingMatcher = MAPPING_PATTERN.matcher(mapping.get()); + if (!mappingMatcher.matches()) { + throw new IllegalArgumentException(format("Invalid mapping for column '%s'", this.name)); + } + + if (mappingMatcher.group(2) != null) { + this.start = parseOffset(mappingMatcher.group(1), "start", this.name); + this.end = parseOffset(mappingMatcher.group(2), "end", this.name); + } + else { + this.start = parseOffset(mappingMatcher.group(1), "start", this.name); + this.end = this.start + this.fieldType.getSize(); + } + } + else { + throw new IllegalArgumentException(format("No mapping defined for column '%s'", this.name)); + } + } + + private static int parseOffset(String group, String offsetName, String columnName) + { + try { + return parseInt(group); + } + catch (NumberFormatException e) { + throw new IllegalArgumentException(format("Unable to parse '%s' offset for column '%s'", offsetName, columnName), e); + } + } + + private static FieldType parseFieldType(String dataFormat, String columnName) + { + try { + if (!dataFormat.equals("")) { + return FieldType.valueOf(dataFormat.toUpperCase(Locale.ENGLISH)); + } + return FieldType.BYTE; + } + catch (IllegalArgumentException e) { + throw new IllegalArgumentException(format("Invalid dataFormat '%s' for column '%s'", dataFormat, columnName)); + } + } + + private static void checkFieldType(String columnName, Type columnType, FieldType fieldType) + { + if (columnType == BIGINT) { + checkFieldTypeOneOf(fieldType, columnName, columnType, FieldType.BYTE, FieldType.SHORT, FieldType.INT, FieldType.LONG); + } + else if (columnType == INTEGER) { + checkFieldTypeOneOf(fieldType, columnName, columnType, FieldType.BYTE, FieldType.SHORT, FieldType.INT); + } + else if (columnType == SMALLINT) { + checkFieldTypeOneOf(fieldType, columnName, columnType, FieldType.BYTE, FieldType.SHORT); + } + else if (columnType == TINYINT) { + checkFieldTypeOneOf(fieldType, columnName, columnType, FieldType.BYTE); + } + else if (columnType == BOOLEAN) { + checkFieldTypeOneOf(fieldType, columnName, columnType, FieldType.BYTE, FieldType.SHORT, FieldType.INT, FieldType.LONG); + } + else if (columnType == DOUBLE) { + checkFieldTypeOneOf(fieldType, columnName, columnType, FieldType.DOUBLE, FieldType.FLOAT); + } + else if (isVarcharType(columnType)) { + checkFieldTypeOneOf(fieldType, columnName, columnType, FieldType.BYTE); + } + } + + private static void checkFieldTypeOneOf(FieldType declaredFieldType, String columnName, Type columnType, FieldType... allowedFieldTypes) + { + checkArgument(Arrays.asList(allowedFieldTypes).contains(declaredFieldType), + format("Wrong dataformat '%s' specified for column '%s'; %s type implies use of %s", + declaredFieldType.name(), + columnName, + columnType, + Joiner.on("/").join(allowedFieldTypes))); + } + + public String getName() + { + return name; + } + + public Type getType() + { + return type; + } + + public int getStart() + { + return start; + } + + public int getEnd() + { + return end; + } + + public FieldType getFieldType() + { + return fieldType; + } + + public int getLength() + { + return end - start; + } + } + + private static boolean isSupportedType(Type type) + { + return isVarcharType(type) || SUPPORTED_PRIMITIVE_TYPES.contains(type); + } + + @Override + protected void appendLong(long value) + { + buffer.putLong(value); + } + + @Override + protected void appendInt(int value) + { + buffer.putInt(value); + } + + @Override + protected void appendShort(short value) + { + buffer.putShort(value); + } + + @Override + protected void appendByte(byte value) + { + buffer.put(value); + } + + @Override + protected void appendDouble(double value) + { + buffer.putDouble(value); + } + + @Override + protected void appendFloat(float value) + { + buffer.putFloat(value); + } + + @Override + protected void appendBoolean(boolean value) + { + buffer.put((byte) (value ? 1 : 0)); + } + + @Override + protected void appendString(String value) + { + byte[] valueBytes = value.getBytes(StandardCharsets.UTF_8); + checkArgument(valueBytes.length == columnMappings.get(currentColumnIndex).getLength(), format( + "length '%s' of message '%s' for column '%s' does not equal expected length '%s'", + valueBytes.length, + value, + columnHandles.get(currentColumnIndex).getName(), + columnMappings.get(currentColumnIndex).getLength())); + buffer.put(valueBytes, 0, valueBytes.length); + } + + @Override + protected void appendByteBuffer(ByteBuffer value) + { + byte[] valueBytes = value.array(); + checkArgument(valueBytes.length == columnMappings.get(currentColumnIndex).getLength(), format( + "length '%s' of message for column '%s' does not equal expected length '%s'", + valueBytes.length, + columnHandles.get(currentColumnIndex).getName(), + columnMappings.get(currentColumnIndex).getLength())); + buffer.put(valueBytes, 0, valueBytes.length); + } + + @Override + public byte[] toByteArray() + { + // make sure entire row has been updated with new values + checkArgument(currentColumnIndex == columnHandles.size(), format("Missing %d columns", columnHandles.size() - currentColumnIndex + 1)); + + resetColumnIndex(); // reset currentColumnIndex to prepare for next row + buffer.clear(); // set buffer position back to 0 to prepare for next row, this method does not affect the backing byte array + return buffer.array(); + } +} diff --git a/presto-kafka/src/main/java/io/prestosql/plugin/kafka/encoder/raw/RawRowEncoderFactory.java b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/encoder/raw/RawRowEncoderFactory.java new file mode 100644 index 000000000..fa2628c60 --- /dev/null +++ b/presto-kafka/src/main/java/io/prestosql/plugin/kafka/encoder/raw/RawRowEncoderFactory.java @@ -0,0 +1,32 @@ +/* + * 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.encoder.raw; + +import io.prestosql.plugin.kafka.encoder.EncoderColumnHandle; +import io.prestosql.plugin.kafka.encoder.RowEncoder; +import io.prestosql.plugin.kafka.encoder.RowEncoderFactory; +import io.prestosql.spi.connector.ConnectorSession; + +import java.util.List; +import java.util.Optional; + +public class RawRowEncoderFactory + implements RowEncoderFactory +{ + @Override + public RowEncoder create(ConnectorSession session, Optional dataSchema, List columnHandles) + { + return new RawRowEncoder(session, columnHandles); + } +} diff --git a/presto-kafka/src/test/java/io/prestosql/plugin/kafka/ConfigurationAwareModules.java b/presto-kafka/src/test/java/io/prestosql/plugin/kafka/ConfigurationAwareModules.java new file mode 100644 index 000000000..e4c4a3d49 --- /dev/null +++ b/presto-kafka/src/test/java/io/prestosql/plugin/kafka/ConfigurationAwareModules.java @@ -0,0 +1,46 @@ +/* + * 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.inject.Binder; +import com.google.inject.Module; +import io.airlift.configuration.AbstractConfigurationAwareModule; + +import static com.google.common.base.Preconditions.checkArgument; +import static java.util.Objects.requireNonNull; + +public class ConfigurationAwareModules +{ + private ConfigurationAwareModules() {} + + public static Module combine(Module... modules) + { + checkArgument(modules.length > 0, "No modules given"); + if (modules.length == 1) { + return modules[0]; + } + return new AbstractConfigurationAwareModule() + { + @Override + protected void setup(Binder binder) + { + for (Module module : modules) { + requireNonNull(module, "module is null"); + install(module); + } + } + }; + } +} diff --git a/presto-kafka/src/test/java/io/prestosql/plugin/kafka/TestKafkaConfig.java b/presto-kafka/src/test/java/io/prestosql/plugin/kafka/TestKafkaConfig.java index cae9beded..ec4042ca0 100644 --- a/presto-kafka/src/test/java/io/prestosql/plugin/kafka/TestKafkaConfig.java +++ b/presto-kafka/src/test/java/io/prestosql/plugin/kafka/TestKafkaConfig.java @@ -30,13 +30,13 @@ public class TestKafkaConfig { assertRecordedDefaults(recordDefaults(KafkaConfig.class) .setNodes("") - .setKafkaConnectTimeout("10s") .setKafkaBufferSize("64kB") .setDefaultSchema("default") .setTableNames("") .setTableDescriptionDir(new File("etc/kafka/")) .setHideInternalColumns(true) - .setMessagesPerSplit(100_000)); + .setMessagesPerSplit(100_000) + .setTimestampUpperBoundPushDownEnabled(false)); } @Test @@ -47,10 +47,10 @@ public class TestKafkaConfig .put("kafka.table-names", "table1, table2, table3") .put("kafka.default-schema", "kafka") .put("kafka.nodes", "localhost:12345,localhost:23456") - .put("kafka.connect-timeout", "1h") .put("kafka.buffer-size", "1MB") .put("kafka.hide-internal-columns", "false") .put("kafka.messages-per-split", "1") + .put("kafka.timestamp-upper-bound-force-push-down-enabled", "true") .build(); KafkaConfig expected = new KafkaConfig() @@ -58,10 +58,10 @@ public class TestKafkaConfig .setTableNames("table1, table2, table3") .setDefaultSchema("kafka") .setNodes("localhost:12345, localhost:23456") - .setKafkaConnectTimeout("1h") .setKafkaBufferSize("1MB") .setHideInternalColumns(false) - .setMessagesPerSplit(1); + .setMessagesPerSplit(1) + .setTimestampUpperBoundPushDownEnabled(true); assertFullMapping(properties, expected); } diff --git a/presto-kafka/src/test/java/io/prestosql/plugin/kafka/encoder/json/TestJsonEncoder.java b/presto-kafka/src/test/java/io/prestosql/plugin/kafka/encoder/json/TestJsonEncoder.java new file mode 100644 index 000000000..a59fcf6fa --- /dev/null +++ b/presto-kafka/src/test/java/io/prestosql/plugin/kafka/encoder/json/TestJsonEncoder.java @@ -0,0 +1,71 @@ +/* + * 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.encoder.json; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import io.prestosql.plugin.kafka.KafkaColumnHandle; +import io.prestosql.spi.type.Type; +import io.prestosql.testing.TestingConnectorSession; +import org.assertj.core.api.ThrowableAssert; +import org.testng.annotations.Test; + +import java.util.Optional; + +import static io.prestosql.spi.type.BigintType.BIGINT; +import static io.prestosql.spi.type.BooleanType.BOOLEAN; +import static io.prestosql.spi.type.DecimalType.createDecimalType; +import static io.prestosql.spi.type.DoubleType.DOUBLE; +import static io.prestosql.spi.type.IntegerType.INTEGER; +import static io.prestosql.spi.type.RealType.REAL; +import static io.prestosql.spi.type.SmallintType.SMALLINT; +import static io.prestosql.spi.type.TinyintType.TINYINT; +import static io.prestosql.spi.type.VarbinaryType.VARBINARY; +import static io.prestosql.spi.type.VarcharType.createUnboundedVarcharType; +import static io.prestosql.spi.type.VarcharType.createVarcharType; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +public class TestJsonEncoder +{ + private static final JsonRowEncoderFactory ENCODER_FACTORY = new JsonRowEncoderFactory(new ObjectMapper()); + + private void assertUnsupportedColumnTypeException(ThrowableAssert.ThrowingCallable callable) + { + assertThatThrownBy(callable) + .isInstanceOf(RuntimeException.class) + .hasMessageMatching("Unsupported column type .* for column .*"); + } + + private void singleColumnEncoder(Type type) + { + ENCODER_FACTORY.create(TestingConnectorSession.SESSION, Optional.empty(), ImmutableList.of(new KafkaColumnHandle("default", type, "default", null, null, false, false, false))); + } + + @Test + public void testColumnValidation() + { + singleColumnEncoder(BIGINT); + singleColumnEncoder(INTEGER); + singleColumnEncoder(SMALLINT); + singleColumnEncoder(TINYINT); + singleColumnEncoder(DOUBLE); + singleColumnEncoder(BOOLEAN); + singleColumnEncoder(createVarcharType(20)); + singleColumnEncoder(createUnboundedVarcharType()); + + assertUnsupportedColumnTypeException(() -> singleColumnEncoder(REAL)); + assertUnsupportedColumnTypeException(() -> singleColumnEncoder(createDecimalType(10, 4))); + assertUnsupportedColumnTypeException(() -> singleColumnEncoder(VARBINARY)); + } +} diff --git a/presto-kafka/src/test/java/io/prestosql/plugin/kafka/encoder/raw/TestRawEncoderMapping.java b/presto-kafka/src/test/java/io/prestosql/plugin/kafka/encoder/raw/TestRawEncoderMapping.java new file mode 100644 index 000000000..7a9aa5c13 --- /dev/null +++ b/presto-kafka/src/test/java/io/prestosql/plugin/kafka/encoder/raw/TestRawEncoderMapping.java @@ -0,0 +1,77 @@ +/* + * 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.encoder.raw; + +import io.airlift.slice.Slices; +import io.prestosql.plugin.kafka.KafkaColumnHandle; +import io.prestosql.plugin.kafka.encoder.EncoderColumnHandle; +import io.prestosql.plugin.kafka.encoder.RowEncoder; +import io.prestosql.spi.block.Block; +import io.prestosql.spi.block.LongArrayBlockBuilder; +import io.prestosql.spi.block.VariableWidthBlockBuilder; +import io.prestosql.testing.TestingConnectorSession; +import org.testcontainers.shaded.com.google.common.collect.ImmutableList; +import org.testng.annotations.Test; + +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.util.Optional; + +import static io.prestosql.spi.type.BigintType.BIGINT; +import static io.prestosql.spi.type.VarcharType.createUnboundedVarcharType; +import static io.prestosql.spi.type.VarcharType.createVarcharType; +import static org.testng.Assert.assertEquals; + +public class TestRawEncoderMapping +{ + private static final RawRowEncoderFactory ENCODER_FACTORY = new RawRowEncoderFactory(); + + @Test + public void testMapping() + { + EncoderColumnHandle col1 = new KafkaColumnHandle("test1", BIGINT, "0", "LONG", null, false, false, false); + EncoderColumnHandle col2 = new KafkaColumnHandle("test2", createUnboundedVarcharType(), "8:14", "BYTE", null, false, false, false); + EncoderColumnHandle col3 = new KafkaColumnHandle("test3", BIGINT, "14", "LONG", null, false, false, false); + EncoderColumnHandle col4 = new KafkaColumnHandle("test4", createUnboundedVarcharType(), "22:28", "BYTE", null, false, false, false); + EncoderColumnHandle col5 = new KafkaColumnHandle("test5", BIGINT, "28", "LONG", null, false, false, false); + EncoderColumnHandle col6 = new KafkaColumnHandle("test6", createVarcharType(6), "36:42", "BYTE", null, false, false, false); + EncoderColumnHandle col7 = new KafkaColumnHandle("test7", createVarcharType(6), "42:48", "BYTE", null, false, false, false); + + RowEncoder rowEncoder = ENCODER_FACTORY.create(TestingConnectorSession.SESSION, Optional.empty(), ImmutableList.of(col1, col2, col3, col4, col5, col6, col7)); + + ByteBuffer buf = ByteBuffer.allocate(48); + buf.putLong(123456789); // 0-8 + buf.put("abcdef".getBytes(StandardCharsets.UTF_8)); // 8-14 + buf.putLong(123456789); // 14-22 + buf.put("abcdef".getBytes(StandardCharsets.UTF_8)); // 22-28 + buf.putLong(123456789); // 28-36 + buf.put("abcdef".getBytes(StandardCharsets.UTF_8)); // 36-42 + buf.put("abcdef".getBytes(StandardCharsets.UTF_8)); // 42-48 + + Block longArrayBlock = new LongArrayBlockBuilder(null, 1).writeLong(123456789).closeEntry().build(); + Block varArrayBlock = new VariableWidthBlockBuilder(null, 1, 6) + .writeBytes(Slices.wrappedBuffer("abcdef".getBytes(StandardCharsets.UTF_8)), 0, 6) + .closeEntry().build(); + + rowEncoder.appendColumnValue(longArrayBlock, 0); + rowEncoder.appendColumnValue(varArrayBlock, 0); + rowEncoder.appendColumnValue(longArrayBlock, 0); + rowEncoder.appendColumnValue(varArrayBlock, 0); + rowEncoder.appendColumnValue(longArrayBlock, 0); + rowEncoder.appendColumnValue(varArrayBlock, 0); + rowEncoder.appendColumnValue(varArrayBlock, 0); + + assertEquals(buf.array(), rowEncoder.toByteArray()); + } +} diff --git a/presto-kafka/src/test/resources/write_test/all_datatypes_avro.json b/presto-kafka/src/test/resources/write_test/all_datatypes_avro.json new file mode 100644 index 000000000..d440c46ec --- /dev/null +++ b/presto-kafka/src/test/resources/write_test/all_datatypes_avro.json @@ -0,0 +1,42 @@ +{ + "tableName": "all_datatypes_avro", + "schemaName": "write_test", + "topicName": "all_datatypes_avro", + "key": { + "dataFormat": "avro", + "dataSchema": "/write_test/all_datatypes_avro_key_schema.avsc", + "fields": [ + { + "name": "kafka_key", + "type": "BIGINT", + "mapping": "kafka_key" + } + ] + }, + "message": { + "dataFormat": "avro", + "dataSchema": "/write_test/all_datatypes_avro_message_schema.avsc", + "fields": [ + { + "name": "f_bigint", + "type": "BIGINT", + "mapping": "f_bigint" + }, + { + "name": "f_double", + "type": "DOUBLE", + "mapping": "f_double" + }, + { + "name": "f_boolean", + "type": "BOOLEAN", + "mapping": "f_boolean" + }, + { + "name": "f_varchar", + "type": "VARCHAR", + "mapping": "f_varchar" + } + ] + } +} diff --git a/presto-kafka/src/test/resources/write_test/all_datatypes_avro_key_schema.avsc b/presto-kafka/src/test/resources/write_test/all_datatypes_avro_key_schema.avsc new file mode 100644 index 000000000..809158f6d --- /dev/null +++ b/presto-kafka/src/test/resources/write_test/all_datatypes_avro_key_schema.avsc @@ -0,0 +1,14 @@ +{ + "type" : "record", + "name" : "all_datatypes_avro_key", + "namespace" : "io.prestosql.plugin.kafka", + "fields" : + [ + { + "name":"kafka_key", + "type":["null", "long"], + "default": null + } + ], + "doc:" : "A basic avro schema for kafka write tests" +} diff --git a/presto-kafka/src/test/resources/write_test/all_datatypes_avro_message_schema.avsc b/presto-kafka/src/test/resources/write_test/all_datatypes_avro_message_schema.avsc new file mode 100644 index 000000000..d682a2159 --- /dev/null +++ b/presto-kafka/src/test/resources/write_test/all_datatypes_avro_message_schema.avsc @@ -0,0 +1,29 @@ +{ + "type" : "record", + "name" : "all_datatypes_avro_message", + "namespace" : "io.prestosql.plugin.kafka", + "fields" : + [ + { + "name":"f_bigint", + "type":["null", "long"], + "default": null + }, + { + "name": "f_double", + "type":["null", "double"], + "default": null + }, + { + "name":"f_boolean", + "type":["null", "boolean"], + "default": null + }, + { + "name":"f_varchar", + "type":["null", "string"], + "default": null + } + ], + "doc:" : "A basic avro schema for kafka write tests" +} diff --git a/presto-kafka/src/test/resources/write_test/all_datatypes_csv.json b/presto-kafka/src/test/resources/write_test/all_datatypes_csv.json new file mode 100644 index 000000000..d36538a1a --- /dev/null +++ b/presto-kafka/src/test/resources/write_test/all_datatypes_csv.json @@ -0,0 +1,55 @@ +{ + "tableName": "all_datatypes_csv", + "schemaName": "write_test", + "topicName": "all_datatypes_csv", + "key": { + "dataFormat": "csv", + "fields": [ + { + "name": "kafka_key", + "type": "BIGINT", + "mapping": "0" + } + ] + }, + "message": { + "dataFormat": "csv", + "fields": [ + { + "name": "f_bigint", + "type": "BIGINT", + "mapping": "0" + }, + { + "name": "f_int", + "type": "INTEGER", + "mapping": "1" + }, + { + "name": "f_smallint", + "type": "SMALLINT", + "mapping": "2" + }, + { + "name": "f_tinyint", + "type": "TINYINT", + "mapping": "3" + }, + { + "name": "f_double", + "type": "DOUBLE", + "mapping": "4" + }, + { + "name": "f_boolean", + "type": "BOOLEAN", + "mapping": "5" + }, + { + "name": "f_varchar", + "type": "VARCHAR", + "mapping": "6" + } + ] + } +} diff --git a/presto-kafka/src/test/resources/write_test/all_datatypes_json.json b/presto-kafka/src/test/resources/write_test/all_datatypes_json.json new file mode 100644 index 000000000..1e8ff6903 --- /dev/null +++ b/presto-kafka/src/test/resources/write_test/all_datatypes_json.json @@ -0,0 +1,55 @@ +{ + "tableName": "all_datatypes_json", + "schemaName": "write_test", + "topicName": "all_datatypes_json", + "key": { + "dataFormat": "json", + "fields": [ + { + "name": "kafka_key", + "type": "BIGINT", + "mapping": "kafka_key" + } + ] + }, + "message": { + "dataFormat": "json", + "fields": [ + { + "name": "f_bigint", + "type": "BIGINT", + "mapping": "f_bigint" + }, + { + "name": "f_int", + "type": "INTEGER", + "mapping": "f_int" + }, + { + "name": "f_smallint", + "type": "SMALLINT", + "mapping": "f_smallint" + }, + { + "name": "f_tinyint", + "type": "TINYINT", + "mapping": "f_tinyint" + }, + { + "name": "f_double", + "type": "DOUBLE", + "mapping": "f_double" + }, + { + "name": "f_boolean", + "type": "BOOLEAN", + "mapping": "f_boolean" + }, + { + "name": "f_varchar", + "type": "VARCHAR", + "mapping": "f_varchar" + } + ] + } +} diff --git a/presto-kafka/src/test/resources/write_test/all_datatypes_raw.json b/presto-kafka/src/test/resources/write_test/all_datatypes_raw.json new file mode 100644 index 000000000..9693944b1 --- /dev/null +++ b/presto-kafka/src/test/resources/write_test/all_datatypes_raw.json @@ -0,0 +1,63 @@ +{ + "tableName": "raw", + "schemaName": "write_test", + "topicName": "raw", + "key": { + "dataFormat": "raw", + "fields": [ + { + "name": "kafka_key", + "dataFormat": "LONG", + "type": "BIGINT", + "mapping": "0" + } + ] + }, + "message": { + "dataFormat": "raw", + "fields": [ + { + "name": "f_varchar", + "dataFormat": "BYTE", + "type": "VARCHAR", + "mapping": "0:4" + }, + { + "name": "f_bigint", + "dataFormat": "LONG", + "type": "BIGINT", + "mapping": "4" + }, + { + "name": "f_int", + "dataFormat": "INT", + "type": "INTEGER", + "mapping": "12" + }, + { + "name": "f_smallint", + "dataFormat": "SHORT", + "type": "SMALLINT", + "mapping": "16" + }, + { + "name": "f_tinyint", + "dataFormat": "BYTE", + "type": "TINYINT", + "mapping": "18" + }, + { + "name": "f_double", + "dataFormat": "DOUBLE", + "type": "DOUBLE", + "mapping": "19" + }, + { + "name": "f_boolean", + "dataFormat": "BYTE", + "type": "BOOLEAN", + "mapping": "27" + } + ] + } +} diff --git a/presto-main/src/main/java/io/prestosql/event/QueryMonitor.java b/presto-main/src/main/java/io/prestosql/event/QueryMonitor.java index e0da88839..4b8dbcc47 100644 --- a/presto-main/src/main/java/io/prestosql/event/QueryMonitor.java +++ b/presto-main/src/main/java/io/prestosql/event/QueryMonitor.java @@ -542,7 +542,7 @@ public class QueryMonitor long spilledWriteTimeMillisPerNode, long spilledReadTimeMillisPerNode) { - log.info("TIMELINE: Query %s :: Transaction:[%s] :: elapsed %sms :: syntaxAnalysisTime %sms :: planning %sms :: logicalPlanningMillis %sms :: physicalPlanningMillis %sms :: distributionPlanTime %sms :: waiting %sms :: scheduling %sms :: executionInitializationTime %sms :: running %sms :: spilledWriteTimeMillisPerNode %sms :: spilledReadTimeMillisPerNode %sms :: finishing %sms :: begin %s :: end %s", + log.debug("TIMELINE: Query %s :: Transaction:[%s] :: elapsed %sms :: syntaxAnalysisTime %sms :: planning %sms :: logicalPlanningMillis %sms :: physicalPlanningMillis %sms :: distributionPlanTime %sms :: waiting %sms :: scheduling %sms :: executionInitializationTime %sms :: running %sms :: spilledWriteTimeMillisPerNode %sms :: spilledReadTimeMillisPerNode %sms :: finishing %sms :: begin %s :: end %s", queryId, transactionId, elapsedMillis, diff --git a/presto-plugin-toolkit/pom.xml b/presto-plugin-toolkit/pom.xml index 88b8b6d82..49b088070 100644 --- a/presto-plugin-toolkit/pom.xml +++ b/presto-plugin-toolkit/pom.xml @@ -91,6 +91,11 @@ joda-time + + io.airlift + slice + + com.fasterxml.jackson.core jackson-annotations diff --git a/presto-plugin-toolkit/src/main/java/io/prestosql/plugin/base/classloader/ClassLoaderSafeConnectorAccessControl.java b/presto-plugin-toolkit/src/main/java/io/prestosql/plugin/base/classloader/ClassLoaderSafeConnectorAccessControl.java new file mode 100644 index 000000000..2d091cac9 --- /dev/null +++ b/presto-plugin-toolkit/src/main/java/io/prestosql/plugin/base/classloader/ClassLoaderSafeConnectorAccessControl.java @@ -0,0 +1,315 @@ +/* + * 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.base.classloader; +// +//import io.prestosql.spi.classloader.ThreadContextClassLoader; +//import io.prestosql.spi.connector.ColumnMetadata; +//import io.prestosql.spi.connector.ConnectorAccessControl; +//import io.prestosql.spi.connector.ConnectorSecurityContext; +//import io.prestosql.spi.connector.SchemaTableName; +//import io.prestosql.spi.security.PrestoPrincipal; +//import io.prestosql.spi.security.Privilege; +// +//import javax.inject.Inject; +// +//import java.util.List; +//import java.util.Optional; +//import java.util.Set; +// +// +//public class ClassLoaderSafeConnectorAccessControl +// implements ConnectorAccessControl +//{ +// private final ConnectorAccessControl delegate; +// private final ClassLoader classLoader; +// +// @Inject +// public ClassLoaderSafeConnectorAccessControl(@ForClassLoaderSafe ConnectorAccessControl delegate, ClassLoader classLoader) +// { +// this.delegate = requireNonNull(delegate, "delegate is null"); +// this.classLoader = requireNonNull(classLoader, "classLoader is null"); +// } +// +// @Override +// public void checkCanCreateSchema(ConnectorSecurityContext context, String schemaName) +// { +// try (ThreadContextClassLoader ignored = new ThreadContextClassLoader(classLoader)) { +// delegate.checkCanCreateSchema(context, schemaName); +// } +// } +// +// @Override +// public void checkCanDropSchema(ConnectorSecurityContext context, String schemaName) +// { +// try (ThreadContextClassLoader ignored = new ThreadContextClassLoader(classLoader)) { +// delegate.checkCanDropSchema(context, schemaName); +// } +// } +// +// @Override +// public void checkCanRenameSchema(ConnectorSecurityContext context, String schemaName, String newSchemaName) +// { +// try (ThreadContextClassLoader ignored = new ThreadContextClassLoader(classLoader)) { +// delegate.checkCanRenameSchema(context, schemaName, newSchemaName); +// } +// } +// +// @Override +// public void checkCanShowSchemas(ConnectorSecurityContext context) +// { +// try (ThreadContextClassLoader ignored = new ThreadContextClassLoader(classLoader)) { +// delegate.checkCanShowSchemas(context); +// } +// } +// +// @Override +// public Set filterSchemas(ConnectorSecurityContext context, Set schemaNames) +// { +// try (ThreadContextClassLoader ignored = new ThreadContextClassLoader(classLoader)) { +// return delegate.filterSchemas(context, schemaNames); +// } +// } +// +// @Override +// public void checkCanCreateTable(ConnectorSecurityContext context, SchemaTableName tableName) +// { +// try (ThreadContextClassLoader ignored = new ThreadContextClassLoader(classLoader)) { +// delegate.checkCanCreateTable(context, tableName); +// } +// } +// +// @Override +// public void checkCanDropTable(ConnectorSecurityContext context, SchemaTableName tableName) +// { +// try (ThreadContextClassLoader ignored = new ThreadContextClassLoader(classLoader)) { +// delegate.checkCanDropTable(context, tableName); +// } +// } +// +// @Override +// public void checkCanRenameTable(ConnectorSecurityContext context, SchemaTableName tableName, SchemaTableName newTableName) +// { +// try (ThreadContextClassLoader ignored = new ThreadContextClassLoader(classLoader)) { +// delegate.checkCanRenameTable(context, tableName, newTableName); +// } +// } +// +// @Override +// public void checkCanSetTableComment(ConnectorSecurityContext context, SchemaTableName tableName) +// { +// try (ThreadContextClassLoader ignored = new ThreadContextClassLoader(classLoader)) { +// delegate.checkCanSetTableComment(context, tableName); +// } +// } +// +// @Override +// public void checkCanShowTablesMetadata(ConnectorSecurityContext context, String schemaName) +// { +// try (ThreadContextClassLoader ignored = new ThreadContextClassLoader(classLoader)) { +// delegate.checkCanShowTablesMetadata(context, schemaName); +// } +// } +// +// @Override +// public Set filterTables(ConnectorSecurityContext context, Set tableNames) +// { +// try (ThreadContextClassLoader ignored = new ThreadContextClassLoader(classLoader)) { +// return delegate.filterTables(context, tableNames); +// } +// } +// +// @Override +// public void checkCanShowColumnsMetadata(ConnectorSecurityContext context, SchemaTableName tableName) +// { +// try (ThreadContextClassLoader ignored = new ThreadContextClassLoader(classLoader)) { +// delegate.checkCanShowColumnsMetadata(context, tableName); +// } +// } +// +// @Override +// public List filterColumns(ConnectorSecurityContext context, SchemaTableName tableName, List columns) +// { +// try (ThreadContextClassLoader ignored = new ThreadContextClassLoader(classLoader)) { +// return delegate.filterColumns(context, tableName, columns); +// } +// } +// +// @Override +// public void checkCanAddColumn(ConnectorSecurityContext context, SchemaTableName tableName) +// { +// try (ThreadContextClassLoader ignored = new ThreadContextClassLoader(classLoader)) { +// delegate.checkCanAddColumn(context, tableName); +// } +// } +// +// @Override +// public void checkCanDropColumn(ConnectorSecurityContext context, SchemaTableName tableName) +// { +// try (ThreadContextClassLoader ignored = new ThreadContextClassLoader(classLoader)) { +// delegate.checkCanDropColumn(context, tableName); +// } +// } +// +// @Override +// public void checkCanRenameColumn(ConnectorSecurityContext context, SchemaTableName tableName) +// { +// try (ThreadContextClassLoader ignored = new ThreadContextClassLoader(classLoader)) { +// delegate.checkCanRenameColumn(context, tableName); +// } +// } +// +// @Override +// public void checkCanSelectFromColumns(ConnectorSecurityContext context, SchemaTableName tableName, Set columnNames) +// { +// try (ThreadContextClassLoader ignored = new ThreadContextClassLoader(classLoader)) { +// delegate.checkCanSelectFromColumns(context, tableName, columnNames); +// } +// } +// +// @Override +// public void checkCanInsertIntoTable(ConnectorSecurityContext context, SchemaTableName tableName) +// { +// try (ThreadContextClassLoader ignored = new ThreadContextClassLoader(classLoader)) { +// delegate.checkCanInsertIntoTable(context, tableName); +// } +// } +// +// @Override +// public void checkCanDeleteFromTable(ConnectorSecurityContext context, SchemaTableName tableName) +// { +// try (ThreadContextClassLoader ignored = new ThreadContextClassLoader(classLoader)) { +// delegate.checkCanDeleteFromTable(context, tableName); +// } +// } +// +// @Override +// public void checkCanCreateView(ConnectorSecurityContext context, SchemaTableName viewName) +// { +// try (ThreadContextClassLoader ignored = new ThreadContextClassLoader(classLoader)) { +// delegate.checkCanCreateView(context, viewName); +// } +// } +// +// @Override +// public void checkCanRenameView(ConnectorSecurityContext context, SchemaTableName viewName, SchemaTableName newViewName) +// { +// try (ThreadContextClassLoader ignored = new ThreadContextClassLoader(classLoader)) { +// delegate.checkCanRenameView(context, viewName, newViewName); +// } +// } +// +// @Override +// public void checkCanDropView(ConnectorSecurityContext context, SchemaTableName viewName) +// { +// try (ThreadContextClassLoader ignored = new ThreadContextClassLoader(classLoader)) { +// delegate.checkCanDropView(context, viewName); +// } +// } +// +// @Override +// public void checkCanCreateViewWithSelectFromColumns(ConnectorSecurityContext context, SchemaTableName tableName, Set columnNames) +// { +// try (ThreadContextClassLoader ignored = new ThreadContextClassLoader(classLoader)) { +// delegate.checkCanCreateViewWithSelectFromColumns(context, tableName, columnNames); +// } +// } +// +// @Override +// public void checkCanSetCatalogSessionProperty(ConnectorSecurityContext context, String propertyName) +// { +// try (ThreadContextClassLoader ignored = new ThreadContextClassLoader(classLoader)) { +// delegate.checkCanSetCatalogSessionProperty(context, propertyName); +// } +// } +// +// @Override +// public void checkCanGrantTablePrivilege(ConnectorSecurityContext context, Privilege privilege, SchemaTableName tableName, PrestoPrincipal grantee, boolean withGrantOption) +// { +// try (ThreadContextClassLoader ignored = new ThreadContextClassLoader(classLoader)) { +// delegate.checkCanGrantTablePrivilege(context, privilege, tableName, grantee, withGrantOption); +// } +// } +// +// @Override +// public void checkCanRevokeTablePrivilege(ConnectorSecurityContext context, Privilege privilege, SchemaTableName tableName, PrestoPrincipal revokee, boolean grantOptionFor) +// { +// try (ThreadContextClassLoader ignored = new ThreadContextClassLoader(classLoader)) { +// delegate.checkCanRevokeTablePrivilege(context, privilege, tableName, revokee, grantOptionFor); +// } +// } +// +// @Override +// public void checkCanCreateRole(ConnectorSecurityContext context, String role, Optional grantor) +// { +// try (ThreadContextClassLoader ignored = new ThreadContextClassLoader(classLoader)) { +// delegate.checkCanCreateRole(context, role, grantor); +// } +// } +// +// @Override +// public void checkCanDropRole(ConnectorSecurityContext context, String role) +// { +// try (ThreadContextClassLoader ignored = new ThreadContextClassLoader(classLoader)) { +// delegate.checkCanDropRole(context, role); +// } +// } +// +// @Override +// public void checkCanGrantRoles(ConnectorSecurityContext context, Set roles, Set grantees, boolean withAdminOption, Optional grantor, String catalogName) +// { +// try (ThreadContextClassLoader ignored = new ThreadContextClassLoader(classLoader)) { +// delegate.checkCanGrantRoles(context, roles, grantees, withAdminOption, grantor, catalogName); +// } +// } +// +// @Override +// public void checkCanRevokeRoles(ConnectorSecurityContext context, Set roles, Set grantees, boolean adminOptionFor, Optional grantor, String catalogName) +// { +// try (ThreadContextClassLoader ignored = new ThreadContextClassLoader(classLoader)) { +// delegate.checkCanRevokeRoles(context, roles, grantees, adminOptionFor, grantor, catalogName); +// } +// } +// +// @Override +// public void checkCanSetRole(ConnectorSecurityContext context, String role, String catalogName) +// { +// try (ThreadContextClassLoader ignored = new ThreadContextClassLoader(classLoader)) { +// delegate.checkCanSetRole(context, role, catalogName); +// } +// } +// +// @Override +// public void checkCanShowRoles(ConnectorSecurityContext context, String catalogName) +// { +// try (ThreadContextClassLoader ignored = new ThreadContextClassLoader(classLoader)) { +// delegate.checkCanShowRoles(context, catalogName); +// } +// } +// +// @Override +// public void checkCanShowCurrentRoles(ConnectorSecurityContext context, String catalogName) +// { +// try (ThreadContextClassLoader ignored = new ThreadContextClassLoader(classLoader)) { +// delegate.checkCanShowCurrentRoles(context, catalogName); +// } +// } +// +// @Override +// public void checkCanShowRoleGrants(ConnectorSecurityContext context, String catalogName) +// { +// try (ThreadContextClassLoader ignored = new ThreadContextClassLoader(classLoader)) { +// delegate.checkCanShowRoleGrants(context, catalogName); +// } +// } +//} diff --git a/presto-spi/src/main/java/io/prestosql/spi/connector/classloader/ClassLoaderSafeConnectorMetadata.java b/presto-plugin-toolkit/src/main/java/io/prestosql/plugin/base/classloader/ClassLoaderSafeConnectorMetadata.java similarity index 99% rename from presto-spi/src/main/java/io/prestosql/spi/connector/classloader/ClassLoaderSafeConnectorMetadata.java rename to presto-plugin-toolkit/src/main/java/io/prestosql/plugin/base/classloader/ClassLoaderSafeConnectorMetadata.java index 15fe7cc75..525df9aab 100644 --- a/presto-spi/src/main/java/io/prestosql/spi/connector/classloader/ClassLoaderSafeConnectorMetadata.java +++ b/presto-plugin-toolkit/src/main/java/io/prestosql/plugin/base/classloader/ClassLoaderSafeConnectorMetadata.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.prestosql.spi.connector.classloader; +package io.prestosql.plugin.base.classloader; import io.airlift.slice.Slice; import io.prestosql.spi.PartialAndFinalAggregationType; @@ -64,6 +64,8 @@ import io.prestosql.spi.statistics.TableStatistics; import io.prestosql.spi.statistics.TableStatisticsMetadata; import io.prestosql.spi.type.Type; +import javax.inject.Inject; + import java.util.Collection; import java.util.List; import java.util.Map; @@ -80,7 +82,8 @@ public class ClassLoaderSafeConnectorMetadata private final ConnectorMetadata delegate; private final ClassLoader classLoader; - public ClassLoaderSafeConnectorMetadata(ConnectorMetadata delegate, ClassLoader classLoader) + @Inject + public ClassLoaderSafeConnectorMetadata(@ForClassLoaderSafe ConnectorMetadata delegate, ClassLoader classLoader) { this.delegate = requireNonNull(delegate, "delegate is null"); this.classLoader = requireNonNull(classLoader, "classLoader is null"); diff --git a/presto-spi/src/main/java/io/prestosql/spi/connector/classloader/ClassLoaderSafeConnectorPageSink.java b/presto-plugin-toolkit/src/main/java/io/prestosql/plugin/base/classloader/ClassLoaderSafeConnectorPageSink.java similarity index 95% rename from presto-spi/src/main/java/io/prestosql/spi/connector/classloader/ClassLoaderSafeConnectorPageSink.java rename to presto-plugin-toolkit/src/main/java/io/prestosql/plugin/base/classloader/ClassLoaderSafeConnectorPageSink.java index bc4d12dbb..3fe398550 100644 --- a/presto-spi/src/main/java/io/prestosql/spi/connector/classloader/ClassLoaderSafeConnectorPageSink.java +++ b/presto-plugin-toolkit/src/main/java/io/prestosql/plugin/base/classloader/ClassLoaderSafeConnectorPageSink.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.prestosql.spi.connector.classloader; +package io.prestosql.plugin.base.classloader; import io.airlift.slice.Slice; import io.prestosql.spi.Page; @@ -24,6 +24,8 @@ import io.prestosql.spi.connector.ConnectorTransactionHandle; import io.prestosql.spi.snapshot.BlockEncodingSerdeProvider; import io.prestosql.spi.snapshot.RestorableConfig; +import javax.inject.Inject; + import java.util.Collection; import java.util.List; import java.util.concurrent.CompletableFuture; @@ -37,7 +39,8 @@ public class ClassLoaderSafeConnectorPageSink private final ConnectorPageSink delegate; private final ClassLoader classLoader; - public ClassLoaderSafeConnectorPageSink(ConnectorPageSink delegate, ClassLoader classLoader) + @Inject + public ClassLoaderSafeConnectorPageSink(@ForClassLoaderSafe ConnectorPageSink delegate, ClassLoader classLoader) { this.delegate = requireNonNull(delegate, "delegate is null"); this.classLoader = requireNonNull(classLoader, "classLoader is null"); diff --git a/presto-spi/src/main/java/io/prestosql/spi/connector/classloader/ClassLoaderSafeConnectorPageSinkProvider.java b/presto-plugin-toolkit/src/main/java/io/prestosql/plugin/base/classloader/ClassLoaderSafeConnectorPageSinkProvider.java similarity index 95% rename from presto-spi/src/main/java/io/prestosql/spi/connector/classloader/ClassLoaderSafeConnectorPageSinkProvider.java rename to presto-plugin-toolkit/src/main/java/io/prestosql/plugin/base/classloader/ClassLoaderSafeConnectorPageSinkProvider.java index 62d19d927..24fd5ae0b 100644 --- a/presto-spi/src/main/java/io/prestosql/spi/connector/classloader/ClassLoaderSafeConnectorPageSinkProvider.java +++ b/presto-plugin-toolkit/src/main/java/io/prestosql/plugin/base/classloader/ClassLoaderSafeConnectorPageSinkProvider.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.prestosql.spi.connector.classloader; +package io.prestosql.plugin.base.classloader; import io.prestosql.spi.classloader.ThreadContextClassLoader; import io.prestosql.spi.connector.ConnectorDeleteAsInsertTableHandle; @@ -25,6 +25,8 @@ import io.prestosql.spi.connector.ConnectorTransactionHandle; import io.prestosql.spi.connector.ConnectorUpdateTableHandle; import io.prestosql.spi.connector.ConnectorVacuumTableHandle; +import javax.inject.Inject; + import static java.util.Objects.requireNonNull; public final class ClassLoaderSafeConnectorPageSinkProvider @@ -33,7 +35,8 @@ public final class ClassLoaderSafeConnectorPageSinkProvider private final ConnectorPageSinkProvider delegate; private final ClassLoader classLoader; - public ClassLoaderSafeConnectorPageSinkProvider(ConnectorPageSinkProvider delegate, ClassLoader classLoader) + @Inject + public ClassLoaderSafeConnectorPageSinkProvider(@ForClassLoaderSafe ConnectorPageSinkProvider delegate, ClassLoader classLoader) { this.delegate = requireNonNull(delegate, "delegate is null"); this.classLoader = requireNonNull(classLoader, "classLoader is null"); diff --git a/presto-spi/src/main/java/io/prestosql/spi/connector/classloader/ClassLoaderSafeConnectorPageSourceProvider.java b/presto-plugin-toolkit/src/main/java/io/prestosql/plugin/base/classloader/ClassLoaderSafeConnectorPageSourceProvider.java similarity index 92% rename from presto-spi/src/main/java/io/prestosql/spi/connector/classloader/ClassLoaderSafeConnectorPageSourceProvider.java rename to presto-plugin-toolkit/src/main/java/io/prestosql/plugin/base/classloader/ClassLoaderSafeConnectorPageSourceProvider.java index 12cf3c521..5f678a3f7 100644 --- a/presto-spi/src/main/java/io/prestosql/spi/connector/classloader/ClassLoaderSafeConnectorPageSourceProvider.java +++ b/presto-plugin-toolkit/src/main/java/io/prestosql/plugin/base/classloader/ClassLoaderSafeConnectorPageSourceProvider.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.prestosql.spi.connector.classloader; +package io.prestosql.plugin.base.classloader; import io.prestosql.spi.classloader.ThreadContextClassLoader; import io.prestosql.spi.connector.ColumnHandle; @@ -23,6 +23,8 @@ import io.prestosql.spi.connector.ConnectorTableHandle; import io.prestosql.spi.connector.ConnectorTransactionHandle; import io.prestosql.spi.dynamicfilter.DynamicFilterSupplier; +import javax.inject.Inject; + import java.util.List; import java.util.Optional; @@ -34,7 +36,8 @@ public class ClassLoaderSafeConnectorPageSourceProvider private final ConnectorPageSourceProvider delegate; private final ClassLoader classLoader; - public ClassLoaderSafeConnectorPageSourceProvider(ConnectorPageSourceProvider delegate, ClassLoader classLoader) + @Inject + public ClassLoaderSafeConnectorPageSourceProvider(@ForClassLoaderSafe ConnectorPageSourceProvider delegate, ClassLoader classLoader) { this.delegate = requireNonNull(delegate, "delegate is null"); this.classLoader = requireNonNull(classLoader, "classLoader is null"); diff --git a/presto-plugin-toolkit/src/main/java/io/prestosql/plugin/base/classloader/ClassLoaderSafeConnectorRecordSetProvider.java b/presto-plugin-toolkit/src/main/java/io/prestosql/plugin/base/classloader/ClassLoaderSafeConnectorRecordSetProvider.java new file mode 100644 index 000000000..814402237 --- /dev/null +++ b/presto-plugin-toolkit/src/main/java/io/prestosql/plugin/base/classloader/ClassLoaderSafeConnectorRecordSetProvider.java @@ -0,0 +1,60 @@ +/* + * 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.base.classloader; + +import io.prestosql.spi.classloader.ThreadContextClassLoader; +import io.prestosql.spi.connector.ColumnHandle; +import io.prestosql.spi.connector.ConnectorRecordSetProvider; +import io.prestosql.spi.connector.ConnectorSession; +import io.prestosql.spi.connector.ConnectorSplit; +import io.prestosql.spi.connector.ConnectorTableHandle; +import io.prestosql.spi.connector.ConnectorTransactionHandle; +import io.prestosql.spi.connector.RecordSet; + +import javax.inject.Inject; + +import java.util.List; + +import static java.util.Objects.requireNonNull; + +public class ClassLoaderSafeConnectorRecordSetProvider + implements ConnectorRecordSetProvider +{ + private final ConnectorRecordSetProvider delegate; + private final ClassLoader classLoader; + + @Inject + public ClassLoaderSafeConnectorRecordSetProvider(@ForClassLoaderSafe ConnectorRecordSetProvider delegate, ClassLoader classLoader) + { + this.delegate = requireNonNull(delegate, "delegate is null"); + this.classLoader = requireNonNull(classLoader, "classLoader is null"); + } + + @Override + public RecordSet getRecordSet(ConnectorTransactionHandle transaction, ConnectorSession session, ConnectorSplit split, ConnectorTableHandle table, List columns) + { + try (ThreadContextClassLoader ignored = new ThreadContextClassLoader(classLoader)) { + return new ClassLoaderSafeRecordSet(delegate.getRecordSet(transaction, session, split, table, columns), classLoader); + } + } + + @Override + public RecordSet getRecordSet(ConnectorTransactionHandle transactionHandle, ConnectorSession session, ConnectorSplit split, List columns) + { + try (ThreadContextClassLoader ignored = new ThreadContextClassLoader(classLoader)) { + return new ClassLoaderSafeRecordSet(delegate.getRecordSet(transactionHandle, session, split, columns), classLoader); + } + } +} diff --git a/presto-spi/src/main/java/io/prestosql/spi/connector/classloader/ClassLoaderSafeConnectorSplitManager.java b/presto-plugin-toolkit/src/main/java/io/prestosql/plugin/base/classloader/ClassLoaderSafeConnectorSplitManager.java similarity index 94% rename from presto-spi/src/main/java/io/prestosql/spi/connector/classloader/ClassLoaderSafeConnectorSplitManager.java rename to presto-plugin-toolkit/src/main/java/io/prestosql/plugin/base/classloader/ClassLoaderSafeConnectorSplitManager.java index 727955327..7af961516 100644 --- a/presto-spi/src/main/java/io/prestosql/spi/connector/classloader/ClassLoaderSafeConnectorSplitManager.java +++ b/presto-plugin-toolkit/src/main/java/io/prestosql/plugin/base/classloader/ClassLoaderSafeConnectorSplitManager.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.prestosql.spi.connector.classloader; +package io.prestosql.plugin.base.classloader; import io.prestosql.spi.classloader.ThreadContextClassLoader; import io.prestosql.spi.connector.ColumnMetadata; @@ -26,6 +26,8 @@ import io.prestosql.spi.dynamicfilter.DynamicFilter; import io.prestosql.spi.predicate.TupleDomain; import io.prestosql.spi.resourcegroups.QueryType; +import javax.inject.Inject; + import java.util.List; import java.util.Map; import java.util.Optional; @@ -40,7 +42,8 @@ public final class ClassLoaderSafeConnectorSplitManager private final ConnectorSplitManager delegate; private final ClassLoader classLoader; - public ClassLoaderSafeConnectorSplitManager(ConnectorSplitManager delegate, ClassLoader classLoader) + @Inject + public ClassLoaderSafeConnectorSplitManager(@ForClassLoaderSafe ConnectorSplitManager delegate, ClassLoader classLoader) { this.delegate = requireNonNull(delegate, "delegate is null"); this.classLoader = requireNonNull(classLoader, "classLoader is null"); diff --git a/presto-spi/src/main/java/io/prestosql/spi/connector/classloader/ClassLoaderSafeNodePartitioningProvider.java b/presto-plugin-toolkit/src/main/java/io/prestosql/plugin/base/classloader/ClassLoaderSafeNodePartitioningProvider.java similarity index 93% rename from presto-spi/src/main/java/io/prestosql/spi/connector/classloader/ClassLoaderSafeNodePartitioningProvider.java rename to presto-plugin-toolkit/src/main/java/io/prestosql/plugin/base/classloader/ClassLoaderSafeNodePartitioningProvider.java index d72719742..3fd871294 100644 --- a/presto-spi/src/main/java/io/prestosql/spi/connector/classloader/ClassLoaderSafeNodePartitioningProvider.java +++ b/presto-plugin-toolkit/src/main/java/io/prestosql/plugin/base/classloader/ClassLoaderSafeNodePartitioningProvider.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.prestosql.spi.connector.classloader; +package io.prestosql.plugin.base.classloader; import io.prestosql.spi.classloader.ThreadContextClassLoader; import io.prestosql.spi.connector.BucketFunction; @@ -24,6 +24,8 @@ import io.prestosql.spi.connector.ConnectorSplit; import io.prestosql.spi.connector.ConnectorTransactionHandle; import io.prestosql.spi.type.Type; +import javax.inject.Inject; + import java.util.List; import java.util.function.ToIntFunction; @@ -35,7 +37,8 @@ public final class ClassLoaderSafeNodePartitioningProvider private final ConnectorNodePartitioningProvider delegate; private final ClassLoader classLoader; - public ClassLoaderSafeNodePartitioningProvider(ConnectorNodePartitioningProvider delegate, ClassLoader classLoader) + @Inject + public ClassLoaderSafeNodePartitioningProvider(@ForClassLoaderSafe ConnectorNodePartitioningProvider delegate, ClassLoader classLoader) { this.delegate = requireNonNull(delegate, "delegate is null"); this.classLoader = requireNonNull(classLoader, "classLoader is null"); diff --git a/presto-plugin-toolkit/src/main/java/io/prestosql/plugin/base/classloader/ClassLoaderSafeRecordSet.java b/presto-plugin-toolkit/src/main/java/io/prestosql/plugin/base/classloader/ClassLoaderSafeRecordSet.java new file mode 100644 index 000000000..00c1dc690 --- /dev/null +++ b/presto-plugin-toolkit/src/main/java/io/prestosql/plugin/base/classloader/ClassLoaderSafeRecordSet.java @@ -0,0 +1,56 @@ +/* + * 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.base.classloader; + +import io.prestosql.spi.classloader.ThreadContextClassLoader; +import io.prestosql.spi.connector.RecordCursor; +import io.prestosql.spi.connector.RecordSet; +import io.prestosql.spi.type.Type; + +import javax.inject.Inject; + +import java.util.List; + +import static java.util.Objects.requireNonNull; + +public class ClassLoaderSafeRecordSet + implements RecordSet +{ + private final RecordSet delegate; + private final ClassLoader classLoader; + + @Inject + public ClassLoaderSafeRecordSet(@ForClassLoaderSafe RecordSet delegate, ClassLoader classLoader) + { + this.delegate = requireNonNull(delegate, "delegate is null"); + this.classLoader = requireNonNull(classLoader, "classLoader is null"); + } + + @Override + public List getColumnTypes() + { + try (ThreadContextClassLoader ignored = new ThreadContextClassLoader(classLoader)) { + return delegate.getColumnTypes(); + } + } + + @Override + public RecordCursor cursor() + { + try (ThreadContextClassLoader ignored = new ThreadContextClassLoader(classLoader)) { + return delegate.cursor(); + } + } +} diff --git a/presto-spi/src/test/java/io/prestosql/spi/connector/classloader/ClassLoaderSafeConnectorMetadataTest.java b/presto-plugin-toolkit/src/test/java/io/prestosql/plugin/base/classloader/ClassLoaderSafeConnectorMetadataTest.java similarity index 99% rename from presto-spi/src/test/java/io/prestosql/spi/connector/classloader/ClassLoaderSafeConnectorMetadataTest.java rename to presto-plugin-toolkit/src/test/java/io/prestosql/plugin/base/classloader/ClassLoaderSafeConnectorMetadataTest.java index f89c3c629..19ab4432c 100644 --- a/presto-spi/src/test/java/io/prestosql/spi/connector/classloader/ClassLoaderSafeConnectorMetadataTest.java +++ b/presto-plugin-toolkit/src/test/java/io/prestosql/plugin/base/classloader/ClassLoaderSafeConnectorMetadataTest.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.prestosql.spi.connector.classloader; +package io.prestosql.plugin.base.classloader; import io.airlift.slice.Slice; import io.prestosql.spi.PartialAndFinalAggregationType; diff --git a/presto-spi/src/test/java/io/prestosql/spi/connector/classloader/ClassLoaderSafeConnectorPageSinkProviderTest.java b/presto-plugin-toolkit/src/test/java/io/prestosql/plugin/base/classloader/ClassLoaderSafeConnectorPageSinkProviderTest.java similarity index 99% rename from presto-spi/src/test/java/io/prestosql/spi/connector/classloader/ClassLoaderSafeConnectorPageSinkProviderTest.java rename to presto-plugin-toolkit/src/test/java/io/prestosql/plugin/base/classloader/ClassLoaderSafeConnectorPageSinkProviderTest.java index 0f4ce9746..f1b156c40 100644 --- a/presto-spi/src/test/java/io/prestosql/spi/connector/classloader/ClassLoaderSafeConnectorPageSinkProviderTest.java +++ b/presto-plugin-toolkit/src/test/java/io/prestosql/plugin/base/classloader/ClassLoaderSafeConnectorPageSinkProviderTest.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.prestosql.spi.connector.classloader; +package io.prestosql.plugin.base.classloader; import io.prestosql.spi.connector.ConnectorDeleteAsInsertTableHandle; import io.prestosql.spi.connector.ConnectorInsertTableHandle; diff --git a/presto-spi/src/test/java/io/prestosql/spi/connector/classloader/ClassLoaderSafeConnectorPageSinkTest.java b/presto-plugin-toolkit/src/test/java/io/prestosql/plugin/base/classloader/ClassLoaderSafeConnectorPageSinkTest.java similarity index 99% rename from presto-spi/src/test/java/io/prestosql/spi/connector/classloader/ClassLoaderSafeConnectorPageSinkTest.java rename to presto-plugin-toolkit/src/test/java/io/prestosql/plugin/base/classloader/ClassLoaderSafeConnectorPageSinkTest.java index f02d581f8..a12dcc4ea 100644 --- a/presto-spi/src/test/java/io/prestosql/spi/connector/classloader/ClassLoaderSafeConnectorPageSinkTest.java +++ b/presto-plugin-toolkit/src/test/java/io/prestosql/plugin/base/classloader/ClassLoaderSafeConnectorPageSinkTest.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.prestosql.spi.connector.classloader; +package io.prestosql.plugin.base.classloader; import io.airlift.slice.Slice; import io.prestosql.spi.Page; diff --git a/presto-spi/src/test/java/io/prestosql/spi/connector/classloader/ClassLoaderSafeConnectorPageSourceProviderTest.java b/presto-plugin-toolkit/src/test/java/io/prestosql/plugin/base/classloader/ClassLoaderSafeConnectorPageSourceProviderTest.java similarity index 99% rename from presto-spi/src/test/java/io/prestosql/spi/connector/classloader/ClassLoaderSafeConnectorPageSourceProviderTest.java rename to presto-plugin-toolkit/src/test/java/io/prestosql/plugin/base/classloader/ClassLoaderSafeConnectorPageSourceProviderTest.java index 342a6087f..4a28f8a59 100644 --- a/presto-spi/src/test/java/io/prestosql/spi/connector/classloader/ClassLoaderSafeConnectorPageSourceProviderTest.java +++ b/presto-plugin-toolkit/src/test/java/io/prestosql/plugin/base/classloader/ClassLoaderSafeConnectorPageSourceProviderTest.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.prestosql.spi.connector.classloader; +package io.prestosql.plugin.base.classloader; import io.prestosql.spi.connector.ColumnHandle; import io.prestosql.spi.connector.ConnectorPageSource; diff --git a/presto-spi/src/test/java/io/prestosql/spi/connector/classloader/ClassLoaderSafeConnectorSplitManagerTest.java b/presto-plugin-toolkit/src/test/java/io/prestosql/plugin/base/classloader/ClassLoaderSafeConnectorSplitManagerTest.java similarity index 99% rename from presto-spi/src/test/java/io/prestosql/spi/connector/classloader/ClassLoaderSafeConnectorSplitManagerTest.java rename to presto-plugin-toolkit/src/test/java/io/prestosql/plugin/base/classloader/ClassLoaderSafeConnectorSplitManagerTest.java index 4e33fa126..a494b8378 100644 --- a/presto-spi/src/test/java/io/prestosql/spi/connector/classloader/ClassLoaderSafeConnectorSplitManagerTest.java +++ b/presto-plugin-toolkit/src/test/java/io/prestosql/plugin/base/classloader/ClassLoaderSafeConnectorSplitManagerTest.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.prestosql.spi.connector.classloader; +package io.prestosql.plugin.base.classloader; import io.prestosql.spi.connector.ColumnMetadata; import io.prestosql.spi.connector.ConnectorSession; diff --git a/presto-spi/src/test/java/io/prestosql/spi/connector/classloader/ClassLoaderSafeNodePartitioningProviderTest.java b/presto-plugin-toolkit/src/test/java/io/prestosql/plugin/base/classloader/ClassLoaderSafeNodePartitioningProviderTest.java similarity index 99% rename from presto-spi/src/test/java/io/prestosql/spi/connector/classloader/ClassLoaderSafeNodePartitioningProviderTest.java rename to presto-plugin-toolkit/src/test/java/io/prestosql/plugin/base/classloader/ClassLoaderSafeNodePartitioningProviderTest.java index cf74ecc9e..e55372d17 100644 --- a/presto-spi/src/test/java/io/prestosql/spi/connector/classloader/ClassLoaderSafeNodePartitioningProviderTest.java +++ b/presto-plugin-toolkit/src/test/java/io/prestosql/plugin/base/classloader/ClassLoaderSafeNodePartitioningProviderTest.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.prestosql.spi.connector.classloader; +package io.prestosql.plugin.base.classloader; import io.prestosql.spi.connector.BucketFunction; import io.prestosql.spi.connector.ConnectorBucketNodeMap; diff --git a/presto-spi/src/test/java/io/prestosql/spi/connector/classloader/TestClassLoaderSafeWrappers.java b/presto-plugin-toolkit/src/test/java/io/prestosql/plugin/base/classloader/TestClassLoaderSafeWrappers.java similarity index 97% rename from presto-spi/src/test/java/io/prestosql/spi/connector/classloader/TestClassLoaderSafeWrappers.java rename to presto-plugin-toolkit/src/test/java/io/prestosql/plugin/base/classloader/TestClassLoaderSafeWrappers.java index e58b71b34..a4501cb76 100644 --- a/presto-spi/src/test/java/io/prestosql/spi/connector/classloader/TestClassLoaderSafeWrappers.java +++ b/presto-plugin-toolkit/src/test/java/io/prestosql/plugin/base/classloader/TestClassLoaderSafeWrappers.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.prestosql.spi.connector.classloader; +package io.prestosql.plugin.base.classloader; import io.prestosql.spi.connector.ConnectorMetadata; import io.prestosql.spi.connector.ConnectorNodePartitioningProvider; diff --git a/presto-spi/src/main/java/io/prestosql/spi/connector/ConnectorMetadata.java b/presto-spi/src/main/java/io/prestosql/spi/connector/ConnectorMetadata.java index c4cc4f906..afd3c0dcf 100644 --- a/presto-spi/src/main/java/io/prestosql/spi/connector/ConnectorMetadata.java +++ b/presto-spi/src/main/java/io/prestosql/spi/connector/ConnectorMetadata.java @@ -582,12 +582,6 @@ public interface ConnectorMetadata return beginInsert(session, tableHandle, columns); } - @Deprecated - default ConnectorInsertTableHandle beginInsert(ConnectorSession session, ConnectorTableHandle tableHandle, List columns) - { - return beginInsert(session, tableHandle); - } - /** * Begin the atomic creation of a table with data. * @@ -626,13 +620,30 @@ public interface ConnectorMetadata default void cleanupQuery(ConnectorSession session) {} /** - * Begin insert query + * @deprecated Use {@link #beginInsert(ConnectorSession, ConnectorTableHandle, List)} instead. */ + @Deprecated default ConnectorInsertTableHandle beginInsert(ConnectorSession session, ConnectorTableHandle tableHandle) { throw new PrestoException(NOT_SUPPORTED, "This connector does not support inserts"); } + /** + * Begin insert query + */ + default ConnectorInsertTableHandle beginInsert(ConnectorSession session, ConnectorTableHandle tableHandle, List columns) + { + return beginInsert(session, tableHandle); + } + + /** + * @return whether connector handles missing columns during insert + */ + default boolean supportsMissingColumnsOnInsert() + { + return false; + } + /** * Begin insert query. */ diff --git a/presto-spi/src/main/java/io/prestosql/spi/type/TypeSignature.java b/presto-spi/src/main/java/io/prestosql/spi/type/TypeSignature.java index 512fa2977..cee88c29c 100644 --- a/presto-spi/src/main/java/io/prestosql/spi/type/TypeSignature.java +++ b/presto-spi/src/main/java/io/prestosql/spi/type/TypeSignature.java @@ -28,7 +28,6 @@ import java.util.TreeSet; import java.util.regex.Pattern; import java.util.stream.Collectors; -import static io.prestosql.spi.type.TypeSignatureParameter.of; import static java.lang.Character.isDigit; import static java.lang.String.format; import static java.util.Arrays.asList; @@ -400,7 +399,7 @@ public class TypeSignature public static TypeSignature arrayType(TypeSignature elementType) { - return new TypeSignature(StandardTypes.ARRAY, of(elementType)); + return new TypeSignature(StandardTypes.ARRAY, TypeSignatureParameter.of(elementType)); } public static TypeSignature arrayType(TypeSignatureParameter elementType) @@ -410,7 +409,7 @@ public class TypeSignature public static TypeSignature mapType(TypeSignature keyType, TypeSignature valueType) { - return new TypeSignature(StandardTypes.MAP, of(keyType), of(valueType)); + return new TypeSignature(StandardTypes.MAP, TypeSignatureParameter.of(keyType), TypeSignatureParameter.of(valueType)); } public static TypeSignature parametricType(String name, TypeSignature... parameters) @@ -425,7 +424,7 @@ public class TypeSignature public static TypeSignature functionType(TypeSignature first, TypeSignature... rest) { List parameters = new ArrayList<>(); - parameters.add(of(first)); + parameters.add(TypeSignatureParameter.of(first)); Arrays.asList(rest).stream() .map(TypeSignatureParameter::of) -- Gitee From 0152ffdcb89b444fc443266ea47d537dfab00e0f Mon Sep 17 00:00:00 2001 From: XHLY Date: Fri, 26 Jun 2020 03:01:40 +0800 Subject: [PATCH 4/8] do code cleanup (step 1/2) - use new DataSize - remove forgoing kafka useless code - take out useless DynamicFilter in Kafka, - discard the forgoing class - KafkaSimpleConsumerManager - ... --- hetu-hazelcast/pom.xml | 1 - pom.xml | 7 + .../plugin/kafka/KafkaColumnHandle.java | 6 +- .../prestosql/plugin/kafka/KafkaConfig.java | 2 +- .../plugin/kafka/KafkaConnectorModule.java | 5 +- .../plugin/kafka/KafkaFilterManager.java | 2 +- .../kafka/KafkaInternalFieldManager.java | 12 +- .../prestosql/plugin/kafka/KafkaMetadata.java | 2 +- .../plugin/kafka/KafkaPageSinkProvider.java | 3 +- .../plugin/kafka/KafkaRecordSet.java | 50 +++- .../kafka/KafkaSimpleConsumerManager.java | 276 +++++++++--------- .../plugin/kafka/KafkaSplitManager.java | 4 +- 12 files changed, 195 insertions(+), 175 deletions(-) diff --git a/hetu-hazelcast/pom.xml b/hetu-hazelcast/pom.xml index ee93729f7..ce85ab9fe 100644 --- a/hetu-hazelcast/pom.xml +++ b/hetu-hazelcast/pom.xml @@ -15,7 +15,6 @@ ${project.parent.basedir} 4.6.3 3.21.0 - 5.1 1.9.4 5.3.27 1.1.1 diff --git a/pom.xml b/pom.xml index 513b834e0..0a4b073ef 100644 --- a/pom.xml +++ b/pom.xml @@ -64,6 +64,7 @@ 0.7.0 5.0.3 2.4.1 + 5.1