diff --git itests/qtest-druid/pom.xml itests/qtest-druid/pom.xml index e566fcf4d7..19cdf918b2 100644 --- itests/qtest-druid/pom.xml +++ itests/qtest-druid/pom.xml @@ -43,7 +43,7 @@ 10.11.1.1 16.0.1 4.1.0 - 1.0.1 + 2.0.0 @@ -207,6 +207,11 @@ kafka_2.11 ${kafka.version} + + org.slf4j + slf4j-api + 1.7.25 + diff --git kafka-handler/README.md kafka-handler/README.md new file mode 100644 index 0000000000..706c77ae25 --- /dev/null +++ kafka-handler/README.md @@ -0,0 +1,217 @@ +#Kafka Storage Handler Module + +Storage Handler that allows user to Connect/Analyse/Transform Kafka topics. +The workflow is as follow, first the user will create an external table that is a view over one Kafka topic, +then the user will be able to run any SQL query including write back to the same table or different kafka backed table. + +## Usage + +### Create Table +Use following statement to create table: +```sql +CREATE EXTERNAL TABLE kafka_table +(`timestamp` timestamp , `page` string, `newPage` boolean, + added int, deleted bigint, delta double) +STORED BY 'org.apache.hadoop.hive.kafka.KafkaStorageHandler' +TBLPROPERTIES +("kafka.topic" = "test-topic", "kafka.bootstrap.servers"="localhost:9092"); +``` +Table property `kafka.topic` is the Kafka Topic to connect to and `kafka.bootstrap.servers` is the Broker connection string. +Both properties are mandatory. +On the write path if such a topic does not exists the topic will be created if Kafka broker admin policy allow such operation. + +By default the serializer and deserializer is Json `org.apache.hadoop.hive.serde2.JsonSerDe`. +If you want to switch serializer/deserializer classes you can use alter table. +```sql +ALTER TABLE kafka_table SET TBLPROPERTIES ("kafka.serde.class"="org.apache.hadoop.hive.serde2.avro.AvroSerDe"); +``` +List of supported Serializer Deserializer: + +|Supported Serializer Deserializer| +|-----| +|org.apache.hadoop.hive.serde2.JsonSerDe| +|org.apache.hadoop.hive.serde2.OpenCSVSerde| +|org.apache.hadoop.hive.serde2.avro.AvroSerDe| +|org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe| +|org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe| + +#### Table definition +In addition to the user defined payload schema Kafka Storage Handler will append additional columns allowing user to query the Kafka metadata fields: +- `__key` Kafka record key (byte array) +- `__partition` Kafka record partition identifier (int 32) +- `__offset` Kafka record offset (int 64) +- `__timestamp` Kafka record timestamp (int 64) + + +### Query Table + +List the table properties and all the partition/offsets information for the topic. +```sql +Describe extended kafka_table; +``` + +Count the number of records with Kafka record timestamp within the last 10 minutes interval. + +```sql +SELECT count(*) from kafka_table +where `__timestamp` > 1000 * to_unix_timestamp(CURRENT_TIMESTAMP - interval '10' MINUTES); +``` +The storage handler allow filter push-down read optimization, +for instance the query above will only read the records with timestamp satisfying the filter predicate. +Please note that such time based seek is only viable if the Kafka broker allow time based lookup (Kafka 0.11 or later versions) + +In addition to **time based seek**, the storage handler reader is able to seek to a particular partition offset using the SQL WHERE clause. +Currently only support OR/AND with (<, <=, >=, >) + +```sql +SELECT count(*) from kafka_table +where (`__offset` < 10 and `__offset`>3 and `__partition` = 0) +or (`__partition` = 0 and `__offset` < 105 and `__offset` > 99) +or (`__offset` = 109); +``` + +User can define a view to take of the last 15 minutes and mask what ever column as follow: + +```sql +CREATE VIEW last_15_minutes_of_kafka_table as select `timestamp`, `user`, delta, added from kafka_table +where `__timestamp` > 1000 * to_unix_timestamp(CURRENT_TIMESTAMP - interval '15' MINUTES); +``` + +Join the Kafka Stream to Hive table. For instance assume you want to join the last 15 minutes of stream to dimension table like the following. +```sql +CREATE TABLE user_table (`user` string, `first_name` string , age int, gender string, comments string) STORED as ORC ; +``` + +Join the view of the last 15 minutes to `user_table`, group by user gender column and compute aggregates +over metrics from fact table and dimension table. + +```sql +SELECT sum(added) as added, sum(deleted) as deleted, avg(delta) as delta, avg(age) as avg_age , gender +FROM last_15_minutes_of_kafka_table join user_table on `last_15_minutes_of_kafka_table`.`user` = `user_table`.`user` +GROUP BY gender limit 10; +``` + + +Join the Stream to the Stream it self. In cases where you want to perform some Ad-Hoc query over the last 15 minutes view. +In the following example we show how you can perform classical user retention analysis over the Kafka Stream. +```sql +-- Steam join over the view it self +-- The example is adapted from https://www.periscopedata.com/blog/how-to-calculate-cohort-retention-in-sql +-- Assuming l15min_wiki is a view of the last 15 minutes +select count( distinct activity.`user`) as active_users, count(distinct future_activity.`user`) as retained_users +from l15min_wiki as activity +left join l15min_wiki as future_activity on + activity.`user` = future_activity.`user` + and activity.`timestamp` = future_activity.`timestamp` - interval '5' minutes ; + +-- Stream to stream join +-- Assuming wiki_kafka_hive is the entire stream. +select floor_hour(activity.`timestamp`), count( distinct activity.`user`) as active_users, count(distinct future_activity.`user`) as retained_users +from wiki_kafka_hive as activity +left join wiki_kafka_hive as future_activity on + activity.`user` = future_activity.`user` + and activity.`timestamp` = future_activity.`timestamp` - interval '1' hour group by floor_hour(activity.`timestamp`); + +``` + +#Configuration + +## Table Properties + +| Property | Description | Mandatory | Default | +|-------------------------------------|------------------------------------------------------------------------------------------------------------------------------------|-----------|-----------------------------------------| +| kafka.topic | Kafka topic name to map the table to. | Yes | null | +| kafka.bootstrap.servers | Table property indicating Kafka broker(s) connection string. | Yes | null | +| kafka.serde.class | Serializer and Deserializer class implementation. | No | org.apache.hadoop.hive.serde2.JsonSerDe | +| hive.kafka.poll.timeout.ms | Parameter indicating Kafka Consumer poll timeout period in millis. FYI this is independent from internal Kafka consumer timeouts. | No | 5000 (5 Seconds) | +| hive.kafka.max.retries | Number of retries for Kafka metadata fetch operations. | No | 6 | +| hive.kafka.metadata.poll.timeout.ms | Number of milliseconds before consumer timeout on fetching Kafka metadata. | No | 30000 (30 Seconds) | +| kafka.write.semantic | Writer semantic, allowed values (BEST_EFFORT, AT_LEAST_ONCE, EXACTLY_ONCE) | No | AT_LEAST_ONCE | +| hive.kafka.optimistic.commit | Boolean value indicate the if the producer should commit during task or delegate the commit to HS2. | No | true | + +### Setting Extra Consumer/Producer properties. +User can inject custom Kafka consumer/producer properties via the Table properties. +To do so user can add any key/value pair of Kafka config to the Hive table property +by prefixing the key with `kafka.consumer` for consumer configs and `kafka.producer` for producer configs. +For instance the following alter table query adds the table property `"kafka.consumer.max.poll.records" = "5000"` +and will inject `max.poll.records=5000` to the Kafka Consumer. +```sql +ALTER TABLE kafka_table SET TBLPROPERTIES ("kafka.consumer.max.poll.records"="5000"); +``` + +#Kafka to Hive ETL PIPE LINE + +load form Kafka every Record exactly once +Goal is to read data and commit both data and its offsets in a single Transaction + +First create the offset table. +```sql +Drop table kafka_table_offsets; +create table kafka_table_offsets(partition_id int, max_offset bigint, insert_time timestamp); +``` + +Initialize the table +```sql +insert overwrite table kafka_table_offsets select `__partition`, min(`__offset`) - 1, CURRENT_TIMESTAMP +from wiki_kafka_hive group by `__partition`, CURRENT_TIMESTAMP ; +``` +Create the end target table on the Hive warehouse. +```sql +Drop table orc_kafka_table; +Create table orc_kafka_table (partition_id int, koffset bigint, ktimestamp bigint, + `timestamp` timestamp , `page` string, `user` string, `diffurl` string, + `isrobot` boolean, added int, deleted int, delta bigint +) stored as ORC; +``` +This an example tp insert up to offset = 2 only + +```sql +From wiki_kafka_hive ktable JOIN kafka_table_offsets offset_table +on (ktable.`__partition` = offset_table.partition_id +and ktable.`__offset` > offset_table.max_offset and ktable.`__offset` < 3 ) +insert into table orc_kafka_table select `__partition`, `__offset`, `__timestamp`, +`timestamp`, `page`, `user`, `diffurl`, `isrobot`, added , deleted , delta +Insert overwrite table kafka_table_offsets select +`__partition`, max(`__offset`), CURRENT_TIMESTAMP group by `__partition`, CURRENT_TIMESTAMP; +``` + +Double check the insert +```sql +select max(`koffset`) from orc_kafka_table limit 10; +select count(*) as c from orc_kafka_table group by partition_id, koffset having c > 1; +``` + +Repeat this periodically to insert all data. + +```sql +From wiki_kafka_hive ktable JOIN kafka_table_offsets offset_table +on (ktable.`__partition` = offset_table.partition_id +and ktable.`__offset` > offset_table.max_offset ) +insert into table orc_kafka_table select `__partition`, `__offset`, `__timestamp`, +`timestamp`, `page`, `user`, `diffurl`, `isrobot`, added , deleted , delta +Insert overwrite table kafka_table_offsets select +`__partition`, max(`__offset`), CURRENT_TIMESTAMP group by `__partition`, CURRENT_TIMESTAMP; +``` + +#ETL from Hive to Kafka + +##INSERT INTO +First create the table in have that will be the target table. Now all the inserts will go to the topic mapped by this Table. + +```sql +CREATE EXTERNAL TABLE moving_avg_wiki_kafka_hive +(`channel` string, `namespace` string,`page` string, `timestamp` timestamp , avg_delta double ) +STORED BY 'org.apache.hadoop.hive.kafka.KafkaStorageHandler' +TBLPROPERTIES +("kafka.topic" = "moving_avg_wiki_kafka_hive_2", +"kafka.bootstrap.servers"="cn105-10.l42scl.hortonworks.com:9092", +-- STORE AS AVRO IN KAFKA +"kafka.serde.class"="org.apache.hadoop.hive.serde2.avro.AvroSerDe"); +``` + +Then insert data into the table. Keep in mind that Kafka is an append only, thus you can not use insert overwrite. +```sql +insert into table moving_avg_wiki_kafka_hive select `channel`, `namespace`, `page`, `timestamp`, +avg(delta) over (order by `timestamp` asc rows between 60 preceding and current row) as avg_delta, +null as `__key`, null as `__partition`, -1, -1,-1, -1 from l15min_wiki; +``` diff --git kafka-handler/pom.xml kafka-handler/pom.xml index 6c58bf1df1..f907e9ddf0 100644 --- kafka-handler/pom.xml +++ kafka-handler/pom.xml @@ -30,7 +30,7 @@ .. - 1.0.1 + 2.0.0 kafka-handler @@ -38,12 +38,18 @@ Hive Kafka Storage Handler - + org.apache.hive hive-exec provided ${project.version} + + + org.slf4j + slf4j-api + + com.google.guava @@ -52,10 +58,22 @@ org.apache.hadoop hadoop-common + + + org.slf4j + slf4j-api + + org.apache.hadoop hadoop-client + + + org.slf4j + slf4j-api + + org.apache.kafka @@ -90,6 +108,12 @@ ${kafka.version} test + + org.slf4j + slf4j-api + 1.7.25 + test + diff --git kafka-handler/src/java/org/apache/hadoop/hive/kafka/GenericKafkaSerDe.java kafka-handler/src/java/org/apache/hadoop/hive/kafka/GenericKafkaSerDe.java deleted file mode 100644 index a0c79b3a9f..0000000000 --- kafka-handler/src/java/org/apache/hadoop/hive/kafka/GenericKafkaSerDe.java +++ /dev/null @@ -1,162 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you 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 org.apache.hadoop.hive.kafka; - -import com.google.common.base.Preconditions; -import com.google.common.base.Supplier; -import com.google.common.base.Suppliers; -import com.google.common.collect.Lists; -import org.apache.avro.Schema; -import org.apache.avro.generic.GenericRecord; -import org.apache.avro.io.DatumReader; -import org.apache.avro.io.DecoderFactory; -import org.apache.avro.specific.SpecificDatumReader; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hive.serde2.AbstractSerDe; -import org.apache.hadoop.hive.serde2.JsonSerDe; -import org.apache.hadoop.hive.serde2.SerDeException; -import org.apache.hadoop.hive.serde2.SerDeStats; -import org.apache.hadoop.hive.serde2.avro.AvroGenericRecordWritable; -import org.apache.hadoop.hive.serde2.avro.AvroSerDe; -import org.apache.hadoop.hive.serde2.avro.AvroSerdeUtils; -import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; -import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory; -import org.apache.hadoop.hive.serde2.objectinspector.StructField; -import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; -import org.apache.hadoop.io.BytesWritable; -import org.apache.hadoop.io.Text; -import org.apache.hadoop.io.Writable; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import javax.annotation.Nullable; -import java.io.IOException; -import java.rmi.server.UID; -import java.util.ArrayList; -import java.util.List; -import java.util.Properties; -import java.util.function.Function; -import java.util.stream.Collectors; - -/** - * Generic Kafka Serde that allow user to delegate Serde to other class like Avro, - * Json or any class that supports {@link BytesWritable}. - */ -public class GenericKafkaSerDe extends AbstractSerDe { - private static final Logger LOG = LoggerFactory.getLogger(GenericKafkaSerDe.class); - - private AbstractSerDe delegateSerDe; - private ObjectInspector objectInspector; - private final List columnNames = Lists.newArrayList(); - private StructObjectInspector delegateObjectInspector; - private final UID uid = new UID(); - @SuppressWarnings("Guava") private Supplier> gdrSupplier; - - @Override public void initialize(@Nullable Configuration conf, Properties tbl) throws SerDeException { - final String className = tbl.getProperty(KafkaStreamingUtils.SERDE_CLASS_NAME, KafkaJsonSerDe.class.getName()); - delegateSerDe = KafkaStreamingUtils.createDelegate(className); - //noinspection deprecation - delegateSerDe.initialize(conf, tbl); - LOG.debug("Using SerDe instance {}", delegateSerDe.getClass().getCanonicalName()); - - if (!(delegateSerDe.getObjectInspector() instanceof StructObjectInspector)) { - throw new SerDeException("Was expecting StructObject Inspector but have " + delegateSerDe.getObjectInspector() - .getClass() - .getName()); - } - delegateObjectInspector = (StructObjectInspector) delegateSerDe.getObjectInspector(); - - // Build column names Order matters here - columnNames.addAll(delegateObjectInspector.getAllStructFieldRefs() - .stream() - .map(StructField::getFieldName) - .collect(Collectors.toList())); - columnNames.addAll(KafkaStreamingUtils.KAFKA_METADATA_COLUMN_NAMES); - - final List inspectors = new ArrayList<>(columnNames.size()); - inspectors.addAll(delegateObjectInspector.getAllStructFieldRefs() - .stream() - .map(StructField::getFieldObjectInspector) - .collect(Collectors.toList())); - inspectors.addAll(KafkaStreamingUtils.KAFKA_METADATA_INSPECTORS); - objectInspector = ObjectInspectorFactory.getStandardStructObjectInspector(columnNames, inspectors); - - // lazy supplier to read Avro Records if needed - gdrSupplier = getReaderSupplier(tbl); - } - - @Override public Class getSerializedClass() { - return delegateSerDe.getSerializedClass(); - } - - @Override public Writable serialize(Object obj, ObjectInspector objInspector) throws SerDeException { - return delegateSerDe.serialize(obj, objInspector); - } - - @Override public SerDeStats getSerDeStats() { - return delegateSerDe.getSerDeStats(); - } - - @Override public Object deserialize(Writable blob) throws SerDeException { - KafkaRecordWritable record = (KafkaRecordWritable) blob; - // switch case the serde nature - final Object row; - if (delegateSerDe instanceof JsonSerDe) { - //@TODO Text constructor copies the data, this op is not needed - row = delegateSerDe.deserialize(new Text(record.getValue())); - } else if (delegateSerDe instanceof AvroSerDe) { - AvroGenericRecordWritable avroGenericRecordWritable = new AvroGenericRecordWritable(); - GenericRecord avroRecord; - try { - avroRecord = gdrSupplier.get().read(null, DecoderFactory.get().binaryDecoder(record.getValue(), null)); - avroGenericRecordWritable.setRecord(avroRecord); - avroGenericRecordWritable.setRecordReaderID(uid); - avroGenericRecordWritable.setFileSchema(avroRecord.getSchema()); - } catch (IOException e) { - throw new SerDeException(e); - } - row = delegateSerDe.deserialize(avroGenericRecordWritable); - } else { - // default assuming delegate Serde know how to deal with - row = delegateSerDe.deserialize(new BytesWritable(record.getValue())); - } - - return columnNames.stream().map(name -> { - Function metaColumnMapper = KafkaStreamingUtils.recordWritableFnMap.get(name); - if (metaColumnMapper != null) { - return metaColumnMapper.apply(record); - } - return delegateObjectInspector.getStructFieldData(row, delegateObjectInspector.getStructFieldRef(name)); - }).collect(Collectors.toList()); - } - - @Override public ObjectInspector getObjectInspector() { - return objectInspector; - } - - @SuppressWarnings("Guava") private Supplier> getReaderSupplier(Properties tbl) { - return Suppliers.memoize(() -> { - String schemaFromProperty = tbl.getProperty(AvroSerdeUtils.AvroTableProperties.SCHEMA_LITERAL.getPropName(), ""); - Preconditions.checkArgument(!schemaFromProperty.isEmpty(), "Avro Schema is empty Can not go further"); - Schema schema = AvroSerdeUtils.getSchemaFor(schemaFromProperty); - LOG.debug("Building Avro Reader with schema {}", schemaFromProperty); - return new SpecificDatumReader<>(schema); - }); - } -} diff --git kafka-handler/src/java/org/apache/hadoop/hive/kafka/HiveKafkaProducer.java kafka-handler/src/java/org/apache/hadoop/hive/kafka/HiveKafkaProducer.java new file mode 100644 index 0000000000..2270e08e2c --- /dev/null +++ kafka-handler/src/java/org/apache/hadoop/hive/kafka/HiveKafkaProducer.java @@ -0,0 +1,256 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.hadoop.hive.kafka; + +import com.google.common.base.Preconditions; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.clients.producer.Callback; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.Producer; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.clients.producer.RecordMetadata; +import org.apache.kafka.clients.producer.internals.TransactionalRequestResult; +import org.apache.kafka.common.Metric; +import org.apache.kafka.common.MetricName; +import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.ProducerFencedException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; +import java.lang.reflect.Field; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; + +/** + * Kafka Producer with public methods to extract the producer state then resuming transaction in another process. + * This Producer is to be used only if you need to extract the transaction state and resume it from a different process. + * Class is mostly taken from Apache Flink Project: + * org.apache.flink.streaming.connectors.kafka.internal.FlinkKafkaProducer + * + * @param key serializer class. + * @param value serializer class. + */ +class HiveKafkaProducer implements Producer { + private static final Logger LOG = LoggerFactory.getLogger(HiveKafkaProducer.class); + + private final KafkaProducer kafkaProducer; + + @Nullable private final String transactionalId; + + HiveKafkaProducer(Properties properties) { + transactionalId = properties.getProperty(ProducerConfig.TRANSACTIONAL_ID_CONFIG); + kafkaProducer = new KafkaProducer<>(properties); + } + + @Override public void initTransactions() { + kafkaProducer.initTransactions(); + } + + @Override public void beginTransaction() throws ProducerFencedException { + kafkaProducer.beginTransaction(); + } + + @Override public void commitTransaction() throws ProducerFencedException { + kafkaProducer.commitTransaction(); + } + + @Override public void abortTransaction() throws ProducerFencedException { + kafkaProducer.abortTransaction(); + } + + @Override public void sendOffsetsToTransaction(Map offsets, String consumerGroupId) + throws ProducerFencedException { + kafkaProducer.sendOffsetsToTransaction(offsets, consumerGroupId); + } + + @Override public Future send(ProducerRecord record) { + return kafkaProducer.send(record); + } + + @Override public Future send(ProducerRecord record, Callback callback) { + return kafkaProducer.send(record, callback); + } + + @Override public List partitionsFor(String topic) { + return kafkaProducer.partitionsFor(topic); + } + + @Override public Map metrics() { + return kafkaProducer.metrics(); + } + + @Override public void close() { + kafkaProducer.close(); + } + + @Override public void close(long timeout, TimeUnit unit) { + kafkaProducer.close(timeout, unit); + } + + @Override public void flush() { + kafkaProducer.flush(); + if (transactionalId != null) { + flushNewPartitions(); + } + } + + /** + * Instead of obtaining producerId and epoch from the transaction coordinator, re-use previously obtained ones, + * so that we can resume transaction after a restart. Implementation of this method is based on + * {@link org.apache.kafka.clients.producer.KafkaProducer#initTransactions}. + */ + synchronized void resumeTransaction(long producerId, short epoch) { + Preconditions.checkState(producerId >= 0 && epoch >= 0, + "Incorrect values for producerId {} and epoch {}", + producerId, + epoch); + LOG.info("Attempting to resume transaction {} with producerId {} and epoch {}", transactionalId, producerId, epoch); + + Object transactionManager = getValue(kafkaProducer, "transactionManager"); + + Object nextSequence = getValue(transactionManager, "nextSequence"); + Object lastAckedSequence = getValue(transactionManager, "lastAckedSequence"); + + invoke(transactionManager, + "transitionTo", + getEnum("org.apache.kafka.clients.producer.internals.TransactionManager$State.INITIALIZING")); + invoke(nextSequence, "clear"); + invoke(lastAckedSequence, "clear"); + + Object producerIdAndEpoch = getValue(transactionManager, "producerIdAndEpoch"); + setValue(producerIdAndEpoch, "producerId", producerId); + setValue(producerIdAndEpoch, "epoch", epoch); + + invoke(transactionManager, + "transitionTo", + getEnum("org.apache.kafka.clients.producer.internals.TransactionManager$State.READY")); + + invoke(transactionManager, + "transitionTo", + getEnum("org.apache.kafka.clients.producer.internals.TransactionManager$State.IN_TRANSACTION")); + setValue(transactionManager, "transactionStarted", true); + } + + @Nullable String getTransactionalId() { + return transactionalId; + } + + long getProducerId() { + Object transactionManager = getValue(kafkaProducer, "transactionManager"); + Object producerIdAndEpoch = getValue(transactionManager, "producerIdAndEpoch"); + return (long) getValue(producerIdAndEpoch, "producerId"); + } + + short getEpoch() { + Object transactionManager = getValue(kafkaProducer, "transactionManager"); + Object producerIdAndEpoch = getValue(transactionManager, "producerIdAndEpoch"); + return (short) getValue(producerIdAndEpoch, "epoch"); + } + + /** + * Besides committing {@link org.apache.kafka.clients.producer.KafkaProducer#commitTransaction} is also adding new + * partitions to the transaction. flushNewPartitions method is moving this logic to pre-commit/flush, to make + * resumeTransaction simpler. + * Otherwise resumeTransaction would require to restore state of the not yet added/"in-flight" partitions. + */ + private void flushNewPartitions() { + LOG.info("Flushing new partitions"); + TransactionalRequestResult result = enqueueNewPartitions(); + Object sender = getValue(kafkaProducer, "sender"); + invoke(sender, "wakeup"); + result.await(); + } + + private synchronized TransactionalRequestResult enqueueNewPartitions() { + Object transactionManager = getValue(kafkaProducer, "transactionManager"); + Object txnRequestHandler = invoke(transactionManager, "addPartitionsToTransactionHandler"); + invoke(transactionManager, + "enqueueRequest", + new Class[] {txnRequestHandler.getClass().getSuperclass()}, + new Object[] {txnRequestHandler}); + return (TransactionalRequestResult) getValue(txnRequestHandler, + txnRequestHandler.getClass().getSuperclass(), + "result"); + } + + @SuppressWarnings("unchecked") private static Enum getEnum(String enumFullName) { + @SuppressWarnings("RegExpRedundantEscape") String[] x = enumFullName.split("\\.(?=[^\\.]+$)"); + if (x.length == 2) { + String enumClassName = x[0]; + String enumName = x[1]; + try { + Class cl = (Class) Class.forName(enumClassName); + return Enum.valueOf(cl, enumName); + } catch (ClassNotFoundException e) { + throw new RuntimeException("Incompatible KafkaProducer version", e); + } + } + return null; + } + + private static Object invoke(Object object, String methodName, Object... args) { + Class[] argTypes = new Class[args.length]; + for (int i = 0; i < args.length; i++) { + argTypes[i] = args[i].getClass(); + } + return invoke(object, methodName, argTypes, args); + } + + private static Object invoke(Object object, String methodName, Class[] argTypes, Object[] args) { + try { + Method method = object.getClass().getDeclaredMethod(methodName, argTypes); + method.setAccessible(true); + return method.invoke(object, args); + } catch (NoSuchMethodException | InvocationTargetException | IllegalAccessException e) { + throw new RuntimeException("Incompatible KafkaProducer version", e); + } + } + + private static Object getValue(Object object, String fieldName) { + return getValue(object, object.getClass(), fieldName); + } + + private static Object getValue(Object object, Class clazz, String fieldName) { + try { + Field field = clazz.getDeclaredField(fieldName); + field.setAccessible(true); + return field.get(object); + } catch (NoSuchFieldException | IllegalAccessException e) { + throw new RuntimeException("Incompatible KafkaProducer version", e); + } + } + + private static void setValue(Object object, String fieldName, Object value) { + try { + Field field = object.getClass().getDeclaredField(fieldName); + field.setAccessible(true); + field.set(object, value); + } catch (NoSuchFieldException | IllegalAccessException e) { + throw new RuntimeException("Incompatible KafkaProducer version", e); + } + } +} diff --git kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaPullerInputFormat.java kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaInputFormat.java similarity index 80% rename from kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaPullerInputFormat.java rename to kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaInputFormat.java index 2d5637d430..c401df9850 100644 --- kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaPullerInputFormat.java +++ kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaInputFormat.java @@ -55,13 +55,13 @@ * The input split will contain the set of topic partition and start/end offsets. * Records will be returned as bytes array. */ -public class KafkaPullerInputFormat extends InputFormat - implements org.apache.hadoop.mapred.InputFormat { +@SuppressWarnings("WeakerAccess") public class KafkaInputFormat extends InputFormat + implements org.apache.hadoop.mapred.InputFormat { - private static final Logger LOG = LoggerFactory.getLogger(KafkaPullerInputFormat.class); + private static final Logger LOG = LoggerFactory.getLogger(KafkaInputFormat.class); @Override public InputSplit[] getSplits(JobConf jobConf, int i) throws IOException { - List inputSplits; + List inputSplits; try { inputSplits = computeSplits(jobConf); } catch (InterruptedException e) { @@ -83,14 +83,19 @@ * * @return full scan input split collection based on Kafka metadata APIs */ - private static List buildFullScanFromKafka(String topic, + private static List buildFullScanFromKafka(String topic, KafkaConsumer consumer, - Path[] tablePaths) { + Path[] tablePaths, int maxTries) { final Map starOffsetsMap; final Map endOffsetsMap; final List topicPartitions; - topicPartitions = fetchTopicPartitions(topic, consumer); + RetryUtils.Task> fetchTPTask = () -> fetchTopicPartitions(topic, consumer); + try { + topicPartitions = RetryUtils.retry(fetchTPTask, (error) -> !KafkaUtils.exceptionIsFatal(error), maxTries); + } catch (Exception e) { + throw new RuntimeException(e); + } starOffsetsMap = consumer.beginningOffsets(topicPartitions); endOffsetsMap = consumer.endOffsets(topicPartitions); @@ -101,7 +106,7 @@ endOffsetsMap.forEach((tp, end) -> LOG.info("TPartition [{}],End offsets [{}]", tp, end)); } return topicPartitions.stream() - .map(topicPartition -> new KafkaPullerInputSplit(topicPartition.topic(), + .map(topicPartition -> new KafkaInputSplit(topicPartition.topic(), topicPartition.partition(), starOffsetsMap.get(topicPartition), endOffsetsMap.get(topicPartition), @@ -109,25 +114,23 @@ .collect(Collectors.toList()); } - private List computeSplits(Configuration configuration) + private List computeSplits(Configuration configuration) throws IOException, InterruptedException { - // this will be used to harness some KAFKA blocking calls + // ExecutorService is used to harness some KAFKA blocking calls and interrupt after some duration final ExecutorService execService = Executors.newSingleThreadExecutor(); - try (KafkaConsumer consumer = new KafkaConsumer(KafkaStreamingUtils.consumerProperties(configuration))) { - final String topic = configuration.get(KafkaStreamingUtils.HIVE_KAFKA_TOPIC); - final long - timeoutMs = - configuration.getLong(KafkaStreamingUtils.HIVE_KAFKA_POLL_TIMEOUT, - KafkaStreamingUtils.DEFAULT_CONSUMER_POLL_TIMEOUT_MS); + + try (KafkaConsumer consumer = new KafkaConsumer(KafkaUtils.consumerProperties(configuration))) { + final String topic = configuration.get(KafkaTableProperties.HIVE_KAFKA_TOPIC.getName()); + final long timeoutMs = configuration.getLong(KafkaTableProperties.KAFKA_FETCH_METADATA_TIMEOUT.getName(), -1); + final int maxTries = configuration.getInt(KafkaTableProperties.MAX_RETRIES.getName(), -1); // hive depends on FileSplits JobConf jobConf = new JobConf(configuration); Path[] tablePaths = org.apache.hadoop.mapred.FileInputFormat.getInputPaths(jobConf); + final Future> futureFullHouse; //noinspection unchecked - Future> - futureFullHouse = - execService.submit(() -> buildFullScanFromKafka(topic, consumer, tablePaths)); - List fullHouse; + futureFullHouse = execService.submit(() -> buildFullScanFromKafka(topic, consumer, tablePaths, maxTries)); + final List fullHouse; try { fullHouse = futureFullHouse.get(timeoutMs, TimeUnit.MILLISECONDS); } catch (TimeoutException | ExecutionException e) { @@ -137,7 +140,7 @@ throw new IOException(e); } - @SuppressWarnings("unchecked") final ImmutableMap.Builder + @SuppressWarnings("unchecked") final ImmutableMap.Builder fullHouseMapBuilder = new ImmutableMap.Builder(); fullHouse.forEach(input -> fullHouseMapBuilder.put(new TopicPartition(input.getTopic(), input.getPartition()), @@ -149,14 +152,14 @@ if (filterExprSerialized != null && !filterExprSerialized.isEmpty()) { ExprNodeGenericFuncDesc filterExpr = SerializationUtilities.deserializeExpression(filterExprSerialized); LOG.info("Kafka trimmer working on Filter tree {}", filterExpr.getExprString()); - Callable> + Callable> trimmerWorker = () -> kafkaScanTrimmer.computeOptimizedScan(filterExpr) .entrySet() .stream() .map(Map.Entry::getValue) .collect(Collectors.toList()); - Future> futureTinyHouse = execService.submit(trimmerWorker); + Future> futureTinyHouse = execService.submit(trimmerWorker); try { return futureTinyHouse.get(timeoutMs, TimeUnit.MILLISECONDS) .stream() @@ -184,10 +187,10 @@ return partitions.stream().map(p -> new TopicPartition(topic, p.partition())).collect(Collectors.toList()); } - @Override public RecordReader getRecordReader(InputSplit inputSplit, + @Override public RecordReader getRecordReader(InputSplit inputSplit, JobConf jobConf, Reporter reporter) { - return new KafkaPullerRecordReader((KafkaPullerInputSplit) inputSplit, jobConf); + return new KafkaRecordReader((KafkaInputSplit) inputSplit, jobConf); } @Override public List getSplits(JobContext jobContext) @@ -197,9 +200,9 @@ .collect(Collectors.toList()); } - @Override public org.apache.hadoop.mapreduce.RecordReader createRecordReader( + @Override public org.apache.hadoop.mapreduce.RecordReader createRecordReader( org.apache.hadoop.mapreduce.InputSplit inputSplit, TaskAttemptContext taskAttemptContext) { - return new KafkaPullerRecordReader(); + return new KafkaRecordReader(); } } diff --git kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaPullerInputSplit.java kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaInputSplit.java similarity index 79% rename from kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaPullerInputSplit.java rename to kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaInputSplit.java index 697469c9e0..cb1f4df1f5 100644 --- kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaPullerInputSplit.java +++ kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaInputSplit.java @@ -34,18 +34,18 @@ /** * Kafka Hadoop Input Split Class. */ -@SuppressWarnings("WeakerAccess") public class KafkaPullerInputSplit extends FileSplit +@SuppressWarnings("WeakerAccess") public class KafkaInputSplit extends FileSplit implements org.apache.hadoop.mapred.InputSplit { private String topic; private long startOffset; private int partition; private long endOffset; - public KafkaPullerInputSplit() { + public KafkaInputSplit() { super(null, 0, 0, (String[]) null); } - public KafkaPullerInputSplit(String topic, int partition, long startOffset, long endOffset, Path dummyPath) { + public KafkaInputSplit(String topic, int partition, long startOffset, long endOffset, Path dummyPath) { super(dummyPath, 0, 0, (String[]) null); this.topic = topic; this.startOffset = startOffset; @@ -109,8 +109,8 @@ public long getEndOffset() { * * @return new split that represents range intersection or null if it is not overlapping */ - @Nullable public static KafkaPullerInputSplit intersectRange(KafkaPullerInputSplit split1, - KafkaPullerInputSplit split2) { + @Nullable public static KafkaInputSplit intersectRange(KafkaInputSplit split1, + KafkaInputSplit split2) { assert (split1.topic.equals(split2.topic)); assert (split1.partition == split2.partition); final long startOffset = Math.max(split1.getStartOffset(), split2.getStartOffset()); @@ -119,7 +119,7 @@ public long getEndOffset() { // there is no overlapping return null; } - return new KafkaPullerInputSplit(split1.topic, split1.partition, startOffset, endOffset, split1.getPath()); + return new KafkaInputSplit(split1.topic, split1.partition, startOffset, endOffset, split1.getPath()); } /** @@ -130,22 +130,22 @@ public long getEndOffset() { * * @return new split with a range including both splits. */ - public static KafkaPullerInputSplit unionRange(KafkaPullerInputSplit split1, KafkaPullerInputSplit split2) { + public static KafkaInputSplit unionRange(KafkaInputSplit split1, KafkaInputSplit split2) { assert (split1.topic.equals(split2.topic)); assert (split1.partition == split2.partition); final long startOffset = Math.min(split1.getStartOffset(), split2.getStartOffset()); final long endOffset = Math.max(split1.getEndOffset(), split2.getEndOffset()); - return new KafkaPullerInputSplit(split1.topic, split1.partition, startOffset, endOffset, split1.getPath()); + return new KafkaInputSplit(split1.topic, split1.partition, startOffset, endOffset, split1.getPath()); } @Override public boolean equals(Object o) { if (this == o) { return true; } - if (!(o instanceof KafkaPullerInputSplit)) { + if (!(o instanceof KafkaInputSplit)) { return false; } - KafkaPullerInputSplit that = (KafkaPullerInputSplit) o; + KafkaInputSplit that = (KafkaInputSplit) o; return Objects.equal(getTopic(), that.getTopic()) && Objects.equal(getStartOffset(), that.getStartOffset()) && Objects.equal(getPartition(), that.getPartition()) @@ -157,7 +157,7 @@ public static KafkaPullerInputSplit unionRange(KafkaPullerInputSplit split1, Kaf } @Override public String toString() { - return "KafkaPullerInputSplit{" + return "KafkaInputSplit{" + "topic='" + topic + '\'' @@ -172,24 +172,20 @@ public static KafkaPullerInputSplit unionRange(KafkaPullerInputSplit split1, Kaf + '}'; } - public static KafkaPullerInputSplit copyOf(KafkaPullerInputSplit other) { - return new KafkaPullerInputSplit(other.getTopic(), + public static KafkaInputSplit copyOf(KafkaInputSplit other) { + return new KafkaInputSplit(other.getTopic(), other.getPartition(), other.getStartOffset(), other.getEndOffset(), other.getPath()); } - @SuppressWarnings("MethodDoesntCallSuperMethod") public KafkaPullerInputSplit clone() { - return copyOf(this); - } - - public static List slice(long sliceSize, final KafkaPullerInputSplit split) { + public static List slice(long sliceSize, final KafkaInputSplit split) { if (split.getEndOffset() - split.getStartOffset() > sliceSize) { - ImmutableList.Builder builder = ImmutableList.builder(); + ImmutableList.Builder builder = ImmutableList.builder(); long start = split.getStartOffset(); while (start < split.getEndOffset() - sliceSize) { - builder.add(new KafkaPullerInputSplit(split.topic, + builder.add(new KafkaInputSplit(split.topic, split.partition, start, start + sliceSize + 1, @@ -198,7 +194,7 @@ public static KafkaPullerInputSplit copyOf(KafkaPullerInputSplit other) { } // last split if (start < split.getEndOffset()) { - builder.add(new KafkaPullerInputSplit(split.topic, + builder.add(new KafkaInputSplit(split.topic, split.partition, start, split.getEndOffset(), diff --git kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaJsonSerDe.java kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaJsonSerDe.java index 5f0143de09..228225cc2b 100644 --- kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaJsonSerDe.java +++ kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaJsonSerDe.java @@ -78,15 +78,14 @@ * Basic JsonSerDe to make use of such storage handler smooth and easy and testing basic primitive Json. * For production please use Hive native JsonSerde. */ -public class KafkaJsonSerDe extends AbstractSerDe { +@SuppressWarnings("unused") class KafkaJsonSerDe extends AbstractSerDe { private static final Logger LOG = LoggerFactory.getLogger(KafkaJsonSerDe.class); private static final ThreadLocal TS_PARSER = ThreadLocal.withInitial(KafkaJsonSerDe::createAutoParser); - private static final Function - typeInfoToObjectInspector = - typeInfo -> PrimitiveObjectInspectorFactory.getPrimitiveWritableObjectInspector(TypeInfoFactory.getPrimitiveTypeInfo( - typeInfo.getTypeName())); + private static final Function TYPEINFO_TO_OI = + typeInfo -> PrimitiveObjectInspectorFactory.getPrimitiveWritableObjectInspector( + TypeInfoFactory.getPrimitiveTypeInfo(typeInfo.getTypeName())); private List columnNames; private List columnTypes; private ObjectInspector inspector; @@ -118,7 +117,7 @@ LOG.debug("types: {}, {} ", columnTypeProperty, columnTypes); } - inspectors = columnTypes.stream().map(typeInfoToObjectInspector).collect(Collectors.toList()); + inspectors = columnTypes.stream().map(TYPEINFO_TO_OI).collect(Collectors.toList()); inspector = ObjectInspectorFactory.getStandardStructObjectInspector(columnNames, inspectors); } @@ -236,8 +235,8 @@ private static DateTimeFormatter createAutoParser() { DateTimeParser timeOrOffset = new DateTimeFormatterBuilder().append(null, - new DateTimeParser[] { new DateTimeFormatterBuilder().appendLiteral('T').toParser(), - new DateTimeFormatterBuilder().appendLiteral(' ').toParser() }) + new DateTimeParser[] {new DateTimeFormatterBuilder().appendLiteral('T').toParser(), + new DateTimeFormatterBuilder().appendLiteral(' ').toParser()}) .appendOptional(ISODateTimeFormat.timeElementParser().getParser()) .appendOptional(offsetElement.getParser()) .toParser(); diff --git kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaOutputFormat.java kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaOutputFormat.java new file mode 100644 index 0000000000..950f7315c2 --- /dev/null +++ kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaOutputFormat.java @@ -0,0 +1,117 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.hadoop.hive.kafka; + +import com.google.common.base.Preconditions; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.ql.exec.FileSinkOperator; +import org.apache.hadoop.hive.ql.exec.Utilities; +import org.apache.hadoop.hive.ql.io.HiveOutputFormat; +import org.apache.hadoop.io.NullWritable; +import org.apache.hadoop.io.Writable; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapred.RecordWriter; +import org.apache.hadoop.util.Progressable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.Properties; + +/** + * Kafka Hive Output Format class used to write Hive Rows to a Kafka Queue. + */ +public class KafkaOutputFormat implements HiveOutputFormat { + private static final Logger LOG = LoggerFactory.getLogger(KafkaOutputFormat.class); + + @Override public FileSinkOperator.RecordWriter getHiveRecordWriter(JobConf jc, + Path finalOutPath, + Class valueClass, + boolean isCompressed, + Properties tableProperties, + Progressable progress) { + final String topic = jc.get(KafkaTableProperties.HIVE_KAFKA_TOPIC.getName()); + final Boolean optimisticCommit = jc.getBoolean(KafkaTableProperties.HIVE_KAFKA_OPTIMISTIC_COMMIT.getName(), true); + final WriteSemantic + writeSemantic = + WriteSemantic.valueOf(jc.get(KafkaTableProperties.WRITE_SEMANTIC_PROPERTY.getName())); + final Properties producerProperties = KafkaUtils.producerProperties(jc); + final FileSinkOperator.RecordWriter recordWriter; + switch (writeSemantic) { + case BEST_EFFORT: + recordWriter = new SimpleKafkaWriter(topic, Utilities.getTaskId(jc), false, producerProperties); + break; + case AT_LEAST_ONCE: + recordWriter = new SimpleKafkaWriter(topic, Utilities.getTaskId(jc), true, producerProperties); + break; + case EXACTLY_ONCE: + FileSystem fs; + try { + fs = finalOutPath.getFileSystem(jc); + } catch (IOException e) { + LOG.error("Can not construct file system instance", e); + throw new RuntimeException(e); + } + final String queryId = Preconditions.checkNotNull(jc.get(HiveConf.ConfVars.HIVEQUERYID.varname, null)); + recordWriter = + new TransactionalKafkaWriter(topic, producerProperties, + new Path(Preconditions.checkNotNull(finalOutPath), queryId), + fs, + optimisticCommit); + break; + default: + throw new IllegalArgumentException(String.format("Unknown delivery semantic [%s]", writeSemantic.toString())); + } + return recordWriter; + } + + @Override public RecordWriter getRecordWriter(FileSystem fileSystem, + JobConf jobConf, + String s, + Progressable progressable) { + throw new RuntimeException("this is not suppose to be here"); + } + + @Override public void checkOutputSpecs(FileSystem fileSystem, JobConf jobConf) { + + } + + /** + * Possible write semantic supported by the Record Writer. + */ + enum WriteSemantic { + /** + * Best effort delivery with no guarantees at all, user can set Producer properties as they wish, + * will carry on when possible unless it is a fatal exception. + */ + BEST_EFFORT, + /** + * Deliver all the record at least once unless the job fails. + * Therefore duplicates can be introduced due to lost ACKs or Tasks retries. + * Currently this is the default. + */ + AT_LEAST_ONCE, + /** + * Deliver every record exactly once. + */ + EXACTLY_ONCE, + } +} diff --git kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaRecordIterator.java kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaRecordIterator.java index c252455348..2225f19a4d 100644 --- kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaRecordIterator.java +++ kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaRecordIterator.java @@ -25,24 +25,31 @@ import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.RetriableException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import javax.annotation.Nullable; +import java.time.Duration; import java.util.Collections; import java.util.Iterator; import java.util.List; import java.util.concurrent.TimeUnit; /** - * Iterator over Kafka Records to read records from a single topic partition inclusive start, exclusive end. + * This class implements an Iterator over a single Kafka topic partition. + * + * Notes: + * The user of this class has to provide a functional Kafka Consumer and then has to clean it afterward. + * The user of this class is responsible for thread safety if the provided consumer is shared across threads. + * */ -public class KafkaRecordIterator implements Iterator> { +class KafkaRecordIterator implements Iterator> { private static final Logger LOG = LoggerFactory.getLogger(KafkaRecordIterator.class); private static final String POLL_TIMEOUT_HINT = String.format("Try increasing poll timeout using Hive Table property [%s]", - KafkaStreamingUtils.HIVE_KAFKA_POLL_TIMEOUT); + KafkaTableProperties.KAFKA_POLL_TIMEOUT.getName()); private static final String ERROR_POLL_TIMEOUT_FORMAT = "Consumer returned [0] record due to exhausted poll timeout [%s]ms from TopicPartition:[%s] " @@ -54,30 +61,38 @@ private final long endOffset; private final long startOffset; private final long pollTimeoutMs; + private final Duration pollTimeoutDurationMs; private final Stopwatch stopwatch = Stopwatch.createUnstarted(); private ConsumerRecords records; + /** + * Holds the kafka consumer position after the last poll() call. + */ private long consumerPosition; private ConsumerRecord nextRecord; private boolean hasMore = true; + /** + * On each Kafka Consumer poll() call we get a batch of records, this Iterator will be used to loop over it. + */ private Iterator> consumerRecordIterator = null; /** - * Iterator over Kafka Records over a single {@code topicPartition} inclusive {@code startOffset}, - * up to exclusive {@code endOffset}. - *

- * If {@code requestedStartOffset} is not null will seek up to that offset - * Else If {@code requestedStartOffset} is null will seek to beginning see - * {@link org.apache.kafka.clients.consumer.Consumer#seekToBeginning(java.util.Collection)} - *

- * When provided with {@code requestedEndOffset}, will return records up to consumer position == endOffset - * Else If {@code requestedEndOffset} is null it will read up to the current end of the stream - * {@link org.apache.kafka.clients.consumer.Consumer#seekToEnd(java.util.Collection)} - *

- * @param consumer functional kafka consumer. - * @param topicPartition kafka topic partition. - * @param requestedStartOffset requested start position. - * @param requestedEndOffset requested end position. If null will read up to current last - * @param pollTimeoutMs poll time out in ms. + * Kafka record Iterator pulling from a single {@code topicPartition} an inclusive {@code requestedStartOffset}, + * up to exclusive {@code requestedEndOffset}. + * + * This iterator can block on polling up to a designated timeout. + * + * If no record is returned by brokers after poll timeout duration such case will be considered as an exception. + * Although the timeout exception it is a retryable exception, therefore users of this class can retry if needed. + * + * @param consumer Functional kafka consumer, user must initialize this and close it. + * @param topicPartition Target Kafka topic partition. + * @param requestedStartOffset Requested start offset position, if NULL iterator will seek to beginning using: + * {@link Consumer#seekToBeginning(java.util.Collection)}. + * + * @param requestedEndOffset Requested end position. If null will read up to last available offset, + * such position is given by: + * {@link Consumer#seekToEnd(java.util.Collection)}. + * @param pollTimeoutMs positive number indicating poll time out in ms. */ KafkaRecordIterator(Consumer consumer, TopicPartition topicPartition, @@ -87,6 +102,7 @@ this.consumer = Preconditions.checkNotNull(consumer, "Consumer can not be null"); this.topicPartition = Preconditions.checkNotNull(topicPartition, "Topic partition can not be null"); this.pollTimeoutMs = pollTimeoutMs; + this.pollTimeoutDurationMs = Duration.ofMillis(pollTimeoutMs); Preconditions.checkState(this.pollTimeoutMs > 0, "Poll timeout has to be positive number"); final List topicPartitionList = Collections.singletonList(topicPartition); // assign topic partition to consumer @@ -138,7 +154,12 @@ } /** - * @throws IllegalStateException if the kafka consumer poll call can not reach the target offset. + * Check if there is more records to be consumed and pull more from the broker if current batch of record is empty. + * This method might block up to {@link this#pollTimeoutMs} to pull records from Kafka Broker. + * + * @throws PollTimeoutException if poll returns 0 record and consumer position did not reach requested endOffset. + * Such an exception is a retryable exception, and it can be a transient exception that if retried may succeed. + * * @return true if has more records to be consumed. */ @Override public boolean hasNext() { @@ -158,20 +179,20 @@ Need to poll at least one more record (consumerPosition < endOffset) AND consume /** * Poll more records from the Kafka Broker. * - * @throws IllegalStateException if no records returned before consumer position reaches target end offset. + * @throws PollTimeoutException if poll returns 0 record and consumer's position < requested endOffset. */ private void pollRecords() { if (LOG.isTraceEnabled()) { stopwatch.reset().start(); } - records = consumer.poll(pollTimeoutMs); + records = consumer.poll(pollTimeoutDurationMs); if (LOG.isTraceEnabled()) { stopwatch.stop(); LOG.trace("Pulled [{}] records in [{}] ms", records.count(), stopwatch.elapsed(TimeUnit.MILLISECONDS)); } // Fail if we can not poll within one lap of pollTimeoutMs. if (records.isEmpty() && consumer.position(topicPartition) < endOffset) { - throw new IllegalStateException(String.format(ERROR_POLL_TIMEOUT_FORMAT, + throw new PollTimeoutException(String.format(ERROR_POLL_TIMEOUT_FORMAT, pollTimeoutMs, topicPartition.toString(), startOffset, @@ -201,4 +222,12 @@ private void findNext() { nextRecord = null; } } + + static final class PollTimeoutException extends RetriableException { + private static final long serialVersionUID = 1L; + + PollTimeoutException(String message) { + super(message); + } + } } diff --git kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaPullerRecordReader.java kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaRecordReader.java similarity index 81% rename from kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaPullerRecordReader.java rename to kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaRecordReader.java index 06a10b4b8b..746de61273 100644 --- kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaPullerRecordReader.java +++ kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaRecordReader.java @@ -36,14 +36,14 @@ /** * Kafka Records Reader implementation. */ -@SuppressWarnings("UnstableApiUsage") public class KafkaPullerRecordReader extends RecordReader - implements org.apache.hadoop.mapred.RecordReader { +@SuppressWarnings("WeakerAccess") public class KafkaRecordReader extends RecordReader + implements org.apache.hadoop.mapred.RecordReader { - private static final Logger LOG = LoggerFactory.getLogger(KafkaPullerRecordReader.class); + private static final Logger LOG = LoggerFactory.getLogger(KafkaRecordReader.class); private KafkaConsumer consumer = null; private Configuration config = null; - private KafkaRecordWritable currentWritableValue; + private KafkaWritable currentWritableValue; private Iterator> recordsCursor = null; private long totalNumberRecords = 0L; @@ -53,13 +53,13 @@ private long startOffset = -1L; private long endOffset = Long.MAX_VALUE; - @SuppressWarnings("WeakerAccess") public KafkaPullerRecordReader() { + @SuppressWarnings("WeakerAccess") public KafkaRecordReader() { } private void initConsumer() { if (consumer == null) { LOG.info("Initializing Kafka Consumer"); - final Properties properties = KafkaStreamingUtils.consumerProperties(config); + final Properties properties = KafkaUtils.consumerProperties(config); String brokerString = properties.getProperty(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG); Preconditions.checkNotNull(brokerString, "broker end point can not be null"); LOG.info("Starting Consumer with Kafka broker string [{}]", brokerString); @@ -67,25 +67,26 @@ private void initConsumer() { } } - @SuppressWarnings("WeakerAccess") public KafkaPullerRecordReader(KafkaPullerInputSplit inputSplit, + @SuppressWarnings("WeakerAccess") public KafkaRecordReader(KafkaInputSplit inputSplit, Configuration jobConf) { initialize(inputSplit, jobConf); } - private synchronized void initialize(KafkaPullerInputSplit inputSplit, Configuration jobConf) { + private synchronized void initialize(KafkaInputSplit inputSplit, Configuration jobConf) { if (!started) { this.config = jobConf; startOffset = inputSplit.getStartOffset(); endOffset = inputSplit.getEndOffset(); TopicPartition topicPartition = new TopicPartition(inputSplit.getTopic(), inputSplit.getPartition()); Preconditions.checkState(startOffset >= 0 && startOffset <= endOffset, - "Start [%s] has to be positive and less or equal than End [%s]", startOffset, endOffset); + "Start [%s] has to be positive and less or equal than End [%s]", + startOffset, + endOffset); totalNumberRecords += endOffset - startOffset; initConsumer(); long pollTimeout = - config.getLong(KafkaStreamingUtils.HIVE_KAFKA_POLL_TIMEOUT, - KafkaStreamingUtils.DEFAULT_CONSUMER_POLL_TIMEOUT_MS); + config.getLong(KafkaTableProperties.KAFKA_POLL_TIMEOUT.getName(), -1); LOG.debug("Consumer poll timeout [{}] ms", pollTimeout); this.recordsCursor = startOffset == endOffset ? @@ -95,12 +96,11 @@ private synchronized void initialize(KafkaPullerInputSplit inputSplit, Configura } } - @Override public void initialize(org.apache.hadoop.mapreduce.InputSplit inputSplit, - TaskAttemptContext context) { - initialize((KafkaPullerInputSplit) inputSplit, context.getConfiguration()); + @Override public void initialize(org.apache.hadoop.mapreduce.InputSplit inputSplit, TaskAttemptContext context) { + initialize((KafkaInputSplit) inputSplit, context.getConfiguration()); } - @Override public boolean next(NullWritable nullWritable, KafkaRecordWritable bytesWritable) { + @Override public boolean next(NullWritable nullWritable, KafkaWritable bytesWritable) { if (started && recordsCursor.hasNext()) { ConsumerRecord record = recordsCursor.next(); bytesWritable.set(record, startOffset, endOffset); @@ -115,8 +115,8 @@ private synchronized void initialize(KafkaPullerInputSplit inputSplit, Configura return NullWritable.get(); } - @Override public KafkaRecordWritable createValue() { - return new KafkaRecordWritable(); + @Override public KafkaWritable createValue() { + return new KafkaWritable(); } @Override public long getPos() { @@ -124,7 +124,7 @@ private synchronized void initialize(KafkaPullerInputSplit inputSplit, Configura } @Override public boolean nextKeyValue() { - currentWritableValue = new KafkaRecordWritable(); + currentWritableValue = new KafkaWritable(); if (next(NullWritable.get(), currentWritableValue)) { return true; } @@ -136,7 +136,7 @@ private synchronized void initialize(KafkaPullerInputSplit inputSplit, Configura return NullWritable.get(); } - @Override public KafkaRecordWritable getCurrentValue() { + @Override public KafkaWritable getCurrentValue() { return Preconditions.checkNotNull(currentWritableValue); } diff --git kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaScanTrimmer.java kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaScanTrimmer.java index 8fbdfdab11..256796a9a4 100644 --- kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaScanTrimmer.java +++ kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaScanTrimmer.java @@ -63,14 +63,14 @@ */ class KafkaScanTrimmer { private static final Logger LOG = LoggerFactory.getLogger(KafkaScanTrimmer.class); - private final Map fullHouse; + private final Map fullHouse; private final KafkaConsumer kafkaConsumer; /** * @param fullHouse initial full scan to be pruned, this is a map of Topic partition to input split. * @param kafkaConsumer kafka consumer used to pull offsets for time filter if needed */ - KafkaScanTrimmer(Map fullHouse, KafkaConsumer kafkaConsumer) { + KafkaScanTrimmer(Map fullHouse, KafkaConsumer kafkaConsumer) { this.fullHouse = fullHouse; this.kafkaConsumer = kafkaConsumer; } @@ -83,8 +83,8 @@ * * @return tiny house of of the full house based on filter expression */ - Map computeOptimizedScan(ExprNodeGenericFuncDesc filterExpression) { - Map optimizedScan = parseAndOptimize(filterExpression); + Map computeOptimizedScan(ExprNodeGenericFuncDesc filterExpression) { + Map optimizedScan = parseAndOptimize(filterExpression); if (LOG.isDebugEnabled()) { if (optimizedScan != null) { @@ -113,7 +113,7 @@ * * @return Map of optimized kafka range scans or null if it is impossible to optimize. */ - @Nullable private Map parseAndOptimize(ExprNodeDesc expression) { + @Nullable private Map parseAndOptimize(ExprNodeDesc expression) { if (expression.getClass() != ExprNodeGenericFuncDesc.class) { return null; } @@ -154,7 +154,7 @@ * * @return leaf scan or null if can not figure out push down */ - @Nullable private Map pushLeaf(ExprNodeGenericFuncDesc expr, + @Nullable private Map pushLeaf(ExprNodeGenericFuncDesc expr, PredicateLeaf.Operator operator, boolean negation) { if (expr.getChildren().size() != 2) { @@ -192,8 +192,7 @@ constantDesc = (ExprNodeConstantDesc) extracted[0]; } - - if (columnDesc.getColumn().equals(KafkaStreamingUtils.MetadataColumn.PARTITION.getName())) { + if (columnDesc.getColumn().equals(MetadataColumn.PARTITION.getName())) { return buildScanFromPartitionPredicate(fullHouse, operator, ((Number) constantDesc.getValue()).intValue(), @@ -201,7 +200,7 @@ negation); } - if (columnDesc.getColumn().equals(KafkaStreamingUtils.MetadataColumn.OFFSET.getName())) { + if (columnDesc.getColumn().equals(MetadataColumn.OFFSET.getName())) { return buildScanFromOffsetPredicate(fullHouse, operator, ((Number) constantDesc.getValue()).longValue(), @@ -209,7 +208,7 @@ negation); } - if (columnDesc.getColumn().equals(KafkaStreamingUtils.MetadataColumn.TIMESTAMP.getName())) { + if (columnDesc.getColumn().equals(MetadataColumn.TIMESTAMP.getName())) { long timestamp = ((Number) constantDesc.getValue()).longValue(); //noinspection unchecked return buildScanForTimesPredicate(fullHouse, operator, timestamp, flip, negation, kafkaConsumer); @@ -229,8 +228,8 @@ * @return filtered kafka scan */ - @VisibleForTesting static Map buildScanFromPartitionPredicate( - Map fullScan, + @VisibleForTesting static Map buildScanFromPartitionPredicate(Map fullScan, PredicateLeaf.Operator operator, int partitionConst, boolean flip, @@ -262,12 +261,12 @@ predicate = topicPartition -> true; } - ImmutableMap.Builder builder = ImmutableMap.builder(); + ImmutableMap.Builder builder = ImmutableMap.builder(); // Filter full scan based on predicate fullScan.entrySet() .stream() .filter(entry -> predicate.test(entry.getKey())) - .forEach(entry -> builder.put(entry.getKey(), entry.getValue().clone())); + .forEach(entry -> builder.put(entry.getKey(), KafkaInputSplit.copyOf(entry.getValue()))); return builder.build(); } @@ -280,8 +279,8 @@ * * @return optimized kafka scan */ - @VisibleForTesting static Map buildScanFromOffsetPredicate(Map fullScan, + @VisibleForTesting static Map buildScanFromOffsetPredicate(Map fullScan, PredicateLeaf.Operator operator, long offsetConst, boolean flip, @@ -320,54 +319,50 @@ endOffset = -1; } - final Map newScan = new HashMap<>(); + final Map newScan = new HashMap<>(); fullScan.forEach((tp, existingInputSplit) -> { - final KafkaPullerInputSplit newInputSplit; + final KafkaInputSplit newInputSplit; if (startOffset != -1 && endOffset == -1) { - newInputSplit = new KafkaPullerInputSplit(tp.topic(), + newInputSplit = new KafkaInputSplit(tp.topic(), tp.partition(), //if the user ask for start offset > max offset will replace with last offset Math.min(startOffset, existingInputSplit.getEndOffset()), existingInputSplit.getEndOffset(), existingInputSplit.getPath()); } else if (endOffset != -1 && startOffset == -1) { - newInputSplit = new KafkaPullerInputSplit(tp.topic(), tp.partition(), existingInputSplit.getStartOffset(), + newInputSplit = new KafkaInputSplit(tp.topic(), tp.partition(), existingInputSplit.getStartOffset(), //@TODO check this, if user ask for non existing end offset ignore it and position head on start // This can be an issue when doing ingestion from kafka into Hive, what happen if there is some gaps // Shall we fail the ingest or carry-on and ignore non existing offsets Math.max(endOffset, existingInputSplit.getStartOffset()), existingInputSplit.getPath()); } else if (endOffset == startOffset + 1) { if (startOffset < existingInputSplit.getStartOffset() || startOffset >= existingInputSplit.getEndOffset()) { - newInputSplit = new KafkaPullerInputSplit(tp.topic(), tp.partition(), + newInputSplit = new KafkaInputSplit(tp.topic(), tp.partition(), // non existing offset will be seeking last offset existingInputSplit.getEndOffset(), existingInputSplit.getEndOffset(), existingInputSplit.getPath()); } else { newInputSplit = - new KafkaPullerInputSplit(tp.topic(), - tp.partition(), - startOffset, - endOffset, - existingInputSplit.getPath()); + new KafkaInputSplit(tp.topic(), tp.partition(), startOffset, endOffset, existingInputSplit.getPath()); } } else { newInputSplit = - new KafkaPullerInputSplit(tp.topic(), + new KafkaInputSplit(tp.topic(), tp.partition(), existingInputSplit.getStartOffset(), existingInputSplit.getEndOffset(), existingInputSplit.getPath()); } - newScan.put(tp, KafkaPullerInputSplit.intersectRange(newInputSplit, existingInputSplit)); + newScan.put(tp, KafkaInputSplit.intersectRange(newInputSplit, existingInputSplit)); }); return newScan; } - @Nullable private static Map buildScanForTimesPredicate( - Map fullHouse, + @Nullable private static Map buildScanForTimesPredicate( + Map fullHouse, PredicateLeaf.Operator operator, long timestamp, boolean flip, @@ -385,11 +380,11 @@ // NULL will be returned for that partition If the message format version in a partition is before 0.10.0 Map offsetAndTimestamp = consumer.offsetsForTimes(timePartitionsMap); return Maps.toMap(fullHouse.keySet(), tp -> { - KafkaPullerInputSplit existing = fullHouse.get(tp); + KafkaInputSplit existing = fullHouse.get(tp); OffsetAndTimestamp foundOffsetAndTime = offsetAndTimestamp.get(tp); //Null in case filter doesn't match or field not existing ie old broker thus return empty scan. final long startOffset = foundOffsetAndTime == null ? existing.getEndOffset() : foundOffsetAndTime.offset(); - return new KafkaPullerInputSplit(Objects.requireNonNull(tp).topic(), + return new KafkaInputSplit(Objects.requireNonNull(tp).topic(), tp.partition(), startOffset, existing.getEndOffset(), @@ -410,21 +405,21 @@ * * @return either full scan or an optimized sub scan. */ - private Map pushAndOp(ExprNodeGenericFuncDesc expr) { - Map currentScan = new HashMap<>(); + private Map pushAndOp(ExprNodeGenericFuncDesc expr) { + Map currentScan = new HashMap<>(); - fullHouse.forEach((tp, input) -> currentScan.put(tp, KafkaPullerInputSplit.copyOf(input))); + fullHouse.forEach((tp, input) -> currentScan.put(tp, KafkaInputSplit.copyOf(input))); for (ExprNodeDesc child : expr.getChildren()) { - Map scan = parseAndOptimize(child); + Map scan = parseAndOptimize(child); if (scan != null) { Set currentKeys = ImmutableSet.copyOf(currentScan.keySet()); currentKeys.forEach(key -> { - KafkaPullerInputSplit newSplit = scan.get(key); - KafkaPullerInputSplit oldSplit = currentScan.get(key); + KafkaInputSplit newSplit = scan.get(key); + KafkaInputSplit oldSplit = currentScan.get(key); currentScan.remove(key); if (newSplit != null) { - KafkaPullerInputSplit intersectionSplit = KafkaPullerInputSplit.intersectRange(newSplit, oldSplit); + KafkaInputSplit intersectionSplit = KafkaInputSplit.intersectRange(newSplit, oldSplit); if (intersectionSplit != null) { currentScan.put(key, intersectionSplit); } @@ -436,18 +431,18 @@ return currentScan; } - @Nullable private Map pushOrOp(ExprNodeGenericFuncDesc expr) { - final Map currentScan = new HashMap<>(); + @Nullable private Map pushOrOp(ExprNodeGenericFuncDesc expr) { + final Map currentScan = new HashMap<>(); for (ExprNodeDesc child : expr.getChildren()) { - Map scan = parseAndOptimize(child); + Map scan = parseAndOptimize(child); if (scan == null) { // if any of the children is unknown bailout return null; } scan.forEach((tp, input) -> { - KafkaPullerInputSplit existingSplit = currentScan.get(tp); - currentScan.put(tp, KafkaPullerInputSplit.unionRange(input, existingSplit == null ? input : existingSplit)); + KafkaInputSplit existingSplit = currentScan.get(tp); + currentScan.put(tp, KafkaInputSplit.unionRange(input, existingSplit == null ? input : existingSplit)); }); } return currentScan; diff --git kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaSerDe.java kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaSerDe.java new file mode 100644 index 0000000000..51cfa24929 --- /dev/null +++ kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaSerDe.java @@ -0,0 +1,380 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.hadoop.hive.kafka; + +import com.google.common.base.Preconditions; +import com.google.common.base.Throwables; +import com.google.common.collect.Lists; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericDatumWriter; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.io.BinaryEncoder; +import org.apache.avro.io.DatumReader; +import org.apache.avro.io.DecoderFactory; +import org.apache.avro.io.EncoderFactory; +import org.apache.avro.specific.SpecificDatumReader; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.serde.serdeConstants; +import org.apache.hadoop.hive.serde2.AbstractSerDe; +import org.apache.hadoop.hive.serde2.SerDeException; +import org.apache.hadoop.hive.serde2.SerDeSpec; +import org.apache.hadoop.hive.serde2.SerDeStats; +import org.apache.hadoop.hive.serde2.avro.AvroGenericRecordWritable; +import org.apache.hadoop.hive.serde2.avro.AvroSerdeUtils; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory; +import org.apache.hadoop.hive.serde2.objectinspector.StructField; +import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorUtils; +import org.apache.hadoop.io.BytesWritable; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.io.Writable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.nio.charset.CharacterCodingException; +import java.nio.charset.Charset; +import java.rmi.server.UID; +import java.util.ArrayList; +import java.util.List; +import java.util.Properties; +import java.util.stream.Collectors; + +/** + * Generic Kafka Serde that allow user to delegate Serde to other class like Avro, + * Json or any class that supports {@link BytesWritable}. + * I the user which to implement their own serde all they need is to implement a serde that extend + * {@link org.apache.hadoop.hive.serde2.AbstractSerDe} and accept {@link BytesWritable} as value + */ +@SerDeSpec(schemaProps = { serdeConstants.LIST_COLUMNS, serdeConstants.LIST_COLUMN_TYPES }) public class KafkaSerDe + extends AbstractSerDe { + private static final Logger LOG = LoggerFactory.getLogger(KafkaSerDe.class); + + /** + * Delegate SerDe used to Serialize and DeSerialize data form/to Kafka. + */ + private AbstractSerDe delegateSerDe; + + /** + * Delegate Object Inspector used to Deserialize the row, this OI is constructed by the {@code delegateSerDe}. + */ + private StructObjectInspector delegateDeserializerOI; + + /** + * Delegate Object Inspector used to Serialize the row as byte array. + */ + private StructObjectInspector delegateSerializerOI; + + /** + * Object Inspector of original row plus metadata. + */ + private ObjectInspector objectInspector; + private final List columnNames = Lists.newArrayList(); + private BytesConverter bytesConverter; + + @Override public void initialize(@Nullable Configuration conf, Properties tbl) throws SerDeException { + //This method is called before {@link org.apache.hadoop.hive.kafka.KafkaStorageHandler.preCreateTable} + //Thus we need to default to org.apache.hadoop.hive.kafka.KafkaUtils.DEFAULT_PROPERTIES if any property is needed + final String + className = + tbl.getProperty(KafkaTableProperties.SERDE_CLASS_NAME.getName(), + KafkaTableProperties.SERDE_CLASS_NAME.getDefaultValue()); + delegateSerDe = KafkaUtils.createDelegate(className); + //noinspection deprecation + delegateSerDe.initialize(conf, tbl); + + if (!(delegateSerDe.getObjectInspector() instanceof StructObjectInspector)) { + throw new SerDeException("Was expecting Struct Object Inspector but have " + delegateSerDe.getObjectInspector() + .getClass() + .getName()); + } + delegateDeserializerOI = (StructObjectInspector) delegateSerDe.getObjectInspector(); + + // Build column names Order matters here + columnNames.addAll(delegateDeserializerOI.getAllStructFieldRefs() + .stream() + .map(StructField::getFieldName) + .collect(Collectors.toList())); + columnNames.addAll(MetadataColumn.KAFKA_METADATA_COLUMN_NAMES); + + final List inspectors = new ArrayList<>(columnNames.size()); + inspectors.addAll(delegateDeserializerOI.getAllStructFieldRefs() + .stream() + .map(StructField::getFieldObjectInspector) + .collect(Collectors.toList())); + inspectors.addAll(MetadataColumn.KAFKA_METADATA_INSPECTORS); + objectInspector = ObjectInspectorFactory.getStandardStructObjectInspector(columnNames, inspectors); + + // Setup Read and Write Path From/To Kafka + if (delegateSerDe.getSerializedClass() == Text.class) { + bytesConverter = new TextBytesConverter(); + } else if (delegateSerDe.getSerializedClass() == AvroGenericRecordWritable.class) { + String schemaFromProperty = tbl.getProperty(AvroSerdeUtils.AvroTableProperties.SCHEMA_LITERAL.getPropName(), ""); + Preconditions.checkArgument(!schemaFromProperty.isEmpty(), "Avro Schema is empty Can not go further"); + Schema schema = AvroSerdeUtils.getSchemaFor(schemaFromProperty); + LOG.debug("Building Avro Reader with schema {}", schemaFromProperty); + bytesConverter = new AvroBytesConverter(schema); + } else { + bytesConverter = new BytesWritableConverter(); + } + } + + @Override public Class getSerializedClass() { + return delegateSerDe.getSerializedClass(); + } + + @Override public Writable serialize(Object obj, ObjectInspector objInspector) throws SerDeException { + + if (!(objInspector instanceof StructObjectInspector)) { + throw new SerDeException("Object inspector has to be " + + StructObjectInspector.class.getName() + + " but got " + + objInspector.getClass().getName()); + } + StructObjectInspector structObjectInspector = (StructObjectInspector) objInspector; + List data = structObjectInspector.getStructFieldsDataAsList(obj); + if (delegateSerializerOI == null) { + //@TODO check if i can cache this if it is the same. + delegateSerializerOI = + new SubStructObjectInspector(structObjectInspector, data.size() - MetadataColumn.values().length); + } + // We always append the metadata columns to the end of the row. + final List row = data.subList(0, data.size() - MetadataColumn.values().length); + //@TODO @FIXME use column names instead of actual positions that can be hard to read and review + Object key = data.get(data.size() - MetadataColumn.KAFKA_METADATA_COLUMN_NAMES.size()); + Object partition = data.get(data.size() - MetadataColumn.KAFKA_METADATA_COLUMN_NAMES.size() + 1); + Object offset = data.get(data.size() - MetadataColumn.KAFKA_METADATA_COLUMN_NAMES.size() + 2); + Object timestamp = data.get(data.size() - MetadataColumn.KAFKA_METADATA_COLUMN_NAMES.size() + 3); + + if (PrimitiveObjectInspectorUtils.getLong(offset, MetadataColumn.OFFSET.getObjectInspector()) != -1) { + LOG.error("Can not insert values into `__offset` column, has to be [-1]"); + throw new SerDeException("Can not insert values into `__offset` column, has to be [-1]"); + } + + final byte[] + keyBytes = + key == null ? null : PrimitiveObjectInspectorFactory.writableBinaryObjectInspector.getPrimitiveJavaObject(key); + final long + recordTs = + timestamp == null ? + -1 : + PrimitiveObjectInspectorUtils.getLong(timestamp, MetadataColumn.TIMESTAMP.getObjectInspector()); + final int + recordPartition = + partition == null ? + -1 : + PrimitiveObjectInspectorUtils.getInt(partition, MetadataColumn.PARTITION.getObjectInspector()); + + //noinspection unchecked + return new KafkaWritable(recordPartition, + recordTs, + bytesConverter.getBytes(delegateSerDe.serialize(row, delegateSerializerOI)), + keyBytes); + } + + @Override public SerDeStats getSerDeStats() { + return delegateSerDe.getSerDeStats(); + } + + @Override public Object deserialize(Writable blob) throws SerDeException { + KafkaWritable record = (KafkaWritable) blob; + final Object row = delegateSerDe.deserialize(bytesConverter.getWritable(record.getValue())); + return columnNames.stream().map(name -> { + final MetadataColumn metadataColumn = MetadataColumn.forName(name); + if (metadataColumn != null) { + return record.getHiveWritable(metadataColumn); + } + return delegateDeserializerOI.getStructFieldData(row, delegateDeserializerOI.getStructFieldRef(name)); + }).collect(Collectors.toList()); + } + + @Override public ObjectInspector getObjectInspector() { + return objectInspector; + } + + /** + * Returns a view of input object inspector list between: + * 0 inclusive and the specified toIndex, exclusive. + */ + private static final class SubStructObjectInspector extends StructObjectInspector { + + private final StructObjectInspector baseOI; + private final List structFields; + + /** + * Returns a live view of the base Object inspector starting form 0 to {@code toIndex} exclusive. + * @param baseOI base Object Inspector. + * @param toIndex toIndex. + */ + private SubStructObjectInspector(StructObjectInspector baseOI, int toIndex) { + this.baseOI = baseOI; + structFields = baseOI.getAllStructFieldRefs().subList(0, toIndex); + } + + /** + * Returns all the fields. + */ + @Override public List getAllStructFieldRefs() { + return structFields; + } + + /** + * Look up a field. + * @param fieldName fieldName to be looked up. + */ + @SuppressWarnings("OptionalGetWithoutIsPresent") @Override public StructField getStructFieldRef(String fieldName) { + return this.getAllStructFieldRefs() + .stream() + .filter(ref -> ref.getFieldName().equals(fieldName)) + .findFirst() + .get(); + } + + /** + * returns null for data = null. + * @param data input. + * @param fieldRef field to extract. + */ + @Override public Object getStructFieldData(Object data, StructField fieldRef) { + return baseOI.getStructFieldData(data, fieldRef); + } + + /** + * returns null for data = null. + * @param data input data. + */ + @Override public List getStructFieldsDataAsList(Object data) { + if (data == null) { + return null; + } + int size = getAllStructFieldRefs().size(); + List res = new ArrayList<>(size); + for (int i = 0; i < size; i++) { + res.add(baseOI.getStructFieldData(data, getAllStructFieldRefs().get(i))); + } + return res; + } + + /** + * Returns the name of the data type that is inspected by this + * ObjectInspector. This is used to display the type information to the user. + * + * For primitive types, the type name is standardized. For other types, the + * type name can be something like "list<int>", "map<int,string>", java class + * names, or user-defined type names similar to typedef. + */ + @Override public String getTypeName() { + return baseOI.getTypeName(); + } + + /** + * An ObjectInspector must inherit from one of the following interfaces if + * getCategory() returns: PRIMITIVE: PrimitiveObjectInspector LIST: + * ListObjectInspector MAP: MapObjectInspector STRUCT: StructObjectInspector. + */ + @Override public Category getCategory() { + return baseOI.getCategory(); + } + } + + /** + * Class that encapsulate the logic of serialize and deserialize bytes array to/from the delegate writable format. + * @param delegate writable class. + */ + private interface BytesConverter { + byte[] getBytes(K writable); + + K getWritable(byte[] value); + } + + private static class AvroBytesConverter implements BytesConverter { + private final Schema schema; + private final DatumReader dataReader; + private final GenericDatumWriter gdw = new GenericDatumWriter<>(); + private final AvroGenericRecordWritable avroGenericRecordWritable = new AvroGenericRecordWritable(); + private final UID uid = new UID(); + + AvroBytesConverter(Schema schema) { + this.schema = schema; + dataReader = new SpecificDatumReader<>(this.schema); + } + + @Override public byte[] getBytes(AvroGenericRecordWritable writable) { + GenericRecord record = writable.getRecord(); + byte[] valueBytes = null; + try (ByteArrayOutputStream out = new ByteArrayOutputStream()) { + BinaryEncoder be = EncoderFactory.get().directBinaryEncoder(out, null); + gdw.setSchema(record.getSchema()); + gdw.write(record, be); + out.flush(); + valueBytes = out.toByteArray(); + } catch (IOException e) { + Throwables.propagate(new SerDeException(e)); + } + return valueBytes; + } + + @Override public AvroGenericRecordWritable getWritable(byte[] value) { + GenericRecord avroRecord = null; + try { + avroRecord = dataReader.read(null, DecoderFactory.get().binaryDecoder(value, null)); + } catch (IOException e) { + Throwables.propagate(new SerDeException(e)); + } + + avroGenericRecordWritable.setRecord(avroRecord); + avroGenericRecordWritable.setRecordReaderID(uid); + avroGenericRecordWritable.setFileSchema(avroRecord.getSchema()); + return avroGenericRecordWritable; + } + } + + private static class BytesWritableConverter implements BytesConverter { + @Override public byte[] getBytes(BytesWritable writable) { + return writable.getBytes(); + } + + @Override public BytesWritable getWritable(byte[] value) { + return new BytesWritable(value); + } + } + + private static class TextBytesConverter implements BytesConverter { + Text text = new Text(); + @Override public byte[] getBytes(Text writable) { + //@TODO There is no reason to decode then encode the string to bytes really + //@FIXME this issue with CTRL-CHAR ^0 added by Text at the end of string and Json serd does not like that. + try { + return writable.decode(writable.getBytes(), 0, writable.getLength()).getBytes(Charset.forName("UTF-8")); + } catch (CharacterCodingException e) { + throw new RuntimeException(e); + } + } + + @Override public Text getWritable(byte[] value) { + text.set(value); + return text; + } + } + +} diff --git kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaStorageHandler.java kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaStorageHandler.java index 96222c934b..0d64cd9c9c 100644 --- kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaStorageHandler.java +++ kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaStorageHandler.java @@ -18,10 +18,21 @@ package org.apache.hadoop.hive.kafka; +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableMap; +import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.DefaultHiveMetaHook; import org.apache.hadoop.hive.metastore.HiveMetaHook; +import org.apache.hadoop.hive.metastore.TableType; +import org.apache.hadoop.hive.metastore.api.LockType; import org.apache.hadoop.hive.metastore.api.MetaException; import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.hadoop.hive.ql.exec.Utilities; +import org.apache.hadoop.hive.ql.hooks.WriteEntity; import org.apache.hadoop.hive.ql.metadata.HiveStorageHandler; import org.apache.hadoop.hive.ql.metadata.StorageHandlerInfo; import org.apache.hadoop.hive.ql.plan.TableDesc; @@ -31,22 +42,29 @@ import org.apache.hadoop.mapred.InputFormat; import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.OutputFormat; -import org.apache.hadoop.mapred.lib.NullOutputFormat; import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.common.errors.ProducerFencedException; import org.apache.kafka.common.serialization.ByteArrayDeserializer; +import org.apache.kafka.common.serialization.ByteArraySerializer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.IOException; +import java.util.Arrays; import java.util.HashMap; +import java.util.HashSet; import java.util.Map; import java.util.Properties; +import java.util.Set; +import java.util.concurrent.TimeUnit; +import java.util.function.Predicate; /** - * Hive Kafka storage handler to allow user querying Stream of tuples from a Kafka queue. + * Hive Kafka storage handler to allow user to read and write from/to Kafka message bus. */ -public class KafkaStorageHandler implements HiveStorageHandler { +@SuppressWarnings("ALL") public class KafkaStorageHandler extends DefaultHiveMetaHook implements HiveStorageHandler { private static final Logger LOG = LoggerFactory.getLogger(KafkaStorageHandler.class); private static final String KAFKA_STORAGE_HANDLER = "org.apache.hadoop.hive.kafka.KafkaStorageHandler"; @@ -54,19 +72,19 @@ private Configuration configuration; @Override public Class getInputFormatClass() { - return KafkaPullerInputFormat.class; + return KafkaInputFormat.class; } @Override public Class getOutputFormatClass() { - return NullOutputFormat.class; + return KafkaOutputFormat.class; } @Override public Class getSerDeClass() { - return GenericKafkaSerDe.class; + return KafkaSerDe.class; } @Override public HiveMetaHook getMetaHook() { - return null; + return this; } @Override public HiveAuthorizationProvider getAuthorizationProvider() { @@ -74,43 +92,33 @@ } @Override public void configureInputJobProperties(TableDesc tableDesc, Map jobProperties) { - String topic = tableDesc.getProperties().getProperty(KafkaStreamingUtils.HIVE_KAFKA_TOPIC, ""); + configureCommonProperties(tableDesc, jobProperties); + } + + private void configureCommonProperties(TableDesc tableDesc, Map jobProperties) { + String topic = tableDesc.getProperties().getProperty(KafkaTableProperties.HIVE_KAFKA_TOPIC.getName(), ""); if (topic.isEmpty()) { throw new IllegalArgumentException("Kafka topic missing set table property->" - + KafkaStreamingUtils.HIVE_KAFKA_TOPIC); + + KafkaTableProperties.HIVE_KAFKA_TOPIC.getName()); } - jobProperties.put(KafkaStreamingUtils.HIVE_KAFKA_TOPIC, topic); - String brokerString = tableDesc.getProperties().getProperty(KafkaStreamingUtils.HIVE_KAFKA_BOOTSTRAP_SERVERS, ""); + jobProperties.put(KafkaTableProperties.HIVE_KAFKA_TOPIC.getName(), topic); + String + brokerString = + tableDesc.getProperties().getProperty(KafkaTableProperties.HIVE_KAFKA_BOOTSTRAP_SERVERS.getName(), ""); if (brokerString.isEmpty()) { throw new IllegalArgumentException("Broker address missing set table property->" - + KafkaStreamingUtils.HIVE_KAFKA_BOOTSTRAP_SERVERS); + + KafkaTableProperties.HIVE_KAFKA_BOOTSTRAP_SERVERS.getName()); + } + jobProperties.put(KafkaTableProperties.HIVE_KAFKA_BOOTSTRAP_SERVERS.getName(), brokerString); + Arrays.stream(KafkaTableProperties.values()) + .filter(tableProperty -> !tableProperty.isMandatory()) + .forEach(tableProperty -> jobProperties.put(tableProperty.getName(), + tableDesc.getProperties().getProperty(tableProperty.getName()))); + // If the user ask for EOS then set the read to only committed. + if (jobProperties.get(KafkaTableProperties.WRITE_SEMANTIC_PROPERTY.getName()) + .equals(KafkaOutputFormat.WriteSemantic.EXACTLY_ONCE.name())) { + jobProperties.put("kafka.consumer.isolation.level", "read_committed"); } - jobProperties.put(KafkaStreamingUtils.HIVE_KAFKA_BOOTSTRAP_SERVERS, brokerString); - - jobProperties.put(KafkaStreamingUtils.SERDE_CLASS_NAME, - tableDesc.getProperties().getProperty(KafkaStreamingUtils.SERDE_CLASS_NAME, KafkaJsonSerDe.class.getName())); - LOG.debug("Table properties: SerDe class name {}", jobProperties.get(KafkaStreamingUtils.SERDE_CLASS_NAME)); - - //set extra properties - tableDesc.getProperties() - .entrySet() - .stream() - .filter(objectObjectEntry -> objectObjectEntry.getKey() - .toString() - .toLowerCase() - .startsWith(KafkaStreamingUtils.CONSUMER_CONFIGURATION_PREFIX)) - .forEach(entry -> { - String - key = - entry.getKey().toString().substring(KafkaStreamingUtils.CONSUMER_CONFIGURATION_PREFIX.length() + 1); - if (KafkaStreamingUtils.FORBIDDEN_PROPERTIES.contains(key)) { - throw new IllegalArgumentException("Not suppose to set Kafka Property " + key); - } - String value = entry.getValue().toString(); - jobProperties.put(key, value); - LOG.info("Setting extra job properties: key [{}] -> value [{}]", key, value); - - }); } @Override public void configureInputJobCredentials(TableDesc tableDesc, Map secrets) { @@ -118,19 +126,21 @@ } @Override public void configureOutputJobProperties(TableDesc tableDesc, Map jobProperties) { - + configureCommonProperties(tableDesc, jobProperties); } @Override public void configureTableJobProperties(TableDesc tableDesc, Map jobProperties) { configureInputJobProperties(tableDesc, jobProperties); + configureOutputJobProperties(tableDesc, jobProperties); } @Override public void configureJobConf(TableDesc tableDesc, JobConf jobConf) { Map properties = new HashMap<>(); configureInputJobProperties(tableDesc, properties); + configureOutputJobProperties(tableDesc, properties); properties.forEach(jobConf::set); try { - KafkaStreamingUtils.copyDependencyJars(jobConf, KafkaStorageHandler.class); + KafkaUtils.copyDependencyJars(jobConf, KafkaStorageHandler.class); } catch (IOException e) { throw new RuntimeException(e); } @@ -149,11 +159,11 @@ } @Override public StorageHandlerInfo getStorageHandlerInfo(Table table) throws MetaException { - String topic = table.getParameters().get(KafkaStreamingUtils.HIVE_KAFKA_TOPIC); + String topic = table.getParameters().get(KafkaTableProperties.HIVE_KAFKA_TOPIC.getName()); if (topic == null || topic.isEmpty()) { throw new MetaException("topic is null or empty"); } - String brokers = table.getParameters().get(KafkaStreamingUtils.HIVE_KAFKA_BOOTSTRAP_SERVERS); + String brokers = table.getParameters().get(KafkaTableProperties.HIVE_KAFKA_BOOTSTRAP_SERVERS.getName()); if (brokers == null || brokers.isEmpty()) { throw new MetaException("kafka brokers string is null or empty"); } @@ -161,19 +171,232 @@ properties.setProperty(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName()); properties.setProperty(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName()); properties.setProperty(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, brokers); + properties.setProperty(CommonClientConfigs.CLIENT_ID_CONFIG, Utilities.getTaskId(getConf())); table.getParameters() .entrySet() .stream() .filter(objectObjectEntry -> objectObjectEntry.getKey() .toLowerCase() - .startsWith(KafkaStreamingUtils.CONSUMER_CONFIGURATION_PREFIX)) + .startsWith(KafkaUtils.CONSUMER_CONFIGURATION_PREFIX)) .forEach(entry -> { - String key = entry.getKey().substring(KafkaStreamingUtils.CONSUMER_CONFIGURATION_PREFIX.length() + 1); - if (KafkaStreamingUtils.FORBIDDEN_PROPERTIES.contains(key)) { + String key = entry.getKey().substring(KafkaUtils.CONSUMER_CONFIGURATION_PREFIX.length() + 1); + if (KafkaUtils.FORBIDDEN_PROPERTIES.contains(key)) { throw new IllegalArgumentException("Not suppose to set Kafka Property " + key); } properties.put(key, entry.getValue()); }); return new KafkaStorageHandlerInfo(topic, properties); } + + private Properties buildProducerProperties(Table table) { + String brokers = table.getParameters().get(KafkaTableProperties.HIVE_KAFKA_BOOTSTRAP_SERVERS.getName()); + if (brokers == null || brokers.isEmpty()) { + throw new RuntimeException("kafka brokers string is null or empty"); + } + final Properties properties = new Properties(); + properties.setProperty(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName()); + properties.setProperty(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName()); + properties.setProperty(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, brokers); + table.getParameters() + .entrySet() + .stream() + .filter(objectObjectEntry -> objectObjectEntry.getKey() + .toLowerCase() + .startsWith(KafkaUtils.PRODUCER_CONFIGURATION_PREFIX)) + .forEach(entry -> { + String key = entry.getKey().substring(KafkaUtils.PRODUCER_CONFIGURATION_PREFIX.length() + 1); + if (KafkaUtils.FORBIDDEN_PROPERTIES.contains(key)) { + throw new IllegalArgumentException("Not suppose to set Kafka Property " + key); + } + properties.put(key, entry.getValue()); + }); + return properties; + } + + @Override public LockType getLockType(WriteEntity writeEntity) { + if (writeEntity.getWriteType().equals(WriteEntity.WriteType.INSERT)) { + return LockType.SHARED_READ; + } + return LockType.SHARED_WRITE; + } + + private String getQueryId() { + return HiveConf.getVar(getConf(), HiveConf.ConfVars.HIVEQUERYID); + } + + @Override public void commitInsertTable(Table table, boolean overwrite) throws MetaException { + boolean + isExactlyOnce = + table.getParameters() + .get(KafkaTableProperties.WRITE_SEMANTIC_PROPERTY.getName()) + .equals(KafkaOutputFormat.WriteSemantic.EXACTLY_ONCE.name()); + String optimiticCommitVal = table.getParameters().get(KafkaTableProperties.HIVE_KAFKA_OPTIMISTIC_COMMIT.getName()); + boolean isTwoPhaseCommit = !Boolean.parseBoolean(optimiticCommitVal); + if (!isExactlyOnce || !isTwoPhaseCommit) { + //Case it is not 2 phase commit no open transaction to handel. + return; + } + + final Path queryWorkingDir = getQueryWorkingDir(table); + final Map> transactionsMap; + final int maxTries = Integer.parseInt(table.getParameters().get(KafkaTableProperties.MAX_RETRIES.getName())); + // We have 4 Stages ahead of us: + // 1 Fetch Transactions state from HDFS. + // 2 Build/inti all the Kafka producers and perform a pre commit call to check if we can go ahead with commit. + // 3 Commit Transactions one by one. + // 4 Clean workingDirectory. + + //First stage fetch the Transactions states + final RetryUtils.Task>> + fetchTransactionStates = + new RetryUtils.Task>>() { + @Override public Map> perform() throws Exception { + return TransactionalKafkaWriter.getTransactionsState(FileSystem.get(getConf()), queryWorkingDir); + } + }; + + try { + transactionsMap = RetryUtils.retry(fetchTransactionStates, (error) -> (error instanceof IOException), maxTries); + } catch (Exception e) { + // Can not go further + LOG.error("Can not fetch Transaction states due [{}]", e.getMessage()); + throw new MetaException(e.getMessage()); + } + + //Second Stage Resume Producers and Pre commit + final Properties baseProducerPros = buildProducerProperties(table); + final Map producersMap = new HashMap<>(); + final RetryUtils.Task buildProducersTask = new RetryUtils.Task() { + @Override public Void perform() throws Exception { + assert producersMap.size() == 0; + transactionsMap.forEach((key, value) -> { + // Base Producer propeties, missing the transaction Id. + baseProducerPros.setProperty(ProducerConfig.TRANSACTIONAL_ID_CONFIG, key); + HiveKafkaProducer producer = new HiveKafkaProducer<>(baseProducerPros); + producer.resumeTransaction(value.getLeft(), value.getRight()); + // This is a dummy RPC call to ensure that the producer still resumable and signal the Pre-commit as per : + // https://cwiki.apache.org/confluence/display/KAFKA/Transactional+Messaging+in+Kafka#EndPhase + producer.sendOffsetsToTransaction(ImmutableMap.of(), "__dry_run"); + producersMap.put(key, producer); + }); + return null; + } + }; + + RetryUtils.CleanupAfterFailure cleanUpTheMap = new RetryUtils.CleanupAfterFailure() { + @Override public void cleanup() { + producersMap.forEach((s, producer) -> producer.close(0, TimeUnit.MILLISECONDS)); + producersMap.clear(); + } + }; + final Predicate + isRetrayable = (error) -> !KafkaUtils.exceptionIsFatal(error) && !(error instanceof ProducerFencedException); + try { + RetryUtils.retry(buildProducersTask, isRetrayable, cleanUpTheMap, maxTries, "Error while Builing Producers"); + } catch (Exception e) { + // Can not go further + LOG.error("Can not fetch build produces due [{}]", e.getMessage()); + throw new MetaException(e.getMessage()); + } + + //Third Stage Commit Transactions, this part is the actual critical section. + //The commit might be retried on error, but keep in mind in some cases, like open transaction can expire + //after timeout duration of 15 mins it is not possible to go further. + final Set committedTx = new HashSet<>(); + final RetryUtils.Task commitTask = new RetryUtils.Task() { + @Override public Object perform() throws Exception { + producersMap.forEach((key, producer) -> { + if (!committedTx.contains(key)) { + producer.commitTransaction(); + committedTx.add(key); + producer.close(); + LOG.info("Committed Transaction [{}]", key); + } + }); + return null; + } + }; + + try { + RetryUtils.retry(commitTask, isRetrayable, maxTries); + } catch (Exception e) { + // at this point we are in a funky state if one commit happend!! close and log it + producersMap.forEach((key, producer) -> producer.close(0, TimeUnit.MILLISECONDS)); + LOG.error("Commit transaction failed", e); + if (committedTx.size() > 0) { + LOG.error("Partial Data Got Commited Some actions need to be Done"); + committedTx.stream().forEach(key -> LOG.error("Transaction [{}] is an orphen commit", key)); + } + throw new MetaException(e.getMessage()); + } + + //Stage four, clean the Query Directory + final RetryUtils.Task cleanQueryDirTask = new RetryUtils.Task() { + @Override public Void perform() throws Exception { + cleanWorkingDirectory(queryWorkingDir); + return null; + } + }; + try { + RetryUtils.retry(cleanQueryDirTask, (error) -> error instanceof IOException, maxTries); + } catch (Exception e) { + //just log it + LOG.error("Faild to clean Query Working Directory [{}] due to [{}]", queryWorkingDir, e.getMessage()); + } + } + + @Override public void preInsertTable(Table table, boolean overwrite) throws MetaException { + if (overwrite) { + throw new MetaException("Kafa Table does not support the overwite SQL Smentic"); + } + } + + @Override public void rollbackInsertTable(Table table, boolean overwrite) throws MetaException { + + } + + @Override public void preCreateTable(Table table) throws MetaException { + if (!table.getTableType().equals(TableType.EXTERNAL_TABLE.toString())) { + throw new MetaException(KAFKA_STORAGE_HANDLER + " supports only " + TableType.EXTERNAL_TABLE); + } + Arrays.stream(KafkaTableProperties.values()) + .filter(KafkaTableProperties::isMandatory) + .forEach(key -> Preconditions.checkNotNull(table.getParameters().get(key.getName()), + "Set Table property " + key.getName())); + // Put all the default at the pre create. + Arrays.stream(KafkaTableProperties.values()).forEach((key) -> { + if (table.getParameters().get(key.getName()) == null) { + table.putToParameters(key.getName(), key.getDefaultValue()); + } + }); + } + + @Override public void rollbackCreateTable(Table table) throws MetaException { + + } + + @Override public void commitCreateTable(Table table) throws MetaException { + commitInsertTable(table, false); + } + + @Override public void preDropTable(Table table) throws MetaException { + + } + + @Override public void rollbackDropTable(Table table) throws MetaException { + + } + + @Override public void commitDropTable(Table table, boolean deleteData) throws MetaException { + + } + + private Path getQueryWorkingDir(Table table) { + return new Path(table.getSd().getLocation(), getQueryId()); + } + + private void cleanWorkingDirectory(Path queryWorkingDir) throws IOException { + FileSystem fs = FileSystem.get(getConf()); + fs.delete(queryWorkingDir, true); + } } diff --git kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaStreamingUtils.java kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaStreamingUtils.java deleted file mode 100644 index 4802c4ec2a..0000000000 --- kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaStreamingUtils.java +++ /dev/null @@ -1,255 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you 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 org.apache.hadoop.hive.kafka; - -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hive.ql.exec.Utilities; -import org.apache.hadoop.hive.serde2.AbstractSerDe; -import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; -import org.apache.hadoop.hive.serde2.objectinspector.primitive.AbstractPrimitiveWritableObjectInspector; -import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory; -import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; -import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; -import org.apache.hadoop.io.BytesWritable; -import org.apache.hadoop.io.IntWritable; -import org.apache.hadoop.io.LongWritable; -import org.apache.hadoop.io.Writable; -import org.apache.hadoop.util.StringUtils; -import org.apache.hive.common.util.ReflectionUtil; -import org.apache.kafka.clients.CommonClientConfigs; -import org.apache.kafka.clients.consumer.ConsumerConfig; -import org.apache.kafka.common.serialization.ByteArrayDeserializer; - -import java.io.IOException; -import java.util.Arrays; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Objects; -import java.util.Properties; -import java.util.Set; -import java.util.function.Function; -import java.util.stream.Collectors; - -/** - * Constant, Table properties, Utilities class. - */ -final class KafkaStreamingUtils { - - /** - * MANDATORY Table property indicating kafka topic backing the table. - */ - static final String HIVE_KAFKA_TOPIC = "kafka.topic"; - /** - * MANDATORY Table property indicating kafka broker(s) connection string. - */ - static final String HIVE_KAFKA_BOOTSTRAP_SERVERS = "kafka.bootstrap.servers"; - /** - * Table property indicating which delegate serde to be used, NOT MANDATORY defaults to {@link KafkaJsonSerDe} - */ - static final String SERDE_CLASS_NAME = "kafka.serde.class"; - /** - * Table property indicating poll/fetch timeout period in millis. - * FYI this is independent from internal Kafka consumer timeouts, defaults to {@DEFAULT_CONSUMER_POLL_TIMEOUT_MS}. - */ - static final String HIVE_KAFKA_POLL_TIMEOUT = "hive.kafka.poll.timeout.ms"; - /** - * Default poll timeout for fetching metadata and record batch. - */ - static final long DEFAULT_CONSUMER_POLL_TIMEOUT_MS = 5000L; // 5 seconds - /** - * Table property prefix used to inject kafka consumer properties, e.g "kafka.consumer.max.poll.records" = "5000" - * this will lead to inject max.poll.records=5000 to the Kafka Consumer. NOT MANDATORY defaults to nothing - */ - static final String CONSUMER_CONFIGURATION_PREFIX = "kafka.consumer"; - /** - * Set of Kafka properties that the user can not set via DDLs. - */ - static final HashSet FORBIDDEN_PROPERTIES = - new HashSet<>(ImmutableList.of(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, - ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, - ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, - ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG)); - - private KafkaStreamingUtils() { - } - - /** - * @param configuration Job configs - * - * @return default consumer properties - */ - static Properties consumerProperties(Configuration configuration) { - final Properties props = new Properties(); - // we are managing the commit offset - props.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false"); - // we are seeking in the stream so no reset - props.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "none"); - String brokerEndPoint = configuration.get(HIVE_KAFKA_BOOTSTRAP_SERVERS); - if (brokerEndPoint == null || brokerEndPoint.isEmpty()) { - throw new IllegalArgumentException("Kafka Broker End Point is missing Please set Config " - + HIVE_KAFKA_BOOTSTRAP_SERVERS); - } - props.setProperty(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, brokerEndPoint); - props.setProperty(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName()); - props.setProperty(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName()); - // user can always override stuff - final Map - kafkaProperties = - configuration.getValByRegex("^" + CONSUMER_CONFIGURATION_PREFIX + "\\..*"); - for (Map.Entry entry : kafkaProperties.entrySet()) { - String key = entry.getKey().substring(CONSUMER_CONFIGURATION_PREFIX.length() + 1); - if (FORBIDDEN_PROPERTIES.contains(key)) { - throw new IllegalArgumentException("Not suppose to set Kafka Property " + key); - } - props.setProperty(key, entry.getValue()); - } - return props; - } - - static void copyDependencyJars(Configuration conf, Class... classes) throws IOException { - Set jars = new HashSet<>(); - FileSystem localFs = FileSystem.getLocal(conf); - jars.addAll(conf.getStringCollection("tmpjars")); - jars.addAll(Arrays.stream(classes).filter(Objects::nonNull).map(clazz -> { - String path = Utilities.jarFinderGetJar(clazz); - if (path == null) { - throw new RuntimeException("Could not find jar for class " + clazz + " in order to ship it to the cluster."); - } - try { - if (!localFs.exists(new Path(path))) { - throw new RuntimeException("Could not validate jar file " + path + " for class " + clazz); - } - } catch (IOException e) { - throw new RuntimeException(e); - } - return path; - }).collect(Collectors.toList())); - - if (jars.isEmpty()) { - return; - } - conf.set("tmpjars", StringUtils.arrayToString(jars.toArray(new String[0]))); - } - - static AbstractSerDe createDelegate(String className) { - final Class clazz; - try { - //noinspection unchecked - clazz = (Class) Class.forName(className); - } catch (ClassNotFoundException e) { - throw new RuntimeException(e); - } - // we are not setting conf thus null is okay - return ReflectionUtil.newInstance(clazz, null); - } - - /** - * Basic Enum class for all the metadata columns appended to the Kafka row by the deserializer. - */ - enum MetadataColumn { - /** - * Record offset column name added as extra metadata column to row as long. - */ - OFFSET("__offset", TypeInfoFactory.longTypeInfo), - /** - * Record Kafka Partition column name added as extra meta column of type int. - */ - PARTITION("__partition", TypeInfoFactory.intTypeInfo), - /** - * Record Kafka key column name added as extra meta column of type binary blob. - */ - KEY("__key", TypeInfoFactory.binaryTypeInfo), - /** - * Record Timestamp column name, added as extra meta column of type long. - */ - TIMESTAMP("__timestamp", TypeInfoFactory.longTypeInfo), - /** - * Start offset given by the input split, this will reflect the actual start of TP or start given by split pruner. - */ - START_OFFSET("__start_offset", TypeInfoFactory.longTypeInfo), - /** - * End offset given by input split at run time. - */ - END_OFFSET("__end_offset", TypeInfoFactory.longTypeInfo); - - private final String name; - private final TypeInfo typeInfo; - - MetadataColumn(String name, TypeInfo typeInfo) { - this.name = name; - this.typeInfo = typeInfo; - } - - public String getName() { - return name; - } - - public AbstractPrimitiveWritableObjectInspector getObjectInspector() { - return PrimitiveObjectInspectorFactory.getPrimitiveWritableObjectInspector(TypeInfoFactory.getPrimitiveTypeInfo( - typeInfo.getTypeName())); - } - } - - //Order at which column and types will be appended to the original row. - /** - * Kafka metadata columns order list. - */ - private static final List KAFKA_METADATA_COLUMNS = - Arrays.asList(MetadataColumn.KEY, - MetadataColumn.PARTITION, - MetadataColumn.OFFSET, - MetadataColumn.TIMESTAMP, - MetadataColumn.START_OFFSET, - MetadataColumn.END_OFFSET); - - /** - * Kafka metadata column names. - */ - static final List KAFKA_METADATA_COLUMN_NAMES = KAFKA_METADATA_COLUMNS - .stream() - .map(MetadataColumn::getName) - .collect(Collectors.toList()); - - /** - * Kafka metadata column inspectors. - */ - static final List KAFKA_METADATA_INSPECTORS = KAFKA_METADATA_COLUMNS - .stream() - .map(MetadataColumn::getObjectInspector) - .collect(Collectors.toList()); - - /** - * Reverse lookup map used to convert records from kafka Writable to hive Writable based on Kafka semantic. - */ - static final Map> - recordWritableFnMap = ImmutableMap.>builder() - .put(MetadataColumn.END_OFFSET.getName(), (record) -> new LongWritable(record.getEndOffset())) - .put(MetadataColumn.KEY.getName(), - record -> record.getRecordKey() == null ? null : new BytesWritable(record.getRecordKey())) - .put(MetadataColumn.OFFSET.getName(), record -> new LongWritable(record.getOffset())) - .put(MetadataColumn.PARTITION.getName(), record -> new IntWritable(record.getPartition())) - .put(MetadataColumn.START_OFFSET.getName(), record -> new LongWritable(record.getStartOffset())) - .put(MetadataColumn.TIMESTAMP.getName(), record -> new LongWritable(record.getTimestamp())) - .build(); -} diff --git kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaTableProperties.java kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaTableProperties.java new file mode 100644 index 0000000000..2e1f6faf1f --- /dev/null +++ kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaTableProperties.java @@ -0,0 +1,83 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.hadoop.hive.kafka; + +import org.apache.hadoop.hive.serde2.JsonSerDe; + +/** + * Table properties used by Kafka Storage handler. + */ +enum KafkaTableProperties { + /** + * MANDATORY Table property indicating kafka topic backing the table. + */ + HIVE_KAFKA_TOPIC("kafka.topic", null), + /** + * MANDATORY Table property indicating kafka broker(s) connection string. + */ + HIVE_KAFKA_BOOTSTRAP_SERVERS("kafka.bootstrap.servers", null), + /** + * Table property indicating which delegate serde to be used. + */ + SERDE_CLASS_NAME("kafka.serde.class", JsonSerDe.class.getName()), + /** + * Table property indicating poll/fetch timeout period in millis. + * FYI this is independent from internal Kafka consumer timeouts. + */ + KAFKA_POLL_TIMEOUT("hive.kafka.poll.timeout.ms", "5000"), + + MAX_RETRIES("hive.kafka.max.retries", "6"), KAFKA_FETCH_METADATA_TIMEOUT("hive.kafka.metadata.poll.timeout.ms", + "30000"), + /** + * Table property indicating the write semantic possible enum values are: + * {@link KafkaOutputFormat.WriteSemantic}. + */ + WRITE_SEMANTIC_PROPERTY("kafka.write.semantic", KafkaOutputFormat.WriteSemantic.AT_LEAST_ONCE.name()), + /** + * Table property that indicates if we should commit within the task or delay it to the Metadata Hook Commit call. + */ + HIVE_KAFKA_OPTIMISTIC_COMMIT("hive.kafka.optimistic.commit", "false"); + + /** + * Kafka storage handler table properties constructor. + * @param name property name. + * @param defaultValue default value, set to NULL if the property is mandatory and need to be set by the user. + */ + KafkaTableProperties(String name, String defaultValue) { + this.name = name; + this.defaultValue = defaultValue; + this.mandatory = defaultValue == null; + } + + public String getName() { + return name; + } + + public String getDefaultValue() { + return defaultValue; + } + + public boolean isMandatory() { + return mandatory; + } + + private final String name; + private final String defaultValue; + private final boolean mandatory; +} diff --git kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaUtils.java kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaUtils.java new file mode 100644 index 0000000000..6ae9c8d276 --- /dev/null +++ kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaUtils.java @@ -0,0 +1,255 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.hadoop.hive.kafka; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.ql.exec.Utilities; +import org.apache.hadoop.hive.serde2.AbstractSerDe; +import org.apache.hadoop.util.StringUtils; +import org.apache.hive.common.util.ReflectionUtil; +import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.errors.AuthenticationException; +import org.apache.kafka.common.errors.AuthorizationException; +import org.apache.kafka.common.errors.InvalidTopicException; +import org.apache.kafka.common.errors.OffsetMetadataTooLarge; +import org.apache.kafka.common.errors.SecurityDisabledException; +import org.apache.kafka.common.errors.SerializationException; +import org.apache.kafka.common.errors.UnknownServerException; +import org.apache.kafka.common.serialization.ByteArrayDeserializer; + +import java.io.IOException; +import java.util.Arrays; +import java.util.HashSet; +import java.util.Map; +import java.util.Objects; +import java.util.Properties; +import java.util.Set; +import java.util.UUID; +import java.util.stream.Collectors; + +/** + * Utils class for Kafka Storage handler plus some Constants. + */ +final class KafkaUtils { + + private KafkaUtils() { + } + + /** + * Table property prefix used to inject kafka consumer properties, e.g "kafka.consumer.max.poll.records" = "5000" + * this will lead to inject max.poll.records=5000 to the Kafka Consumer. NOT MANDATORY defaults to nothing + */ + static final String CONSUMER_CONFIGURATION_PREFIX = "kafka.consumer"; + + /** + * Table property prefix used to inject kafka producer properties, e.g "kafka.producer.lingers.ms" = "100". + */ + static final String PRODUCER_CONFIGURATION_PREFIX = "kafka.producer"; + + /** + * Set of Kafka properties that the user can not set via DDLs. + */ + static final Set + FORBIDDEN_PROPERTIES = + new HashSet<>(ImmutableList.of(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, + ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, + ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, + ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, + ProducerConfig.TRANSACTIONAL_ID_CONFIG, + ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, + ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG)); + + /** + * @param configuration Job configs + * + * @return default consumer properties + */ + static Properties consumerProperties(Configuration configuration) { + final Properties props = new Properties(); + // we are managing the commit offset + props.setProperty(CommonClientConfigs.CLIENT_ID_CONFIG, Utilities.getTaskId(configuration)); + props.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false"); + // we are seeking in the stream so no reset + props.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "none"); + String brokerEndPoint = configuration.get(KafkaTableProperties.HIVE_KAFKA_BOOTSTRAP_SERVERS.getName()); + if (brokerEndPoint == null || brokerEndPoint.isEmpty()) { + throw new IllegalArgumentException("Kafka Broker End Point is missing Please set Config " + + KafkaTableProperties.HIVE_KAFKA_BOOTSTRAP_SERVERS.getName()); + } + props.setProperty(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, brokerEndPoint); + props.setProperty(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName()); + props.setProperty(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName()); + // user can always override stuff + props.putAll(extractExtraProperties(configuration, CONSUMER_CONFIGURATION_PREFIX)); + return props; + } + + private static Map extractExtraProperties(final Configuration configuration, String prefix) { + ImmutableMap.Builder builder = ImmutableMap.builder(); + final Map kafkaProperties = configuration.getValByRegex("^" + prefix + "\\..*"); + for (Map.Entry entry : kafkaProperties.entrySet()) { + String key = entry.getKey().substring(prefix.length() + 1); + if (FORBIDDEN_PROPERTIES.contains(key)) { + throw new IllegalArgumentException("Not suppose to set Kafka Property " + key); + } + builder.put(key, entry.getValue()); + } + return builder.build(); + } + + static Properties producerProperties(Configuration configuration) { + final String writeSemanticValue = configuration.get(KafkaTableProperties.WRITE_SEMANTIC_PROPERTY.getName()); + final KafkaOutputFormat.WriteSemantic writeSemantic = KafkaOutputFormat.WriteSemantic.valueOf(writeSemanticValue); + final Properties properties = new Properties(); + String brokerEndPoint = configuration.get(KafkaTableProperties.HIVE_KAFKA_BOOTSTRAP_SERVERS.getName()); + if (brokerEndPoint == null || brokerEndPoint.isEmpty()) { + throw new IllegalArgumentException("Kafka Broker End Point is missing Please set Config " + + KafkaTableProperties.HIVE_KAFKA_BOOTSTRAP_SERVERS.getName()); + } + properties.setProperty(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, brokerEndPoint); + // user can always override stuff + properties.putAll(extractExtraProperties(configuration, PRODUCER_CONFIGURATION_PREFIX)); + String taskId = configuration.get("mapred.task.id", null); + properties.setProperty(CommonClientConfigs.CLIENT_ID_CONFIG, + taskId == null ? "random_" + UUID.randomUUID().toString() : taskId); + switch (writeSemantic) { + case BEST_EFFORT: + break; + case AT_LEAST_ONCE: + properties.setProperty(ProducerConfig.RETRIES_CONFIG, String.valueOf(Integer.MAX_VALUE)); + //The number of acknowledgments the producer requires the leader to have received before considering a request as + // complete, all means from all replicas. + properties.setProperty(ProducerConfig.ACKS_CONFIG, "all"); + break; + case EXACTLY_ONCE: + // Assuming that TaskId is ReducerId_attemptId. need the Reducer ID to fence out zombie kafka producers. + String reducerId = getTaskId(configuration); + //The number of acknowledgments the producer requires the leader to have received before considering a request as + // complete, all means from all replicas. + properties.setProperty(ProducerConfig.ACKS_CONFIG, "all"); + properties.setProperty(ProducerConfig.RETRIES_CONFIG, String.valueOf(Integer.MAX_VALUE)); + properties.setProperty(ProducerConfig.TRANSACTIONAL_ID_CONFIG, reducerId); + //Producer set to be IDEMPOTENT eg ensure that send() retries are idempotent. + properties.setProperty(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, "true"); + break; + default: + throw new IllegalArgumentException("Unknown Semantic " + writeSemantic); + } + return properties; + } + + @SuppressWarnings("SameParameterValue") static void copyDependencyJars(Configuration conf, Class... classes) + throws IOException { + Set jars = new HashSet<>(); + FileSystem localFs = FileSystem.getLocal(conf); + jars.addAll(conf.getStringCollection("tmpjars")); + jars.addAll(Arrays.stream(classes) + .filter(Objects::nonNull) + .map(clazz -> { + String path = Utilities.jarFinderGetJar(clazz); + if (path == null) { + throw new RuntimeException("Could not find jar for class " + + clazz + + " in order to ship it to the cluster."); + } + try { + if (!localFs.exists(new Path(path))) { + throw new RuntimeException("Could not validate jar file " + path + " for class " + clazz); + } + } catch (IOException e) { + throw new RuntimeException(e); + } + return path; + }).collect(Collectors.toList())); + + if (jars.isEmpty()) { + return; + } + conf.set("tmpjars", StringUtils.arrayToString(jars.toArray(new String[0]))); + } + + static AbstractSerDe createDelegate(String className) { + final Class clazz; + try { + //noinspection unchecked + clazz = (Class) Class.forName(className); + } catch (ClassNotFoundException e) { + throw new RuntimeException(e); + } + // we are not setting conf thus null is okay + return ReflectionUtil.newInstance(clazz, null); + } + + static ProducerRecord toProducerRecord(String topic, KafkaWritable value) { + return new ProducerRecord<>(topic, + value.getPartition() != -1 ? value.getPartition() : null, + value.getTimestamp() != -1L ? value.getTimestamp() : null, + value.getRecordKey(), + value.getValue()); + } + + /** + * Check if the exception is Non-Retriable there a show stopper all we can do is clean and exit. + * @param exception input exception object. + * @return true if the exception is fatal thus we only can abort and rethrow the cause. + */ + static boolean exceptionIsFatal(final Throwable exception) { + final boolean + securityException = + exception instanceof AuthenticationException + || exception instanceof AuthorizationException + || exception instanceof SecurityDisabledException; + + final boolean + communicationException = + exception instanceof InvalidTopicException + || exception instanceof UnknownServerException + || exception instanceof SerializationException + || exception instanceof OffsetMetadataTooLarge + || exception instanceof IllegalStateException; + + return securityException || communicationException; + } + + /** + * Computes the kafka producer transaction id. The Tx id HAS to be the same across task restarts, + * that is why we are excluding the attempt id by removing the last string after last `_`. + * Assuming the taskId format is taskId_[m-r]_attemptId. + * + * @param hiveConf Hive Configuration. + * @return the taskId without the attempt id. + */ + static String getTaskId(Configuration hiveConf) { + String id = Preconditions.checkNotNull(hiveConf.get("mapred.task.id", null)); + int index = id.lastIndexOf("_"); + if (index != -1) { + return id.substring(0, index); + } + return id; + } + +} diff --git kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaRecordWritable.java kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaWritable.java similarity index 74% rename from kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaRecordWritable.java rename to kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaWritable.java index 1b00f8549f..681b666fdf 100644 --- kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaRecordWritable.java +++ kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaWritable.java @@ -18,6 +18,9 @@ package org.apache.hadoop.hive.kafka; +import org.apache.hadoop.io.BytesWritable; +import org.apache.hadoop.io.IntWritable; +import org.apache.hadoop.io.LongWritable; import org.apache.hadoop.io.Writable; import org.apache.kafka.clients.consumer.ConsumerRecord; @@ -30,43 +33,27 @@ /** * Writable implementation of Kafka ConsumerRecord. - * Serialized in the form + * Serialized in the form: * {@code timestamp} long| {@code partition} (int) | {@code offset} (long) | * {@code startOffset} (long) | {@code endOffset} (long) | {@code value.size()} (int) | * {@code value} (byte []) | {@code recordKey.size()}| {@code recordKey (byte [])} */ -public class KafkaRecordWritable implements Writable { +public class KafkaWritable implements Writable { - /** - * Kafka partition id - */ private int partition; - /** - * Record Offset - */ private long offset; + private long timestamp; + private byte[] value; + private byte[] recordKey; + /** - * Fist offset given by the input split used to pull the event {@link KafkaPullerInputSplit#getStartOffset()} + * Fist offset given by the input split used to pull the event {@link KafkaInputSplit#getStartOffset()}. */ private long startOffset; /** - * Last Offset given by the input split used to pull the event {@link KafkaPullerInputSplit#getEndOffset()} + * Last Offset given by the input split used to pull the event {@link KafkaInputSplit#getEndOffset()}. */ private long endOffset; - /** - * Event timestamp provided by Kafka Record {@link ConsumerRecord#timestamp()} - */ - private long timestamp; - /** - * Record value - */ - private byte[] value; - - /** - * Record key content or null - */ - private byte[] recordKey; - void set(ConsumerRecord consumerRecord, long startOffset, long endOffset) { this.partition = consumerRecord.partition(); @@ -78,13 +65,13 @@ void set(ConsumerRecord consumerRecord, long startOffset, long e this.endOffset = endOffset; } - KafkaRecordWritable(int partition, - long offset, - long timestamp, - byte[] value, - long startOffset, - long endOffset, - @Nullable byte[] recordKey) { + KafkaWritable(int partition, + long offset, + long timestamp, + byte[] value, + long startOffset, + long endOffset, + @Nullable byte[] recordKey) { this.partition = partition; this.offset = offset; this.timestamp = timestamp; @@ -94,7 +81,11 @@ void set(ConsumerRecord consumerRecord, long startOffset, long e this.endOffset = endOffset; } - @SuppressWarnings("WeakerAccess") public KafkaRecordWritable() { + KafkaWritable(int partition, long timestamp, byte[] value, @Nullable byte[] recordKey) { + this(partition, -1, timestamp, value, -1, -1, recordKey); + } + + @SuppressWarnings("WeakerAccess") public KafkaWritable() { } @Override public void write(DataOutput dataOutput) throws IOException { @@ -139,7 +130,7 @@ int getPartition() { return partition; } - long getOffset() { + @SuppressWarnings("WeakerAccess") long getOffset() { return offset; } @@ -151,16 +142,15 @@ long getTimestamp() { return value; } - long getStartOffset() { + @SuppressWarnings("WeakerAccess") long getStartOffset() { return startOffset; } - long getEndOffset() { + @SuppressWarnings("WeakerAccess") long getEndOffset() { return endOffset; } - @Nullable - byte[] getRecordKey() { + @Nullable byte[] getRecordKey() { return recordKey; } @@ -168,10 +158,10 @@ long getEndOffset() { if (this == o) { return true; } - if (!(o instanceof KafkaRecordWritable)) { + if (!(o instanceof KafkaWritable)) { return false; } - KafkaRecordWritable writable = (KafkaRecordWritable) o; + KafkaWritable writable = (KafkaWritable) o; return partition == writable.partition && offset == writable.offset && startOffset == writable.startOffset @@ -189,7 +179,7 @@ long getEndOffset() { } @Override public String toString() { - return "KafkaRecordWritable{" + return "KafkaWritable{" + "partition=" + partition + ", offset=" @@ -206,4 +196,24 @@ long getEndOffset() { + Arrays.toString(recordKey) + '}'; } + + Writable getHiveWritable(MetadataColumn metadataColumn) { + switch (metadataColumn) { + case OFFSET: + return new LongWritable(getOffset()); + case PARTITION: + return new IntWritable(getPartition()); + case TIMESTAMP: + return new LongWritable(getTimestamp()); + case KEY: + return getRecordKey() == null ? null : new BytesWritable(getRecordKey()); + case START_OFFSET: + return new LongWritable(getStartOffset()); + case END_OFFSET: + return new LongWritable(getEndOffset()); + default: + throw new IllegalArgumentException("Unknown metadata column [" + metadataColumn.getName() + "]"); + } + } + } diff --git kafka-handler/src/java/org/apache/hadoop/hive/kafka/MetadataColumn.java kafka-handler/src/java/org/apache/hadoop/hive/kafka/MetadataColumn.java new file mode 100644 index 0000000000..60e1aea55d --- /dev/null +++ kafka-handler/src/java/org/apache/hadoop/hive/kafka/MetadataColumn.java @@ -0,0 +1,120 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.hadoop.hive.kafka; + +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.AbstractPrimitiveWritableObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; + +import javax.annotation.Nullable; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.function.Function; +import java.util.stream.Collectors; + +/** + * Enum class for all the metadata columns appended to the Kafka row by the Hive Serializer/Deserializer. + * + *

+ *Design Notes: + * + * It is important to note that the order at which columns are appended matters, the order is governed by: + * {@link MetadataColumn#KAFKA_METADATA_COLUMNS}. + * + * If you add a new Column make sure to added its Writable converter to {@link KafkaWritable}. + * + */ +enum MetadataColumn { + + /** + * Kafka Record's offset column name added as extra metadata column to row as long. + */ + OFFSET("__offset", TypeInfoFactory.longTypeInfo), + /** + * Record Kafka Partition column name added as extra meta column of type int. + */ + PARTITION("__partition", TypeInfoFactory.intTypeInfo), + /** + * Record Kafka key column name added as extra meta column of type binary blob. + */ + KEY("__key", TypeInfoFactory.binaryTypeInfo), + /** + * Record Timestamp column name, added as extra meta column of type long. + */ + TIMESTAMP("__timestamp", TypeInfoFactory.longTypeInfo), + /** + * Start offset given by the input split, this will reflect the actual start of TP or start given by split pruner. + */ + // @TODO To be removed next PR it is here to make review easy + START_OFFSET("__start_offset", TypeInfoFactory.longTypeInfo), + /** + * End offset given by input split at run time. + */ + // @TODO To be removed next PR it is here to make review easy + END_OFFSET("__end_offset", TypeInfoFactory.longTypeInfo); + + /** + * Kafka metadata columns list that indicates the order of appearance for each column in final row. + */ + private static final List + KAFKA_METADATA_COLUMNS = + Arrays.asList(KEY, PARTITION, OFFSET, TIMESTAMP, START_OFFSET, END_OFFSET); + + static final List + KAFKA_METADATA_INSPECTORS = + KAFKA_METADATA_COLUMNS.stream().map(MetadataColumn::getObjectInspector).collect(Collectors.toList()); + + static final List + KAFKA_METADATA_COLUMN_NAMES = + KAFKA_METADATA_COLUMNS.stream().map(MetadataColumn::getName).collect(Collectors.toList()); + + private final String name; + private final TypeInfo typeInfo; + + MetadataColumn(String name, TypeInfo typeInfo) { + this.name = name; + this.typeInfo = typeInfo; + } + + public String getName() { + return name; + } + + public AbstractPrimitiveWritableObjectInspector getObjectInspector() { + return PrimitiveObjectInspectorFactory.getPrimitiveWritableObjectInspector(TypeInfoFactory.getPrimitiveTypeInfo( + typeInfo.getTypeName())); + } + + private static final Map + NAMES_MAP = + Arrays.stream(MetadataColumn.values()).collect(Collectors.toMap(MetadataColumn::getName, Function.identity())); + /** + * Column name to MetadataColumn instance. + * @param name column name. + * @return instance of {@link MetadataColumn} or null if column name is absent + */ + @Nullable + static MetadataColumn forName(String name) { + return NAMES_MAP.get(name); + } + +} diff --git kafka-handler/src/java/org/apache/hadoop/hive/kafka/RetryUtils.java kafka-handler/src/java/org/apache/hadoop/hive/kafka/RetryUtils.java new file mode 100644 index 0000000000..b2bb208b19 --- /dev/null +++ kafka-handler/src/java/org/apache/hadoop/hive/kafka/RetryUtils.java @@ -0,0 +1,154 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.hadoop.hive.kafka; + +import com.google.common.base.Preconditions; +import com.google.common.base.Throwables; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; +import java.util.concurrent.ThreadLocalRandom; +import java.util.function.Predicate; + +/** + * Retry utils class mostly taken from Apache Druid Project org.apache.druid.java.util.common.RetryUtils. + */ +public final class RetryUtils { + private static final Logger LOG = LoggerFactory.getLogger(RetryUtils.class); + private static final long MAX_SLEEP_MILLIS = 60000; + private static final long BASE_SLEEP_MILLIS = 1000; + + private RetryUtils() { + } + + /** + * Task to be performed. + * @param returned type of the task. + */ + public interface Task { + /** + * This method is tried up to maxTries times unless it succeeds. + */ + T perform() throws Exception; + } + + /** + * Cleanup procedure after each failed attempt. + */ + @SuppressWarnings("WeakerAccess") public interface CleanupAfterFailure { + /** + * This is called once {@link Task#perform()} fails. Retrying is stopped once this method throws an exception, + * so errors inside this method should be ignored if you don't want to stop retrying. + */ + void cleanup(); + } + + /** + * Retry an operation using fuzzy exponentially increasing backoff. The wait time after the nth failed attempt is + * min(60000ms, 1000ms * pow(2, n - 1)), fuzzed by a number drawn from a Gaussian distribution with mean 0 and + * standard deviation 0.2. + * + * If maxTries is exhausted, or if shouldRetry returns false, the last exception thrown by "f" will be thrown + * by this function. + * + * @param f the operation + * @param shouldRetry predicate determining whether we should retry after a particular exception thrown by "f" + * @param quietTries first quietTries attempts will LOG exceptions at DEBUG level rather than WARN + * @param maxTries maximum number of attempts + * + * @return result of the first successful operation + * + * @throws Exception if maxTries is exhausted, or shouldRetry returns false + */ + @SuppressWarnings("WeakerAccess") static T retry(final Task f, + final Predicate shouldRetry, + final int quietTries, + final int maxTries, + @Nullable final CleanupAfterFailure cleanupAfterFailure, + @Nullable final String messageOnRetry) throws Exception { + Preconditions.checkArgument(maxTries > 0, "maxTries > 0"); + Preconditions.checkArgument(quietTries >= 0, "quietTries >= 0"); + int nTry = 0; + final int maxRetries = maxTries - 1; + while (true) { + try { + nTry++; + return f.perform(); + } catch (Throwable e) { + if (cleanupAfterFailure != null) { + cleanupAfterFailure.cleanup(); + } + if (nTry < maxTries && shouldRetry.test(e)) { + awaitNextRetry(e, messageOnRetry, nTry, maxRetries, nTry <= quietTries); + } else { + Throwables.propagateIfInstanceOf(e, Exception.class); + throw Throwables.propagate(e); + } + } + } + } + + static T retry(final Task f, Predicate shouldRetry, final int maxTries) throws Exception { + return retry(f, shouldRetry, 0, maxTries); + } + + @SuppressWarnings({ "WeakerAccess", "SameParameterValue" }) static T retry(final Task f, + final Predicate shouldRetry, + final int quietTries, + final int maxTries) throws Exception { + return retry(f, shouldRetry, quietTries, maxTries, null, null); + } + + @SuppressWarnings("unused") public static T retry(final Task f, + final Predicate shouldRetry, + final CleanupAfterFailure onEachFailure, + final int maxTries, + final String messageOnRetry) throws Exception { + return retry(f, shouldRetry, 0, maxTries, onEachFailure, messageOnRetry); + } + + private static void awaitNextRetry(final Throwable e, + @Nullable final String messageOnRetry, + final int nTry, + final int maxRetries, + final boolean quiet) throws InterruptedException { + final long sleepMillis = nextRetrySleepMillis(nTry); + final String fullMessage; + + if (messageOnRetry == null) { + fullMessage = String.format("Retrying (%d of %d) in %,dms.", nTry, maxRetries, sleepMillis); + } else { + fullMessage = String.format("%s, retrying (%d of %d) in %,dms.", messageOnRetry, nTry, maxRetries, sleepMillis); + } + + if (quiet) { + LOG.debug(fullMessage, e); + } else { + LOG.warn(fullMessage, e); + } + + Thread.sleep(sleepMillis); + } + + private static long nextRetrySleepMillis(final int nTry) { + final double fuzzyMultiplier = Math.min(Math.max(1 + 0.2 * ThreadLocalRandom.current().nextGaussian(), 0), 2); + return (long) (Math.min(MAX_SLEEP_MILLIS, BASE_SLEEP_MILLIS * Math.pow(2, nTry - 1)) * fuzzyMultiplier); + } +} diff --git kafka-handler/src/java/org/apache/hadoop/hive/kafka/SimpleKafkaWriter.java kafka-handler/src/java/org/apache/hadoop/hive/kafka/SimpleKafkaWriter.java new file mode 100644 index 0000000000..c95bdb02de --- /dev/null +++ kafka-handler/src/java/org/apache/hadoop/hive/kafka/SimpleKafkaWriter.java @@ -0,0 +1,185 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.hadoop.hive.kafka; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import org.apache.hadoop.hive.ql.exec.FileSinkOperator; +import org.apache.hadoop.io.BytesWritable; +import org.apache.hadoop.io.Writable; +import org.apache.hadoop.mapred.RecordWriter; +import org.apache.hadoop.mapred.Reporter; +import org.apache.kafka.clients.producer.Callback; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.errors.TimeoutException; +import org.apache.kafka.common.serialization.ByteArraySerializer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.util.Properties; +import java.util.UUID; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReference; + +/** + * Hive to Kafka Simple Record Writer. It can be used to achieve AT LEAST ONCE semantic, or no guaranties at all. + */ +class SimpleKafkaWriter implements FileSinkOperator.RecordWriter, RecordWriter { + + private static final Logger LOG = LoggerFactory.getLogger(SimpleKafkaWriter.class); + private static final String + TIMEOUT_CONFIG_HINT = + "Try increasing producer property [`retries`] and [`retry.backoff.ms`] to avoid this error [{}]."; + private static final String + ABORT_MSG = + "Writer [%s] aborting Send. Caused by [%s]. Sending to topic [%s]. Record offset [%s];"; + private static final String + ACTION_ABORT = + "WriterId [{}] lost record from Topic [{}], delivery Semantic [{}] -> ACTION=ABORT, ERROR caused by [{}]"; + private static final String + ACTION_CARRY_ON = + "WriterId [{}], lost record from Topic [{}], delivery Semantic [{}] -> ACTION=CARRY-ON"; + + private final String topic; + private final String writerId; + private final KafkaOutputFormat.WriteSemantic writeSemantic; + private final KafkaProducer producer; + private final Callback callback; + private final AtomicReference sendExceptionRef = new AtomicReference<>(); + private final AtomicLong lostRecords = new AtomicLong(0L); + private long sentRecords = 0L; + + /** + * @param topic Kafka Topic. + * @param writerId Writer Id use for logging. + * @param atLeastOnce true if the desired delivery semantic is at least once. + * @param properties Kafka Producer properties. + */ + SimpleKafkaWriter(String topic, @Nullable String writerId, boolean atLeastOnce, Properties properties) { + this.writeSemantic = + atLeastOnce ? KafkaOutputFormat.WriteSemantic.AT_LEAST_ONCE : KafkaOutputFormat.WriteSemantic.BEST_EFFORT; + this.writerId = writerId == null ? UUID.randomUUID().toString() : writerId; + this.topic = Preconditions.checkNotNull(topic, "Topic can not be null"); + Preconditions.checkState(properties.getProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG) != null, + "set [" + ProducerConfig.BOOTSTRAP_SERVERS_CONFIG + "] property"); + producer = new KafkaProducer<>(properties, new ByteArraySerializer(), new ByteArraySerializer()); + + this.callback = (metadata, exception) -> { + if (exception != null) { + lostRecords.getAndIncrement(); + switch (writeSemantic) { + case BEST_EFFORT: + LOG.warn(ACTION_CARRY_ON, getWriterId(), topic, writeSemantic); + break; + case AT_LEAST_ONCE: + LOG.error(ACTION_ABORT, getWriterId(), topic, writeSemantic, exception.getMessage()); + sendExceptionRef.compareAndSet(null, exception); + break; + default: + throw new IllegalArgumentException("Unsupported delivery semantic " + writeSemantic); + } + } + }; + LOG.info("Starting WriterId [{}], Delivery Semantic [{}], Target Kafka Topic [{}]", + writerId, writeSemantic, + topic); + } + + @Override public void write(Writable w) throws IOException { + checkExceptions(); + try { + sentRecords++; + producer.send(KafkaUtils.toProducerRecord(topic, (KafkaWritable) w), callback); + } catch (KafkaException kafkaException) { + handleKafkaException(kafkaException); + checkExceptions(); + } + } + + private void handleKafkaException(KafkaException kafkaException) { + if (kafkaException instanceof TimeoutException) { + //This might happen if the producer cannot send data to the Kafka cluster and thus, its internal buffer fills up. + LOG.error(TIMEOUT_CONFIG_HINT, kafkaException.getMessage()); + } + if (KafkaUtils.exceptionIsFatal(kafkaException)) { + LOG.error(String.format(ABORT_MSG, writerId, kafkaException.getMessage(), topic, -1L)); + sendExceptionRef.compareAndSet(null, kafkaException); + } else { + if (writeSemantic == KafkaOutputFormat.WriteSemantic.AT_LEAST_ONCE) { + LOG.error(ACTION_ABORT, writerId, topic, writeSemantic, kafkaException.getMessage()); + sendExceptionRef.compareAndSet(null, kafkaException); + } else { + LOG.warn(ACTION_CARRY_ON, writerId, topic, writeSemantic); + } + } + } + + @Override public void close(boolean abort) throws IOException { + if (abort) { + LOG.info("Aborting is set to TRUE, Closing writerId [{}] without flush.", writerId); + producer.close(0, TimeUnit.MICROSECONDS); + return; + } else { + LOG.info("Flushing Kafka Producer with writerId [{}]", writerId); + producer.flush(); + LOG.info("Closing WriterId [{}]", writerId); + producer.close(); + } + LOG.info("Closed WriterId [{}] Delivery semantic [{}], Topic[{}], Total sent Records [{}], Total Lost Records [{}]", + writerId, writeSemantic, + topic, + sentRecords, + lostRecords.get()); + checkExceptions(); + } + + @VisibleForTesting String getWriterId() { + return writerId; + } + + @VisibleForTesting long getLostRecords() { + return lostRecords.get(); + } + + @VisibleForTesting long getSentRecords() { + return sentRecords; + } + + @Override public void write(BytesWritable bytesWritable, KafkaWritable kafkaWritable) throws IOException { + this.write(kafkaWritable); + } + + @Override public void close(Reporter reporter) throws IOException { + this.close(false); + } + + private void checkExceptions() throws IOException { + if (sendExceptionRef.get() != null) { + LOG.error("Send Exception Aborting write from writerId [{}]", writerId); + producer.close(0, TimeUnit.MICROSECONDS); + throw new IOException(sendExceptionRef.get()); + } + } + +} diff --git kafka-handler/src/java/org/apache/hadoop/hive/kafka/TransactionalKafkaWriter.java kafka-handler/src/java/org/apache/hadoop/hive/kafka/TransactionalKafkaWriter.java new file mode 100644 index 0000000000..fb4d034b2e --- /dev/null +++ kafka-handler/src/java/org/apache/hadoop/hive/kafka/TransactionalKafkaWriter.java @@ -0,0 +1,363 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.hadoop.hive.kafka; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableMap; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.ql.exec.FileSinkOperator; +import org.apache.hadoop.io.BytesWritable; +import org.apache.hadoop.io.Writable; +import org.apache.hadoop.mapred.RecordWriter; +import org.apache.hadoop.mapred.Reporter; +import org.apache.kafka.clients.producer.Callback; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.AuthenticationException; +import org.apache.kafka.common.errors.OutOfOrderSequenceException; +import org.apache.kafka.common.errors.ProducerFencedException; +import org.apache.kafka.common.errors.TimeoutException; +import org.apache.kafka.common.serialization.ByteArraySerializer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; +import java.util.Properties; +import java.util.Set; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; +import java.util.stream.Collectors; + +/** + * Transactional Kafka Record Writer used to achieve Exactly once semantic. + */ +class TransactionalKafkaWriter implements FileSinkOperator.RecordWriter, RecordWriter { + + private static final Logger LOG = LoggerFactory.getLogger(TransactionalKafkaWriter.class); + private static final String TRANSACTION_DIR = "transaction_states"; + + private final String topic; + private final HiveKafkaProducer producer; + private final Callback callback; + private final AtomicReference sendExceptionRef = new AtomicReference<>(); + private final Path openTxFileName; + private final boolean optimisticCommit; + private final FileSystem fileSystem; + private final Map offsets = new HashMap<>(); + private final String writerIdTopicId; + private final long producerId; + private final short producerEpoch; + private long sentRecords = 0L; + + /** + * @param topic Kafka topic. + * @param producerProperties kafka producer properties. + * @param queryWorkingPath the Query working directory as, table_directory/hive_query_id. + * Used to store the state of the transaction and/or log sent records and partitions. + * for more information see: + * {@link KafkaStorageHandler#getQueryWorkingDir(org.apache.hadoop.hive.metastore.api.Table)} + * @param fileSystem file system handler. + * @param optimisticCommit if true the commit will happen at the task level otherwise will be delegated to HS2. + */ + TransactionalKafkaWriter(String topic, Properties producerProperties, + Path queryWorkingPath, + FileSystem fileSystem, + @Nullable Boolean optimisticCommit) { + this.fileSystem = fileSystem; + this.topic = Preconditions.checkNotNull(topic, "NULL topic !!"); + + Preconditions.checkState(producerProperties.getProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG) != null, + "set [" + ProducerConfig.BOOTSTRAP_SERVERS_CONFIG + "] property"); + producerProperties.setProperty(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName()); + producerProperties.setProperty(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName()); + this.producer = new HiveKafkaProducer<>(producerProperties); + this.optimisticCommit = optimisticCommit == null ? true : optimisticCommit; + this.callback = (metadata, exception) -> { + if (exception != null) { + sendExceptionRef.compareAndSet(null, exception); + } else { + //According to https://kafka.apache.org/0110/javadoc/org/apache/kafka/clients/producer/KafkaProducer.html + //Callbacks form the same TopicPartition will return in order thus this will keep track of most recent offset. + final TopicPartition tp = new TopicPartition(metadata.topic(), metadata.partition()); + offsets.put(tp, metadata.offset()); + } + }; + // Start Tx + assert producer.getTransactionalId() != null; + try { + producer.initTransactions(); + producer.beginTransaction(); + } catch (Exception exception) { + logHints(exception); + if (tryToAbortTx(exception)) { + LOG.error("Aborting Transaction [{}] cause by ERROR [{}]", + producer.getTransactionalId(), + exception.getMessage()); + producer.abortTransaction(); + } + LOG.error("Closing writer [{}] caused by ERROR [{}]", producer.getTransactionalId(), exception.getMessage()); + producer.close(0, TimeUnit.MILLISECONDS); + throw exception; + } + writerIdTopicId = String.format("WriterId [%s], Kafka Topic [%s]", producer.getTransactionalId(), topic); + producerEpoch = this.optimisticCommit ? -1 : producer.getEpoch(); + producerId = this.optimisticCommit ? -1 : producer.getProducerId(); + LOG.info("DONE with Initialization of {}, Epoch[{}], internal ID[{}]", writerIdTopicId, producerEpoch, producerId); + //Writer base working directory + openTxFileName = + this.optimisticCommit ? + null : + new Path(new Path(new Path(queryWorkingPath, TRANSACTION_DIR), producer.getTransactionalId()), + String.valueOf(producerEpoch)); + } + + @Override public void write(Writable w) throws IOException { + checkExceptions(); + try { + sentRecords++; + producer.send(KafkaUtils.toProducerRecord(topic, (KafkaWritable) w), callback); + } catch (Exception e) { + if (tryToAbortTx(e)) { + // producer.send() may throw a KafkaException which wraps a FencedException re throw its wrapped inner cause. + producer.abortTransaction(); + } + producer.close(0, TimeUnit.MILLISECONDS); + sendExceptionRef.compareAndSet(null, e); + checkExceptions(); + } + } + + private void logHints(Exception e) { + if (e instanceof TimeoutException) { + LOG.error("Maybe Try to increase [`retry.backoff.ms`] to avoid this error [{}].", e.getMessage()); + } + } + + /** + * The non Abort Close method can be split into 2 parts. + * Part one is to Flush to Kafka all the buffered Records then Log (Topic-Partition, Offset). + * Part two is To either commit the TX or Save the state of the TX to WAL and let HS2 do the commit. + * + * @param abort if set to true will abort flush and exit + * @throws IOException exception causing the failure + */ + @Override public void close(boolean abort) throws IOException { + if (abort) { + // Case Abort, try to AbortTransaction -> Close producer ASAP -> Exit; + LOG.warn("Aborting Transaction and Sending from {}", writerIdTopicId); + try { + producer.abortTransaction(); + } catch (Exception e) { + LOG.error("Aborting Transaction {} failed due to [{}]", writerIdTopicId, e.getMessage()); + } + producer.close(0, TimeUnit.MILLISECONDS); + return; + } + + // Normal Case -> lOG and Commit then Close + LOG.info("Flushing Kafka buffer of writerId {}", writerIdTopicId); + producer.flush(); + + // No exception good let's log to a file whatever Flushed. + String formattedMsg = "Topic[%s] Partition [%s] -> Last offset [%s]"; + String + flushedOffsetMsg = + offsets.entrySet() + .stream() + .map(topicPartitionLongEntry -> String.format(formattedMsg, + topicPartitionLongEntry.getKey().topic(), + topicPartitionLongEntry.getKey().partition(), + topicPartitionLongEntry.getValue())) + .collect(Collectors.joining(",")); + + LOG.info("WriterId {} flushed the following [{}] ", writerIdTopicId, flushedOffsetMsg); + // OPTIMISTIC COMMIT OR PERSIST STATE OF THE TX_WAL + checkExceptions(); + if (optimisticCommit) { + // Case Commit at the task level + commitTransaction(); + } else { + // Case delegate TX commit to HS2 + persistTxState(); + } + checkExceptions(); + producer.close(); + LOG.info("Closed writerId [{}], Sent [{}] records to Topic [{}]", + producer.getTransactionalId(), + sentRecords, + topic); + } + + private void commitTransaction() { + LOG.info("Attempting Optimistic commit by {}", writerIdTopicId); + try { + producer.commitTransaction(); + } catch (Exception e) { + sendExceptionRef.compareAndSet(null, e); + } + } + + /** + * Write the Kafka Consumer PID and Epoch to checkpoint file {@link TransactionalKafkaWriter#openTxFileName}. + */ + private void persistTxState() { + LOG.info("Committing state to path [{}] by [{}]", openTxFileName.toString(), writerIdTopicId); + try (FSDataOutputStream outStream = fileSystem.create(openTxFileName)) { + outStream.writeLong(producerId); + outStream.writeShort(producerEpoch); + } catch (Exception e) { + sendExceptionRef.compareAndSet(null, e); + } + } + + @Override public void write(BytesWritable bytesWritable, KafkaWritable kafkaWritable) throws IOException { + write(kafkaWritable); + } + + @Override public void close(Reporter reporter) throws IOException { + close(false); + } + + @VisibleForTesting long getSentRecords() { + return sentRecords; + } + + @VisibleForTesting short getProducerEpoch() { + return producerEpoch; + } + + @VisibleForTesting long getProducerId() { + return producerId; + } + + /** + * Checks for existing exception. In case of exception will close consumer and rethrow as IOException + * @throws IOException abort if possible, close consumer then rethrow exception. + */ + private void checkExceptions() throws IOException { + if (sendExceptionRef.get() != null && sendExceptionRef.get() instanceof KafkaException && sendExceptionRef.get() + .getCause() instanceof ProducerFencedException) { + // producer.send() may throw a KafkaException which wraps a FencedException re throw its wrapped inner cause. + sendExceptionRef.updateAndGet(e -> (KafkaException) e.getCause()); + } + if (sendExceptionRef.get() != null) { + final Exception exception = sendExceptionRef.get(); + logHints(exception); + if (tryToAbortTx(exception)) { + LOG.error("Aborting Transaction [{}] cause by ERROR [{}]", writerIdTopicId, exception.getMessage()); + producer.abortTransaction(); + } + LOG.error("Closing writer [{}] caused by ERROR [{}]", writerIdTopicId, exception.getMessage()); + producer.close(0, TimeUnit.MILLISECONDS); + throw new IOException(exception); + } + } + + private boolean tryToAbortTx(Throwable e) { + // According to https://kafka.apache.org/0110/javadoc/org/apache/kafka/clients/producer/KafkaProducer.html + // We can't recover from these exceptions, so our only option is to close the producer and exit. + boolean + isNotFencedOut = + !(e instanceof ProducerFencedException) + && !(e instanceof OutOfOrderSequenceException) + && !(e instanceof AuthenticationException); + // producer.send() may throw a KafkaException which wraps a FencedException therefore check inner cause. + boolean causeIsNotFencedOut = !(e.getCause() != null && e.getCause() instanceof ProducerFencedException); + return isNotFencedOut && causeIsNotFencedOut; + } + + /** + * Given a query workingDirectory as table_directory/hive_query_id/ will fetch the open transaction states. + * Table directory is {@link org.apache.hadoop.hive.metastore.api.Table#getSd()#getLocation()}. + * Hive Query ID is inferred from the JobConf see {@link KafkaStorageHandler#getQueryId()}. + * + * The path to a transaction state is as follow. + * .../{@code queryWorkingDir}/{@code TRANSACTION_DIR}/{@code writerId}/{@code producerEpoch} + * + * The actual state is stored in the file {@code producerEpoch}. + * The file contains a {@link Long} as internal producer Id and a {@link Short} as the producer epoch. + * According to Kafka API, highest epoch corresponds to the active Producer, therefore if there is multiple + * {@code producerEpoch} files will pick the maximum based on {@link Short::compareTo}. + * + * @param fs File system handler. + * @param queryWorkingDir Query working Directory, see: + * {@link KafkaStorageHandler#getQueryWorkingDir(org.apache.hadoop.hive.metastore.api.Table)}. + * @return Map of Transaction Ids to Pair of Kafka Producer internal ID (Long) and producer epoch (short) + * @throws IOException if any of the IO operations fail. + */ + static Map> getTransactionsState(FileSystem fs, Path queryWorkingDir) throws IOException { + //list all current Dir + final Path transactionWorkingDir = new Path(queryWorkingDir, TRANSACTION_DIR); + final FileStatus[] files = fs.listStatus(transactionWorkingDir); + final Set + transactionSet = + Arrays.stream(files).filter(FileStatus::isDirectory).collect(Collectors.toSet()); + Set setOfTxPath = transactionSet.stream().map(FileStatus::getPath).collect(Collectors.toSet()); + ImmutableMap.Builder> builder = ImmutableMap.builder(); + setOfTxPath.forEach(path -> { + final String txId = path.getName(); + try { + FileStatus[] epochFiles = fs.listStatus(path); + // List all the Epoch if any and select the max. + // According to Kafka API recent venison of Producer with the same TxID will have greater epoch and same PID. + Optional + maxEpoch = + Arrays.stream(epochFiles) + .filter(FileStatus::isFile) + .map(fileStatus -> Short.valueOf(fileStatus.getPath().getName())) + .max(Short::compareTo); + short + epoch = + maxEpoch.orElseThrow(() -> new RuntimeException("Missing sub directory epoch from directory [" + + path.toString() + + "]")); + Path openTxFileName = new Path(path, String.valueOf(epoch)); + long internalId; + try (FSDataInputStream inStream = fs.open(openTxFileName)) { + internalId = inStream.readLong(); + short fileEpoch = inStream.readShort(); + if (epoch != fileEpoch) { + throw new RuntimeException(String.format("Was expecting [%s] but got [%s] from path [%s]", + epoch, + fileEpoch, + path.toString())); + } + } catch (IOException e) { + throw new RuntimeException(e); + } + builder.put(txId, Pair.of(internalId, epoch)); + } catch (IOException e) { + throw new RuntimeException(e); + } + }); + return builder.build(); + } +} diff --git kafka-handler/src/test/org/apache/hadoop/hive/kafka/HiveKafkaProducerTest.java kafka-handler/src/test/org/apache/hadoop/hive/kafka/HiveKafkaProducerTest.java new file mode 100644 index 0000000000..db2515c8ed --- /dev/null +++ kafka-handler/src/test/org/apache/hadoop/hive/kafka/HiveKafkaProducerTest.java @@ -0,0 +1,179 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.hadoop.hive.kafka; + +import com.google.common.collect.ImmutableMap; +import org.apache.kafka.clients.consumer.ConsumerConfig; +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.clients.producer.ProducerConfig; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.serialization.ByteArrayDeserializer; +import org.apache.kafka.common.serialization.ByteArraySerializer; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.nio.charset.Charset; +import java.time.Duration; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Properties; +import java.util.UUID; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +/** + * Test class for Hive Kafka Producer. + */ +@SuppressWarnings("unchecked") public class HiveKafkaProducerTest { + + private static final Logger LOG = LoggerFactory.getLogger(HiveKafkaProducerTest.class); + private static final int RECORD_NUMBER = 17384; + private static final byte[] KEY_BYTES = "KEY".getBytes(Charset.forName("UTF-8")); + private static final KafkaBrokerResource KAFKA_BROKER_RESOURCE = new KafkaBrokerResource(); + + private static final String TOPIC = "test-tx-producer"; + private static final List> + RECORDS = + IntStream.range(0, RECORD_NUMBER).mapToObj(number -> { + final byte[] value = ("VALUE-" + Integer.toString(number)).getBytes(Charset.forName("UTF-8")); + return new ProducerRecord<>(TOPIC, value, KEY_BYTES); + }).collect(Collectors.toList()); + + @BeforeClass public static void setupCluster() throws Throwable { + KAFKA_BROKER_RESOURCE.before(); + } + + @AfterClass public static void tearDownCluster() { + KAFKA_BROKER_RESOURCE.after(); + } + + private KafkaConsumer consumer; + private Properties producerProperties; + private HiveKafkaProducer producer; + + @Before public void setUp() { + LOG.info("setting up Config"); + Properties consumerProps = new Properties(); + consumerProps.setProperty("enable.auto.commit", "false"); + consumerProps.setProperty("auto.offset.reset", "none"); + consumerProps.setProperty("bootstrap.servers", KafkaBrokerResource.BROKER_IP_PORT); + consumerProps.setProperty("key.deserializer", ByteArrayDeserializer.class.getName()); + consumerProps.setProperty("value.deserializer", ByteArrayDeserializer.class.getName()); + consumerProps.setProperty("request.timeout.ms", "3002"); + consumerProps.setProperty("fetch.max.wait.ms", "3001"); + consumerProps.setProperty("session.timeout.ms", "3001"); + consumerProps.setProperty("metadata.max.age.ms", "100"); + consumerProps.setProperty(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "read_committed"); + this.consumer = new KafkaConsumer<>(consumerProps); + + String txId = UUID.randomUUID().toString(); + producerProperties = new Properties(); + producerProperties.setProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, KafkaBrokerResource.BROKER_IP_PORT); + producerProperties.setProperty(ProducerConfig.TRANSACTIONAL_ID_CONFIG, txId); + producerProperties.setProperty(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName()); + producerProperties.setProperty(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName()); + producer = new HiveKafkaProducer(producerProperties); + } + + @After public void tearDown() { + LOG.info("tearDown"); + consumer.close(); + consumer = null; + } + + @Test public void resumeTransaction() { + producer.initTransactions(); + producer.beginTransaction(); + long pid = producer.getProducerId(); + short epoch = producer.getEpoch(); + Assert.assertTrue(pid > -1); + Assert.assertTrue(epoch > -1); + //noinspection unchecked + RECORDS.forEach(producer::send); + producer.flush(); + producer.close(); + + HiveKafkaProducer secondProducer = new HiveKafkaProducer(producerProperties); + secondProducer.resumeTransaction(pid, epoch); + secondProducer.sendOffsetsToTransaction(ImmutableMap.of(), "__dummy_consumer_group"); + secondProducer.commitTransaction(); + secondProducer.close(); + + Collection assignment = Collections.singletonList(new TopicPartition(TOPIC, 0)); + consumer.assign(assignment); + consumer.seekToBeginning(assignment); + long numRecords = 0; + @SuppressWarnings("unchecked") final List> actualRecords = new ArrayList(); + while (numRecords < RECORD_NUMBER) { + ConsumerRecords consumerRecords = consumer.poll(Duration.ofMillis(1000)); + actualRecords.addAll(consumerRecords.records(new TopicPartition(TOPIC, 0))); + numRecords += consumerRecords.count(); + } + Assert.assertEquals("Size matters !!", RECORDS.size(), actualRecords.size()); + Iterator> expectedIt = RECORDS.iterator(); + Iterator> actualIt = actualRecords.iterator(); + while (expectedIt.hasNext()) { + ProducerRecord expected = expectedIt.next(); + ConsumerRecord actual = actualIt.next(); + Assert.assertArrayEquals("value not matching", expected.value(), actual.value()); + Assert.assertArrayEquals("key not matching", expected.key(), actual.key()); + } + } + + @Test(expected = org.apache.kafka.common.KafkaException.class) public void testWrongEpochAndId() { + HiveKafkaProducer secondProducer = new HiveKafkaProducer(producerProperties); + secondProducer.resumeTransaction(3434L, (short) 12); + secondProducer.sendOffsetsToTransaction(ImmutableMap.of(), "__dummy_consumer_group"); + secondProducer.close(); + } + + @Test(expected = org.apache.kafka.common.KafkaException.class) public void testWrongEpoch() { + producer.initTransactions(); + producer.beginTransaction(); + long pid = producer.getProducerId(); + producer.close(); + HiveKafkaProducer secondProducer = new HiveKafkaProducer(producerProperties); + secondProducer.resumeTransaction(pid, (short) 12); + secondProducer.sendOffsetsToTransaction(ImmutableMap.of(), "__dummy_consumer_group"); + secondProducer.close(); + } + + @Test(expected = org.apache.kafka.common.KafkaException.class) public void testWrongPID() { + producer.initTransactions(); + producer.beginTransaction(); + short epoch = producer.getEpoch(); + producer.close(); + HiveKafkaProducer secondProducer = new HiveKafkaProducer(producerProperties); + secondProducer.resumeTransaction(45L, epoch); + secondProducer.sendOffsetsToTransaction(ImmutableMap.of(), "__dummy_consumer_group"); + secondProducer.close(); + } +} diff --git kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaBrokerResource.java kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaBrokerResource.java new file mode 100644 index 0000000000..fbcbe9a19a --- /dev/null +++ kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaBrokerResource.java @@ -0,0 +1,100 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.hadoop.hive.kafka; + +import kafka.admin.RackAwareMode; +import kafka.server.KafkaConfig; +import kafka.server.KafkaServer; +import kafka.utils.TestUtils; +import kafka.zk.AdminZkClient; +import kafka.zk.EmbeddedZookeeper; +import org.apache.commons.io.FileUtils; +import org.apache.kafka.common.utils.Time; +import org.junit.rules.ExternalResource; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.Properties; + +/** + * Test Helper Class to start and stop a kafka broker. + */ +class KafkaBrokerResource extends ExternalResource { + private static final Logger LOG = LoggerFactory.getLogger(KafkaBrokerResource.class); + private static final String TOPIC = "TEST-CREATE_TOPIC"; + static final String BROKER_IP_PORT = "127.0.0.1:9092"; + private EmbeddedZookeeper zkServer; + private KafkaServer kafkaServer; + private AdminZkClient adminZkClient; + private Path tmpLogDir; + + /** + * Override to set up your specific external resource. + * + * @throws Throwable if setup fails (which will disable {@code after} + */ + @Override protected void before() throws Throwable { + // Start the ZK and the Broker + LOG.info("init embedded Zookeeper"); + zkServer = new EmbeddedZookeeper(); + tmpLogDir = Files.createTempDirectory("kafka-log-dir-").toAbsolutePath(); + String zkConnect = "127.0.0.1:" + zkServer.port(); + LOG.info("init kafka broker"); + Properties brokerProps = new Properties(); + brokerProps.setProperty("zookeeper.connect", zkConnect); + brokerProps.setProperty("broker.id", "0"); + brokerProps.setProperty("log.dir", tmpLogDir.toString()); + brokerProps.setProperty("listeners", "PLAINTEXT://" + BROKER_IP_PORT); + brokerProps.setProperty("offsets.topic.replication.factor", "1"); + brokerProps.setProperty("transaction.state.log.replication.factor", "1"); + brokerProps.setProperty("transaction.state.log.min.isr", "1"); + KafkaConfig config = new KafkaConfig(brokerProps); + kafkaServer = TestUtils.createServer(config, Time.SYSTEM); + kafkaServer.startup(); + kafkaServer.zkClient(); + adminZkClient = new AdminZkClient(kafkaServer.zkClient()); + LOG.info("Creating kafka TOPIC [{}]", TOPIC); + adminZkClient.createTopic(TOPIC, 1, 1, new Properties(), RackAwareMode.Disabled$.MODULE$); + } + + /** + * Override to tear down your specific external resource. + */ + @Override protected void after() { + super.after(); + try { + FileUtils.deleteDirectory(new File(tmpLogDir.toString())); + } catch (IOException e) { + LOG.error("Error cleaning " + tmpLogDir.toString(), e); + } + if (kafkaServer != null) { + kafkaServer.shutdown(); + kafkaServer.awaitShutdown(); + } + zkServer.shutdown(); + } + + void deleteTopic(@SuppressWarnings("SameParameterValue") String topic) { + adminZkClient.deleteTopic(topic); + } +} diff --git kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaInputSplitTest.java kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaInputSplitTest.java new file mode 100644 index 0000000000..6e95a541ec --- /dev/null +++ kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaInputSplitTest.java @@ -0,0 +1,126 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.hadoop.hive.kafka; + +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.DataInputBuffer; +import org.apache.hadoop.io.DataOutputBuffer; +import org.junit.Assert; +import org.junit.Test; + +import java.io.IOException; +import java.util.List; + +/** + * Kafka Hadoop InputSplit Test. + */ +public class KafkaInputSplitTest { + private final KafkaInputSplit expectedInputSplit; + + public KafkaInputSplitTest() { + String topic = "my_topic"; + this.expectedInputSplit = new KafkaInputSplit(topic, 1, 50L, 56L, new Path("/tmp")); + } + + @Test public void testWriteRead() throws IOException { + DataOutputBuffer output = new DataOutputBuffer(); + this.expectedInputSplit.write(output); + KafkaInputSplit kafkaInputSplit = new KafkaInputSplit(); + DataInputBuffer input = new DataInputBuffer(); + input.reset(output.getData(), 0, output.getLength()); + kafkaInputSplit.readFields(input); + Assert.assertEquals(this.expectedInputSplit, kafkaInputSplit); + } + + @Test public void andRangeOverLapping() { + KafkaInputSplit kafkaInputSplit = new KafkaInputSplit("test-topic", 2, 10, 400, new Path("/tmp")); + + KafkaInputSplit kafkaInputSplit2 = new KafkaInputSplit("test-topic", 2, 3, 200, new Path("/tmp")); + + Assert.assertEquals(new KafkaInputSplit("test-topic", 2, 10, 200, new Path("/tmp")), + KafkaInputSplit.intersectRange(kafkaInputSplit, kafkaInputSplit2)); + + } + + @Test public void andRangeNonOverLapping() { + KafkaInputSplit kafkaInputSplit = new KafkaInputSplit("test-topic", 2, 10, 400, new Path("/tmp")); + + KafkaInputSplit kafkaInputSplit2 = + new KafkaInputSplit("test-topic", 2, 550, 700, new Path("/tmp")); + + Assert.assertNull(KafkaInputSplit.intersectRange(kafkaInputSplit, kafkaInputSplit2)); + + } + + @Test public void orRange() { + KafkaInputSplit kafkaInputSplit = + new KafkaInputSplit("test-topic", 2, 300, 400, new Path("/tmp")); + + KafkaInputSplit kafkaInputSplit2 = new KafkaInputSplit("test-topic", 2, 3, 600, new Path("/tmp")); + + Assert.assertEquals(kafkaInputSplit2, + KafkaInputSplit.unionRange(kafkaInputSplit, kafkaInputSplit2)); + + KafkaInputSplit kafkaInputSplit3 = + new KafkaInputSplit("test-topic", 2, 700, 6000, new Path("/tmp")); + + Assert.assertEquals(new KafkaInputSplit("test-topic", 2, 300, 6000, new Path("/tmp")), + KafkaInputSplit.unionRange(kafkaInputSplit, kafkaInputSplit3)); + } + + @Test public void copyOf() { + KafkaInputSplit kafkaInputSplit = + new KafkaInputSplit("test-topic", 2, 300, 400, new Path("/tmp")); + + KafkaInputSplit copyOf = KafkaInputSplit.copyOf(kafkaInputSplit); + Assert.assertEquals(kafkaInputSplit, copyOf); + Assert.assertNotSame(kafkaInputSplit, copyOf); + } + + @Test public void testClone() { + KafkaInputSplit kafkaInputSplit = + new KafkaInputSplit("test-topic", 2, 300, 400, new Path("/tmp")); + + KafkaInputSplit clone = KafkaInputSplit.copyOf(kafkaInputSplit); + Assert.assertEquals(kafkaInputSplit, clone); + Assert.assertNotSame(clone, kafkaInputSplit); + + } + + @Test public void testSlice() { + KafkaInputSplit kafkaInputSplit = + new KafkaInputSplit("test-topic", 2, 300, 400, new Path("/tmp")); + List kafkaInputSplitList = KafkaInputSplit.slice(14, kafkaInputSplit); + Assert.assertEquals(kafkaInputSplitList.stream() + .mapToLong(kafkaPullerInputSplit1 -> kafkaPullerInputSplit1.getEndOffset() + - kafkaPullerInputSplit1.getStartOffset()) + .sum(), kafkaInputSplit.getEndOffset() - kafkaInputSplit.getStartOffset()); + Assert.assertEquals(1, + kafkaInputSplitList.stream() + .filter(kafkaPullerInputSplit1 -> kafkaInputSplit.getStartOffset() + == kafkaPullerInputSplit1.getStartOffset()) + .count()); + Assert.assertEquals(1, + kafkaInputSplitList.stream() + .filter(kafkaPullerInputSplit1 -> kafkaInputSplit.getEndOffset() + == kafkaPullerInputSplit1.getEndOffset()) + .count()); + + } +} diff --git kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaPullerInputSplitTest.java kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaPullerInputSplitTest.java deleted file mode 100644 index 00f95ca329..0000000000 --- kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaPullerInputSplitTest.java +++ /dev/null @@ -1,131 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you 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 org.apache.hadoop.hive.kafka; - -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.io.DataInputBuffer; -import org.apache.hadoop.io.DataOutputBuffer; -import org.junit.Assert; -import org.junit.Test; - -import java.io.DataInput; -import java.io.DataOutput; -import java.io.IOException; -import java.util.List; - -/** - * Kafka Hadoop InputSplit Test. - */ -public class KafkaPullerInputSplitTest { - private KafkaPullerInputSplit expectedInputSplit; - - public KafkaPullerInputSplitTest() { - String topic = "my_topic"; - this.expectedInputSplit = new KafkaPullerInputSplit(topic, 1, 50L, 56L, new Path("/tmp")); - } - - @Test public void testWriteRead() throws IOException { - DataOutput output = new DataOutputBuffer(); - this.expectedInputSplit.write(output); - KafkaPullerInputSplit kafkaPullerInputSplit = new KafkaPullerInputSplit(); - DataInput input = new DataInputBuffer(); - ((DataInputBuffer) input).reset(((DataOutputBuffer) output).getData(), 0, ((DataOutputBuffer) output).getLength()); - kafkaPullerInputSplit.readFields(input); - Assert.assertEquals(this.expectedInputSplit, kafkaPullerInputSplit); - } - - @Test public void andRangeOverLapping() { - KafkaPullerInputSplit kafkaPullerInputSplit = new KafkaPullerInputSplit("test-topic", 2, 10, 400, new Path("/tmp")); - - KafkaPullerInputSplit kafkaPullerInputSplit2 = new KafkaPullerInputSplit("test-topic", 2, 3, 200, new Path("/tmp")); - - Assert.assertEquals(new KafkaPullerInputSplit("test-topic", 2, 10, 200, new Path("/tmp")), - KafkaPullerInputSplit.intersectRange(kafkaPullerInputSplit, kafkaPullerInputSplit2)); - - } - - @Test public void andRangeNonOverLapping() { - KafkaPullerInputSplit kafkaPullerInputSplit = new KafkaPullerInputSplit("test-topic", 2, 10, 400, new Path("/tmp")); - - KafkaPullerInputSplit - kafkaPullerInputSplit2 = - new KafkaPullerInputSplit("test-topic", 2, 550, 700, new Path("/tmp")); - - Assert.assertEquals(null, KafkaPullerInputSplit.intersectRange(kafkaPullerInputSplit, kafkaPullerInputSplit2)); - - } - - @Test public void orRange() { - KafkaPullerInputSplit - kafkaPullerInputSplit = - new KafkaPullerInputSplit("test-topic", 2, 300, 400, new Path("/tmp")); - - KafkaPullerInputSplit kafkaPullerInputSplit2 = new KafkaPullerInputSplit("test-topic", 2, 3, 600, new Path("/tmp")); - - Assert.assertEquals(kafkaPullerInputSplit2, - KafkaPullerInputSplit.unionRange(kafkaPullerInputSplit, kafkaPullerInputSplit2)); - - KafkaPullerInputSplit - kafkaPullerInputSplit3 = - new KafkaPullerInputSplit("test-topic", 2, 700, 6000, new Path("/tmp")); - - Assert.assertEquals(new KafkaPullerInputSplit("test-topic", 2, 300, 6000, new Path("/tmp")), - KafkaPullerInputSplit.unionRange(kafkaPullerInputSplit, kafkaPullerInputSplit3)); - } - - @Test public void copyOf() { - KafkaPullerInputSplit - kafkaPullerInputSplit = - new KafkaPullerInputSplit("test-topic", 2, 300, 400, new Path("/tmp")); - - KafkaPullerInputSplit copyOf = KafkaPullerInputSplit.copyOf(kafkaPullerInputSplit); - Assert.assertEquals(kafkaPullerInputSplit, copyOf); - Assert.assertTrue(kafkaPullerInputSplit != copyOf); - } - - @Test public void testClone() { - KafkaPullerInputSplit - kafkaPullerInputSplit = - new KafkaPullerInputSplit("test-topic", 2, 300, 400, new Path("/tmp")); - - KafkaPullerInputSplit clone = kafkaPullerInputSplit.clone(); - Assert.assertEquals(kafkaPullerInputSplit, clone); - Assert.assertTrue(clone != kafkaPullerInputSplit); - - } - - @Test public void testSlice() { - KafkaPullerInputSplit - kafkaPullerInputSplit = - new KafkaPullerInputSplit("test-topic", 2, 300, 400, new Path("/tmp")); - List kafkaPullerInputSplitList = KafkaPullerInputSplit.slice(14, kafkaPullerInputSplit); - Assert.assertEquals(kafkaPullerInputSplitList.stream() - .mapToLong(kafkaPullerInputSplit1 -> kafkaPullerInputSplit1.getEndOffset() - - kafkaPullerInputSplit1.getStartOffset()) - .sum(), kafkaPullerInputSplit.getEndOffset() - kafkaPullerInputSplit.getStartOffset()); - Assert.assertTrue(kafkaPullerInputSplitList.stream() - .filter(kafkaPullerInputSplit1 -> kafkaPullerInputSplit.getStartOffset() - == kafkaPullerInputSplit1.getStartOffset()) - .count() == 1); - Assert.assertTrue(kafkaPullerInputSplitList.stream() - .filter(kafkaPullerInputSplit1 -> kafkaPullerInputSplit.getEndOffset() == kafkaPullerInputSplit1.getEndOffset()) - .count() == 1); - - } -} diff --git kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaRecordIteratorTest.java kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaRecordIteratorTest.java index e048fb3197..3d3f598bc0 100644 --- kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaRecordIteratorTest.java +++ kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaRecordIteratorTest.java @@ -19,158 +19,151 @@ package org.apache.hadoop.hive.kafka; import com.google.common.collect.ImmutableList; -import kafka.admin.AdminUtils; -import kafka.admin.RackAwareMode; -import kafka.server.KafkaConfig; -import kafka.server.KafkaServer; -import kafka.utils.MockTime; -import kafka.utils.TestUtils; -import kafka.utils.ZKStringSerializer$; -import kafka.utils.ZkUtils; -import kafka.zk.EmbeddedZookeeper; -import org.I0Itec.zkclient.ZkClient; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.mapreduce.TaskAttemptContext; import org.apache.hadoop.mapreduce.TaskAttemptID; import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl; +import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.KafkaConsumer; import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.serialization.ByteArrayDeserializer; -import org.apache.kafka.common.utils.Time; import org.junit.After; import org.junit.AfterClass; import org.junit.Assert; import org.junit.Before; import org.junit.BeforeClass; import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; +import javax.annotation.Nullable; import java.nio.charset.Charset; -import java.nio.file.Files; +import java.util.Arrays; import java.util.Iterator; import java.util.List; import java.util.Properties; +import java.util.UUID; import java.util.stream.Collectors; import java.util.stream.IntStream; /** * Kafka Iterator Tests. */ -public class KafkaRecordIteratorTest { +@RunWith(Parameterized.class) public class KafkaRecordIteratorTest { private static final Logger LOG = LoggerFactory.getLogger(KafkaRecordIteratorTest.class); - private static final int RECORD_NUMBER = 100; + private static final int RECORD_NUMBER = 19384; private static final String TOPIC = "my_test_topic"; - private static final TopicPartition TOPIC_PARTITION = new TopicPartition(TOPIC, 0); + private static final String TX_TOPIC = "tx_test_topic"; private static final byte[] KEY_BYTES = "KEY".getBytes(Charset.forName("UTF-8")); - private static final List> - RECORDS = - IntStream.range(0, RECORD_NUMBER).mapToObj(number -> { - final byte[] value = ("VALUE-" + Integer.toString(number)).getBytes(Charset.forName("UTF-8")); - return new ConsumerRecord<>(TOPIC, 0, (long) number, 0L, null, 0L, 0, 0, KEY_BYTES, value); - }).collect(Collectors.toList()); + + private static final KafkaBrokerResource BROKER_RESOURCE = new KafkaBrokerResource(); + private static final List> RECORDS = getRecords(TOPIC); + private static final List> TX_RECORDS = getRecords(TX_TOPIC); private static final long POLL_TIMEOUT_MS = 900L; - private static ZkUtils zkUtils; - private static ZkClient zkClient; private static KafkaProducer producer; - private static KafkaServer kafkaServer; + + @Parameterized.Parameters public static Iterable data() { + return Arrays.asList(new Object[][] {{TOPIC, true, RECORDS}, {TX_TOPIC, false, TX_RECORDS}}); + } + + private static List> getRecords(String topic) { + return IntStream.range(0, RECORD_NUMBER).mapToObj(number -> { + final byte[] value = ("VALUE-" + Integer.toString(number)).getBytes(Charset.forName("UTF-8")); + return new ConsumerRecord<>(topic, 0, (long) number, 0L, null, 0L, 0, 0, KEY_BYTES, value); + }).collect(Collectors.toList()); + } + + private final String currentTopic; + private final boolean readUncommitted; + private final List> expectedRecords; + private final TopicPartition topicPartition; private KafkaConsumer consumer = null; private KafkaRecordIterator kafkaRecordIterator = null; private final Configuration conf = new Configuration(); - private static EmbeddedZookeeper zkServer; - public KafkaRecordIteratorTest() { + public KafkaRecordIteratorTest(String currentTopic, + boolean readUncommitted, + List> expectedRecords) { + this.currentTopic = currentTopic; + // when true means the the topic is not Transactional topic + this.readUncommitted = readUncommitted; + this.expectedRecords = expectedRecords; + this.topicPartition = new TopicPartition(currentTopic, 0); } - @BeforeClass public static void setupCluster() throws IOException { - LOG.info("init embedded Zookeeper"); - zkServer = new EmbeddedZookeeper(); - String zkConnect = "127.0.0.1:" + zkServer.port(); - zkClient = new ZkClient(zkConnect, 3000, 3000, ZKStringSerializer$.MODULE$); - zkUtils = ZkUtils.apply(zkClient, false); - LOG.info("init kafka broker"); - Properties brokerProps = new Properties(); - brokerProps.setProperty("zookeeper.connect", zkConnect); - brokerProps.setProperty("broker.id", "0"); - brokerProps.setProperty("log.dir", Files.createTempDirectory("kafka-log-dir-").toAbsolutePath().toString()); - brokerProps.setProperty("listeners", "PLAINTEXT://127.0.0.1:9092"); - brokerProps.setProperty("offsets.TOPIC.replication.factor", "1"); - KafkaConfig config = new KafkaConfig(brokerProps); - Time mock = new MockTime(); - kafkaServer = TestUtils.createServer(config, mock); - kafkaServer.startup(); - LOG.info("Creating kafka TOPIC [{}]", TOPIC); - AdminUtils.createTopic(zkUtils, TOPIC, 1, 1, new Properties(), RackAwareMode.Disabled$.MODULE$); - setupProducer(); - sendData(); + @BeforeClass public static void setupCluster() throws Throwable { + BROKER_RESOURCE.before(); + sendData(RECORDS, null); + sendData(TX_RECORDS, UUID.randomUUID().toString()); } @Before public void setUp() { LOG.info("setting up consumer"); - this.setupConsumer(); + setupConsumer(); this.kafkaRecordIterator = null; } @Test public void testHasNextAbsoluteStartEnd() { - this.kafkaRecordIterator = new KafkaRecordIterator(this.consumer, TOPIC_PARTITION, 0L, (long) RECORDS.size(), POLL_TIMEOUT_MS); - this.compareIterator(RECORDS, this.kafkaRecordIterator); + this.kafkaRecordIterator = + new KafkaRecordIterator(this.consumer, topicPartition, 0L, (long) expectedRecords.size(), POLL_TIMEOUT_MS); + this.compareIterator(expectedRecords, this.kafkaRecordIterator); } @Test public void testHasNextGivenStartEnd() { - this.kafkaRecordIterator = new KafkaRecordIterator(this.consumer, TOPIC_PARTITION, 2L, 4L, POLL_TIMEOUT_MS); - this.compareIterator(RECORDS.stream() + this.kafkaRecordIterator = new KafkaRecordIterator(this.consumer, topicPartition, 2L, 4L, POLL_TIMEOUT_MS); + this.compareIterator(expectedRecords.stream() .filter((consumerRecord) -> consumerRecord.offset() >= 2L && consumerRecord.offset() < 4L) .collect(Collectors.toList()), this.kafkaRecordIterator); } @Test public void testHasNextNoOffsets() { - this.kafkaRecordIterator = new KafkaRecordIterator(this.consumer, TOPIC_PARTITION, POLL_TIMEOUT_MS); - this.compareIterator(RECORDS, this.kafkaRecordIterator); + this.kafkaRecordIterator = new KafkaRecordIterator(this.consumer, topicPartition, POLL_TIMEOUT_MS); + this.compareIterator(expectedRecords, this.kafkaRecordIterator); } @Test public void testHasNextLastRecord() { - long startOffset = (long) (RECORDS.size() - 1); - long lastOffset = (long) RECORDS.size(); + long startOffset = (long) (expectedRecords.size() - 1); + long lastOffset = (long) expectedRecords.size(); this.kafkaRecordIterator = - new KafkaRecordIterator(this.consumer, TOPIC_PARTITION, startOffset, lastOffset, POLL_TIMEOUT_MS); - this.compareIterator(RECORDS.stream() + new KafkaRecordIterator(this.consumer, topicPartition, startOffset, lastOffset, POLL_TIMEOUT_MS); + this.compareIterator(expectedRecords.stream() .filter((consumerRecord) -> consumerRecord.offset() >= startOffset && consumerRecord.offset() < lastOffset) .collect(Collectors.toList()), this.kafkaRecordIterator); } @Test public void testHasNextFirstRecord() { - this.kafkaRecordIterator = new KafkaRecordIterator(this.consumer, TOPIC_PARTITION, 0L, 1L, POLL_TIMEOUT_MS); - this.compareIterator(RECORDS.stream() + this.kafkaRecordIterator = new KafkaRecordIterator(this.consumer, topicPartition, 0L, 1L, POLL_TIMEOUT_MS); + this.compareIterator(expectedRecords.stream() .filter((consumerRecord) -> consumerRecord.offset() >= 0L && consumerRecord.offset() < 1L) .collect(Collectors.toList()), this.kafkaRecordIterator); } @Test public void testHasNextNoStart() { - this.kafkaRecordIterator = - new KafkaRecordIterator(this.consumer, TOPIC_PARTITION, null, 10L, POLL_TIMEOUT_MS); - this.compareIterator(RECORDS.stream() + this.kafkaRecordIterator = new KafkaRecordIterator(this.consumer, topicPartition, null, 10L, POLL_TIMEOUT_MS); + this.compareIterator(expectedRecords.stream() .filter((consumerRecord) -> consumerRecord.offset() >= 0L && consumerRecord.offset() < 10L) .collect(Collectors.toList()), this.kafkaRecordIterator); } @Test public void testHasNextNoEnd() { - long lastOffset = (long) RECORDS.size(); - this.kafkaRecordIterator = - new KafkaRecordIterator(this.consumer, TOPIC_PARTITION, 5L, null, POLL_TIMEOUT_MS); - this.compareIterator(RECORDS.stream() + long lastOffset = (long) expectedRecords.size(); + this.kafkaRecordIterator = new KafkaRecordIterator(this.consumer, topicPartition, 5L, null, POLL_TIMEOUT_MS); + this.compareIterator(expectedRecords.stream() .filter((consumerRecord) -> consumerRecord.offset() >= 5L && consumerRecord.offset() < lastOffset) .collect(Collectors.toList()), this.kafkaRecordIterator); } - @Test public void testRecordReader() throws IOException { - List + @Test public void testRecordReader() { + List serRecords = - RECORDS.stream() - .map((consumerRecord) -> new KafkaRecordWritable(consumerRecord.partition(), + expectedRecords.stream() + .map((consumerRecord) -> new KafkaWritable(consumerRecord.partition(), consumerRecord.offset(), consumerRecord.timestamp(), consumerRecord.value(), @@ -178,12 +171,12 @@ public KafkaRecordIteratorTest() { 100L, consumerRecord.key())) .collect(Collectors.toList()); - KafkaPullerRecordReader recordReader = new KafkaPullerRecordReader(); + KafkaRecordReader recordReader = new KafkaRecordReader(); TaskAttemptContext context = new TaskAttemptContextImpl(this.conf, new TaskAttemptID()); - recordReader.initialize(new KafkaPullerInputSplit(TOPIC, 0, 50L, 100L, null), context); + recordReader.initialize(new KafkaInputSplit(currentTopic, 0, 50L, 100L, null), context); for (int i = 50; i < 100; ++i) { - KafkaRecordWritable record = new KafkaRecordWritable(); + KafkaWritable record = new KafkaWritable(); Assert.assertTrue(recordReader.next(null, record)); Assert.assertEquals(serRecords.get(i), record); } @@ -191,106 +184,124 @@ public KafkaRecordIteratorTest() { recordReader.close(); } - @Test(expected = IllegalStateException.class) public void testPullingBeyondLimit() { + @Test(expected = KafkaRecordIterator.PollTimeoutException.class) public void testPullingBeyondLimit() { //FYI In the Tx world Commits can introduce offset gaps therefore - //this (RECORD_NUMBER + 1) as beyond limit offset is only true if the topic has not Tx or any Control msg. - this.kafkaRecordIterator = new KafkaRecordIterator(this.consumer, TOPIC_PARTITION, 19383L, (long) RECORD_NUMBER + 1, POLL_TIMEOUT_MS); - this.compareIterator(RECORDS.stream() - .filter((consumerRecord) -> consumerRecord.offset() >= 19383L) + //thus(RECORD_NUMBER + 1) as beyond limit offset is only true if the topic has not Tx or any Control msg. + long increment = readUncommitted ? 1 : 2; + long requestedEnd = expectedRecords.size() + increment; + long requestedStart = expectedRecords.size() - 1; + this.kafkaRecordIterator = + new KafkaRecordIterator(this.consumer, topicPartition, requestedStart, requestedEnd, POLL_TIMEOUT_MS); + this.compareIterator(expectedRecords.stream() + .filter((consumerRecord) -> consumerRecord.offset() >= requestedStart) .collect(Collectors.toList()), this.kafkaRecordIterator); } @Test(expected = IllegalStateException.class) public void testPullingStartGreaterThanEnd() { - this.kafkaRecordIterator = new KafkaRecordIterator(this.consumer, TOPIC_PARTITION, 10L, 1L, POLL_TIMEOUT_MS); - this.compareIterator(RECORDS, this.kafkaRecordIterator); + this.kafkaRecordIterator = new KafkaRecordIterator(this.consumer, topicPartition, 10L, 1L, POLL_TIMEOUT_MS); + this.compareIterator(expectedRecords, this.kafkaRecordIterator); } - @Test(expected = IllegalStateException.class) public void testPullingFromEmptyTopic() { + @Test(expected = KafkaRecordIterator.PollTimeoutException.class) public void testPullingFromEmptyTopic() { this.kafkaRecordIterator = new KafkaRecordIterator(this.consumer, new TopicPartition("noHere", 0), 0L, 100L, POLL_TIMEOUT_MS); - this.compareIterator(RECORDS, this.kafkaRecordIterator); + this.compareIterator(expectedRecords, this.kafkaRecordIterator); } - @Test(expected = IllegalStateException.class) public void testPullingFromEmptyPartition() { + @Test(expected = KafkaRecordIterator.PollTimeoutException.class) public void testPullingFromEmptyPartition() { this.kafkaRecordIterator = - new KafkaRecordIterator(this.consumer, new TopicPartition(TOPIC, 1), 0L, 100L, POLL_TIMEOUT_MS); - this.compareIterator(RECORDS, this.kafkaRecordIterator); + new KafkaRecordIterator(this.consumer, new TopicPartition(currentTopic, 1), 0L, 100L, POLL_TIMEOUT_MS); + this.compareIterator(expectedRecords, this.kafkaRecordIterator); } @Test public void testStartIsEqualEnd() { - this.kafkaRecordIterator = new KafkaRecordIterator(this.consumer, TOPIC_PARTITION, 10L, 10L, POLL_TIMEOUT_MS); + this.kafkaRecordIterator = new KafkaRecordIterator(this.consumer, topicPartition, 10L, 10L, POLL_TIMEOUT_MS); this.compareIterator(ImmutableList.of(), this.kafkaRecordIterator); } @Test public void testStartIsTheLastOffset() { this.kafkaRecordIterator = new KafkaRecordIterator(this.consumer, - TOPIC_PARTITION, - (long) RECORD_NUMBER, - (long) RECORD_NUMBER, + topicPartition, + (long) expectedRecords.size(), + (long) expectedRecords.size(), POLL_TIMEOUT_MS); this.compareIterator(ImmutableList.of(), this.kafkaRecordIterator); } @Test public void testStartIsTheFirstOffset() { - this.kafkaRecordIterator = new KafkaRecordIterator(this.consumer, TOPIC_PARTITION, 0L, 0L, POLL_TIMEOUT_MS); + this.kafkaRecordIterator = new KafkaRecordIterator(this.consumer, topicPartition, 0L, 0L, POLL_TIMEOUT_MS); this.compareIterator(ImmutableList.of(), this.kafkaRecordIterator); } private void compareIterator(List> expected, Iterator> kafkaRecordIterator) { expected.forEach((expectedRecord) -> { - Assert.assertTrue("record with offset " + expectedRecord.offset(), kafkaRecordIterator.hasNext()); + Assert.assertTrue("Record with offset is missing" + expectedRecord.offset(), kafkaRecordIterator.hasNext()); ConsumerRecord record = kafkaRecordIterator.next(); - Assert.assertEquals(record.topic(), TOPIC); - Assert.assertEquals(0, record.partition()); + Assert.assertEquals(expectedRecord.topic(), record.topic()); + Assert.assertEquals(expectedRecord.partition(), record.partition()); Assert.assertEquals("Offsets not matching", expectedRecord.offset(), record.offset()); byte[] binaryExceptedValue = expectedRecord.value(); byte[] binaryExceptedKey = expectedRecord.key(); byte[] binaryValue = (byte[]) record.value(); byte[] binaryKey = (byte[]) record.key(); - Assert.assertArrayEquals(binaryExceptedValue, binaryValue); - Assert.assertArrayEquals(binaryExceptedKey, binaryKey); + Assert.assertArrayEquals("Values not matching", binaryExceptedValue, binaryValue); + Assert.assertArrayEquals("Keys not matching", binaryExceptedKey, binaryKey); }); Assert.assertFalse(kafkaRecordIterator.hasNext()); } - private static void setupProducer() { - LOG.info("Setting up kafka producer"); - Properties producerProps = new Properties(); - producerProps.setProperty("bootstrap.servers", "127.0.0.1:9092"); - producerProps.setProperty("key.serializer", "org.apache.kafka.common.serialization.ByteArraySerializer"); - producerProps.setProperty("value.serializer", "org.apache.kafka.common.serialization.ByteArraySerializer"); - producerProps.setProperty("max.block.ms", "10000"); - producer = new KafkaProducer<>(producerProps); - LOG.info("kafka producer started"); - } - private void setupConsumer() { Properties consumerProps = new Properties(); consumerProps.setProperty("enable.auto.commit", "false"); consumerProps.setProperty("auto.offset.reset", "none"); - consumerProps.setProperty("bootstrap.servers", "127.0.0.1:9092"); - conf.set("kafka.bootstrap.servers", "127.0.0.1:9092"); + consumerProps.setProperty("bootstrap.servers", KafkaBrokerResource.BROKER_IP_PORT); + conf.set("kafka.bootstrap.servers", KafkaBrokerResource.BROKER_IP_PORT); + conf.set(KafkaTableProperties.KAFKA_POLL_TIMEOUT.getName(), + KafkaTableProperties.KAFKA_POLL_TIMEOUT.getDefaultValue()); + consumerProps.setProperty("key.deserializer", ByteArrayDeserializer.class.getName()); consumerProps.setProperty("value.deserializer", ByteArrayDeserializer.class.getName()); consumerProps.setProperty("request.timeout.ms", "3002"); consumerProps.setProperty("fetch.max.wait.ms", "3001"); consumerProps.setProperty("session.timeout.ms", "3001"); consumerProps.setProperty("metadata.max.age.ms", "100"); + if (!readUncommitted) { + consumerProps.setProperty(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "read_committed"); + conf.set("kafka.consumer.isolation.level", "read_committed"); + } consumerProps.setProperty("max.poll.records", String.valueOf(RECORD_NUMBER - 1)); this.consumer = new KafkaConsumer<>(consumerProps); } - private static void sendData() { + private static void sendData(List> recordList, @Nullable String txId) { + LOG.info("Setting up kafka producer"); + Properties producerProps = new Properties(); + producerProps.setProperty("bootstrap.servers", KafkaBrokerResource.BROKER_IP_PORT); + producerProps.setProperty("key.serializer", "org.apache.kafka.common.serialization.ByteArraySerializer"); + producerProps.setProperty("value.serializer", "org.apache.kafka.common.serialization.ByteArraySerializer"); + producerProps.setProperty("max.block.ms", "10000"); + if (txId != null) { + producerProps.setProperty(ProducerConfig.TRANSACTIONAL_ID_CONFIG, txId); + } + producer = new KafkaProducer<>(producerProps); + LOG.info("kafka producer started"); LOG.info("Sending [{}] records", RECORDS.size()); - RECORDS.stream() + if (txId != null) { + producer.initTransactions(); + producer.beginTransaction(); + } + recordList.stream() .map(consumerRecord -> new ProducerRecord<>(consumerRecord.topic(), consumerRecord.partition(), consumerRecord.timestamp(), consumerRecord.key(), consumerRecord.value())) .forEach(producerRecord -> producer.send(producerRecord)); + if (txId != null) { + producer.commitTransaction(); + } producer.close(); } @@ -302,13 +313,6 @@ private static void sendData() { } @AfterClass public static void tearDownCluster() { - if (kafkaServer != null) { - kafkaServer.shutdown(); - kafkaServer.zkUtils().close(); - kafkaServer.awaitShutdown(); - } - zkServer.shutdown(); - zkClient.close(); - zkUtils.close(); + BROKER_RESOURCE.after(); } } diff --git kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaScanTrimmerTest.java kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaScanTrimmerTest.java index 2a40bff4b5..7b1a1ae8ec 100644 --- kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaScanTrimmerTest.java +++ kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaScanTrimmerTest.java @@ -59,30 +59,30 @@ private final ExprNodeDesc seventyFiveLong = ConstantExprBuilder.build(75L); private final ExprNodeDesc fortyLong = ConstantExprBuilder.build(40L); - private ExprNodeDesc + private final ExprNodeDesc partitionColumn = new ExprNodeColumnDesc(TypeInfoFactory.intTypeInfo, - KafkaStreamingUtils.MetadataColumn.PARTITION.getName(), + MetadataColumn.PARTITION.getName(), null, false); - private ExprNodeDesc + private final ExprNodeDesc offsetColumn = new ExprNodeColumnDesc(TypeInfoFactory.longTypeInfo, - KafkaStreamingUtils.MetadataColumn.OFFSET.getName(), + MetadataColumn.OFFSET.getName(), null, false); - private String topic = "my_topic"; - private Map + private final String topic = "my_topic"; + private final Map fullHouse = ImmutableMap.of(new TopicPartition(topic, 0), - new KafkaPullerInputSplit(topic, 0, 0, 45, PATH), + new KafkaInputSplit(topic, 0, 0, 45, PATH), new TopicPartition(topic, 1), - new KafkaPullerInputSplit(topic, 1, 5, 1005, PATH), + new KafkaInputSplit(topic, 1, 5, 1005, PATH), new TopicPartition(topic, 2), - new KafkaPullerInputSplit(topic, 2, 9, 100, PATH), + new KafkaInputSplit(topic, 2, 9, 100, PATH), new TopicPartition(topic, 3), - new KafkaPullerInputSplit(topic, 3, 0, 100, PATH)); + new KafkaInputSplit(topic, 3, 0, 100, PATH)); @Test public void computeOptimizedScanPartitionBinaryOpFilter() { KafkaScanTrimmer kafkaScanTrimmer = new KafkaScanTrimmer(fullHouse, null); @@ -192,8 +192,8 @@ .deserializeExpression(SerializationUtilities.serializeExpression(orExpression))); TopicPartition tpZero = new TopicPartition(topic, 0); TopicPartition toThree = new TopicPartition(topic, 3); - KafkaPullerInputSplit split1 = new KafkaPullerInputSplit(topic, 0, 30, 41, PATH); - KafkaPullerInputSplit split2 = new KafkaPullerInputSplit(topic, 3, 35, 75, PATH); + KafkaInputSplit split1 = new KafkaInputSplit(topic, 0, 30, 41, PATH); + KafkaInputSplit split2 = new KafkaInputSplit(topic, 3, 35, 75, PATH); Map expected = ImmutableMap.of(tpZero, split1, toThree, split2); Assert.assertEquals(expected, actual); @@ -354,7 +354,7 @@ Map expected = Maps.transformValues(fullHouse, - entry -> new KafkaPullerInputSplit(Objects.requireNonNull(entry).getTopic(), + entry -> new KafkaInputSplit(Objects.requireNonNull(entry).getTopic(), entry.getPartition(), constantOffset, constantOffset + 1, @@ -367,7 +367,7 @@ KafkaScanTrimmer.buildScanFromOffsetPredicate(fullHouse, PredicateLeaf.Operator.EQUALS, 3000000L, false, false); expected = Maps.transformValues(fullHouse, - entry -> new KafkaPullerInputSplit(Objects.requireNonNull(entry).getTopic(), + entry -> new KafkaInputSplit(Objects.requireNonNull(entry).getTopic(), entry.getPartition(), entry.getEndOffset(), entry.getEndOffset(), @@ -379,7 +379,7 @@ expected = Maps.transformValues(fullHouse, - entry -> new KafkaPullerInputSplit(Objects.requireNonNull(entry).getTopic(), + entry -> new KafkaInputSplit(Objects.requireNonNull(entry).getTopic(), entry.getPartition(), entry.getStartOffset() > 0 ? entry.getEndOffset() : 0, entry.getStartOffset() > 0 ? entry.getEndOffset() : 1, @@ -402,7 +402,7 @@ Map expected = Maps.transformValues(fullHouse, - entry -> new KafkaPullerInputSplit(Objects.requireNonNull(entry).getTopic(), + entry -> new KafkaInputSplit(Objects.requireNonNull(entry).getTopic(), entry.getPartition(), entry.getStartOffset(), Math.min(constantOffset, entry.getEndOffset()), @@ -419,7 +419,7 @@ expected = Maps.transformValues(fullHouse, - entry -> new KafkaPullerInputSplit(Objects.requireNonNull(entry).getTopic(), + entry -> new KafkaInputSplit(Objects.requireNonNull(entry).getTopic(), entry.getPartition(), Math.min(entry.getEndOffset(), Math.max(entry.getStartOffset(), constantOffset + 1)), entry.getEndOffset(), @@ -436,7 +436,7 @@ expected = Maps.transformValues(fullHouse, - entry -> new KafkaPullerInputSplit(Objects.requireNonNull(entry).getTopic(), + entry -> new KafkaInputSplit(Objects.requireNonNull(entry).getTopic(), entry.getPartition(), Math.min(entry.getEndOffset(), Math.max(entry.getStartOffset(), constantOffset)), entry.getEndOffset(), @@ -453,7 +453,7 @@ expected = Maps.transformValues(fullHouse, - entry -> new KafkaPullerInputSplit(Objects.requireNonNull(entry).getTopic(), + entry -> new KafkaInputSplit(Objects.requireNonNull(entry).getTopic(), entry.getPartition(), entry.getStartOffset(), Math.min(constantOffset + 1, entry.getEndOffset()), @@ -476,7 +476,7 @@ Map expected = Maps.transformValues(fullHouse, - entry -> new KafkaPullerInputSplit(Objects.requireNonNull(entry).getTopic(), + entry -> new KafkaInputSplit(Objects.requireNonNull(entry).getTopic(), entry.getPartition(), entry.getStartOffset(), Math.min(constantOffset + 1, entry.getEndOffset()), @@ -493,7 +493,7 @@ expected = Maps.transformValues(fullHouse, - entry -> new KafkaPullerInputSplit(Objects.requireNonNull(entry).getTopic(), + entry -> new KafkaInputSplit(Objects.requireNonNull(entry).getTopic(), entry.getPartition(), Math.min(entry.getEndOffset(), Math.max(entry.getStartOffset(), constantOffset)), entry.getEndOffset(), @@ -510,7 +510,7 @@ expected = Maps.transformValues(fullHouse, - entry -> new KafkaPullerInputSplit(Objects.requireNonNull(entry).getTopic(), + entry -> new KafkaInputSplit(Objects.requireNonNull(entry).getTopic(), entry.getPartition(), Math.min(entry.getEndOffset(), Math.max(entry.getStartOffset(), constantOffset + 1)), entry.getEndOffset(), @@ -527,7 +527,7 @@ expected = Maps.transformValues(fullHouse, - entry -> new KafkaPullerInputSplit(Objects.requireNonNull(entry).getTopic(), + entry -> new KafkaInputSplit(Objects.requireNonNull(entry).getTopic(), entry.getPartition(), entry.getStartOffset(), Math.min(constantOffset, entry.getEndOffset()), diff --git kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaStorageHandlerTest.java kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaStorageHandlerTest.java new file mode 100644 index 0000000000..5c64e2bcae --- /dev/null +++ kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaStorageHandlerTest.java @@ -0,0 +1,145 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.hadoop.hive.kafka; + +import org.apache.hadoop.hive.metastore.TableType; +import org.apache.hadoop.hive.metastore.api.MetaException; +import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.hadoop.hive.ql.exec.Utilities; +import org.apache.hadoop.hive.ql.plan.TableDesc; +import org.apache.hadoop.hive.serde2.avro.AvroSerDe; +import org.apache.hadoop.mapred.JobConf; +import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.junit.Assert; +import org.junit.Test; +import org.mockito.Mockito; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; +import java.util.Properties; + +/** + * Test class for properties setting. + */ +public class KafkaStorageHandlerTest { + + private static final String TEST_TOPIC = "test-topic"; + private static final String LOCALHOST_9291 = "localhost:9291"; + + @Test public void configureJobPropertiesWithDefaultValues() throws MetaException { + KafkaStorageHandler kafkaStorageHandler = new KafkaStorageHandler(); + TableDesc tableDesc = Mockito.mock(TableDesc.class); + Properties properties = new Properties(); + Table preCreateTable = new Table(); + preCreateTable.putToParameters(KafkaTableProperties.HIVE_KAFKA_TOPIC.getName(), TEST_TOPIC); + preCreateTable.putToParameters(KafkaTableProperties.HIVE_KAFKA_BOOTSTRAP_SERVERS.getName(), LOCALHOST_9291); + preCreateTable.setTableType(TableType.EXTERNAL_TABLE.toString()); + kafkaStorageHandler.preCreateTable(preCreateTable); + preCreateTable.getParameters().forEach(properties::setProperty); + Mockito.when(tableDesc.getProperties()).thenReturn(properties); + Map jobProperties = new HashMap<>(); + kafkaStorageHandler.configureInputJobProperties(tableDesc, jobProperties); + kafkaStorageHandler.configureOutputJobProperties(tableDesc, jobProperties); + Assert.assertEquals(jobProperties.get(KafkaTableProperties.HIVE_KAFKA_TOPIC.getName()), TEST_TOPIC); + Assert.assertEquals(jobProperties.get(KafkaTableProperties.HIVE_KAFKA_BOOTSTRAP_SERVERS.getName()), LOCALHOST_9291); + Arrays.stream(KafkaTableProperties.values()) + .filter(key -> !key.isMandatory()) + .forEach((key) -> Assert.assertEquals("Wrong match for key " + key.getName(), + key.getDefaultValue(), + jobProperties.get(key.getName()))); + } + + @Test public void configureInputJobProperties() throws MetaException { + KafkaStorageHandler kafkaStorageHandler = new KafkaStorageHandler(); + TableDesc tableDesc = Mockito.mock(TableDesc.class); + Properties properties = new Properties(); + // set the mandatory properties + Table preCreateTable = new Table(); + preCreateTable.setTableType(TableType.EXTERNAL_TABLE.toString()); + preCreateTable.putToParameters(KafkaTableProperties.HIVE_KAFKA_TOPIC.getName(), TEST_TOPIC); + preCreateTable.putToParameters(KafkaTableProperties.HIVE_KAFKA_BOOTSTRAP_SERVERS.getName(), LOCALHOST_9291); + kafkaStorageHandler.preCreateTable(preCreateTable); + preCreateTable.getParameters().forEach(properties::setProperty); + properties.setProperty(KafkaTableProperties.WRITE_SEMANTIC_PROPERTY.getName(), + KafkaOutputFormat.WriteSemantic.EXACTLY_ONCE.name()); + properties.setProperty(KafkaTableProperties.SERDE_CLASS_NAME.getName(), AvroSerDe.class.getName()); + properties.setProperty(KafkaTableProperties.KAFKA_POLL_TIMEOUT.getName(), "7000"); + Mockito.when(tableDesc.getProperties()).thenReturn(properties); + Map jobProperties = new HashMap<>(); + kafkaStorageHandler.configureInputJobProperties(tableDesc, jobProperties); + + Assert.assertEquals(jobProperties.get(KafkaTableProperties.HIVE_KAFKA_TOPIC.getName()), TEST_TOPIC); + Assert.assertEquals(jobProperties.get(KafkaTableProperties.HIVE_KAFKA_BOOTSTRAP_SERVERS.getName()), LOCALHOST_9291); + Assert.assertEquals(AvroSerDe.class.getName(), jobProperties.get(KafkaTableProperties.SERDE_CLASS_NAME.getName())); + Assert.assertEquals("7000", jobProperties.get(KafkaTableProperties.KAFKA_POLL_TIMEOUT.getName())); + JobConf jobConf = new JobConf(); + jobProperties.forEach(jobConf::set); + jobConf.set("mapred.task.id", "task_id_test_0001"); + Properties kafkaProperties = KafkaUtils.consumerProperties(jobConf); + Assert.assertEquals(LOCALHOST_9291, kafkaProperties.get(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG)); + Assert.assertEquals("read_committed", kafkaProperties.get(ConsumerConfig.ISOLATION_LEVEL_CONFIG)); + Assert.assertEquals("false", kafkaProperties.get(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG)); + Assert.assertEquals("none", kafkaProperties.get(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG)); + Assert.assertEquals(Utilities.getTaskId(jobConf), kafkaProperties.get(CommonClientConfigs.CLIENT_ID_CONFIG)); + + } + + @Test public void configureOutJobProperties() throws MetaException { + KafkaStorageHandler kafkaStorageHandler = new KafkaStorageHandler(); + TableDesc tableDesc = Mockito.mock(TableDesc.class); + Properties properties = new Properties(); + // set the mandatory properties + Table preCreateTable = new Table(); + preCreateTable.putToParameters(KafkaTableProperties.HIVE_KAFKA_TOPIC.getName(), TEST_TOPIC); + preCreateTable.putToParameters(KafkaTableProperties.HIVE_KAFKA_BOOTSTRAP_SERVERS.getName(), LOCALHOST_9291); + preCreateTable.setTableType(TableType.EXTERNAL_TABLE.toString()); + kafkaStorageHandler.preCreateTable(preCreateTable); + preCreateTable.getParameters().forEach(properties::setProperty); + + properties.setProperty(KafkaTableProperties.WRITE_SEMANTIC_PROPERTY.getName(), + KafkaOutputFormat.WriteSemantic.EXACTLY_ONCE.name()); + properties.setProperty(KafkaTableProperties.HIVE_KAFKA_OPTIMISTIC_COMMIT.getName(), "false"); + Mockito.when(tableDesc.getProperties()).thenReturn(properties); + Map jobProperties = new HashMap<>(); + kafkaStorageHandler.configureOutputJobProperties(tableDesc, jobProperties); + Assert.assertEquals("false", jobProperties.get(KafkaTableProperties.HIVE_KAFKA_OPTIMISTIC_COMMIT.getName())); + Assert.assertEquals(jobProperties.get(KafkaTableProperties.HIVE_KAFKA_TOPIC.getName()), TEST_TOPIC); + Assert.assertEquals(jobProperties.get(KafkaTableProperties.HIVE_KAFKA_BOOTSTRAP_SERVERS.getName()), LOCALHOST_9291); + Assert.assertEquals(KafkaOutputFormat.WriteSemantic.EXACTLY_ONCE.name(), + jobProperties.get(KafkaTableProperties.WRITE_SEMANTIC_PROPERTY.getName())); + Assert.assertEquals(jobProperties.get(KafkaUtils.CONSUMER_CONFIGURATION_PREFIX + + "." + + ConsumerConfig.ISOLATION_LEVEL_CONFIG), "read_committed"); + Assert.assertEquals(KafkaOutputFormat.WriteSemantic.EXACTLY_ONCE.name(), + jobProperties.get(KafkaTableProperties.WRITE_SEMANTIC_PROPERTY.getName())); + + JobConf jobConf = new JobConf(); + jobProperties.forEach(jobConf::set); + jobConf.set("mapred.task.id", "task_id_test"); + Properties producerProperties = KafkaUtils.producerProperties(jobConf); + Assert.assertEquals(LOCALHOST_9291, producerProperties.get(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG)); + Assert.assertEquals("task_id_test", producerProperties.get(CommonClientConfigs.CLIENT_ID_CONFIG)); + Assert.assertEquals("all", producerProperties.get(ProducerConfig.ACKS_CONFIG)); + Assert.assertEquals(String.valueOf(Integer.MAX_VALUE), producerProperties.get(ProducerConfig.RETRIES_CONFIG)); + Assert.assertEquals("true", producerProperties.get(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG)); + } +} diff --git kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaStreamingUtilsTest.java kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaUtilsTest.java similarity index 65% rename from kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaStreamingUtilsTest.java rename to kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaUtilsTest.java index 071df3f858..8aebb9254e 100644 --- kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaStreamingUtilsTest.java +++ kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaUtilsTest.java @@ -31,15 +31,12 @@ import java.util.List; import java.util.Properties; import java.util.stream.Collectors; -import java.util.stream.Stream; - -import static org.apache.hadoop.hive.kafka.KafkaStreamingUtils.*; /** * Test for Utility class. */ -public class KafkaStreamingUtilsTest { - public KafkaStreamingUtilsTest() { +public class KafkaUtilsTest { + public KafkaUtilsTest() { } @Test public void testConsumerProperties() { @@ -47,7 +44,7 @@ public KafkaStreamingUtilsTest() { configuration.set("kafka.bootstrap.servers", "localhost:9090"); configuration.set("kafka.consumer.fetch.max.wait.ms", "40"); configuration.set("kafka.consumer.my.new.wait.ms", "400"); - Properties properties = consumerProperties(configuration); + Properties properties = KafkaUtils.consumerProperties(configuration); Assert.assertEquals("localhost:9090", properties.getProperty("bootstrap.servers")); Assert.assertEquals("40", properties.getProperty("fetch.max.wait.ms")); Assert.assertEquals("400", properties.getProperty("my.new.wait.ms")); @@ -57,14 +54,14 @@ public KafkaStreamingUtilsTest() { Configuration configuration = new Configuration(); configuration.set("kafka.bootstrap.servers", "localhost:9090"); configuration.set("kafka.consumer." + ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false"); - consumerProperties(configuration); + KafkaUtils.consumerProperties(configuration); } @Test(expected = IllegalArgumentException.class) public void canNotSetForbiddenProp2() { Configuration configuration = new Configuration(); configuration.set("kafka.bootstrap.servers", "localhost:9090"); configuration.set("kafka.consumer." + ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "value"); - consumerProperties(configuration); + KafkaUtils.consumerProperties(configuration); } @Test public void testMetadataEnumLookupMapper() { @@ -84,25 +81,44 @@ public KafkaStreamingUtilsTest() { new LongWritable(ts), new LongWritable(startOffset), new LongWritable(endOffset)); - KafkaRecordWritable KRWritable = new KafkaRecordWritable(partition, offset, ts, value, startOffset, endOffset, key); + KafkaWritable kafkaWritable = new KafkaWritable(partition, offset, ts, value, startOffset, endOffset, key); List actual = - KAFKA_METADATA_COLUMN_NAMES.stream() - .map(recordWritableFnMap::get) - .map(fn -> fn.apply(KRWritable)) + MetadataColumn.KAFKA_METADATA_COLUMN_NAMES.stream() + .map(MetadataColumn::forName) + .map(kafkaWritable::getHiveWritable) .collect(Collectors.toList()); Assert.assertEquals(expectedWritables, actual); } - @Test - public void testEnsureThatAllTheColumnAreListed() { - Assert.assertEquals(MetadataColumn.values().length, KafkaStreamingUtils.KAFKA_METADATA_COLUMN_NAMES.size()); - Assert.assertEquals(MetadataColumn.values().length, KafkaStreamingUtils.KAFKA_METADATA_INSPECTORS.size()); + @Test public void testEnsureThatAllTheColumnAreListed() { + Assert.assertEquals(MetadataColumn.values().length, MetadataColumn.KAFKA_METADATA_COLUMN_NAMES.size()); + Assert.assertEquals(MetadataColumn.values().length, MetadataColumn.KAFKA_METADATA_INSPECTORS.size()); Assert.assertFalse(Arrays.stream(MetadataColumn.values()) .map(MetadataColumn::getName) - .anyMatch(name -> !KAFKA_METADATA_COLUMN_NAMES.contains(name))); - Arrays.stream(MetadataColumn.values()).forEach(element -> Assert.assertNotNull(KafkaStreamingUtils.recordWritableFnMap.get(element.getName()))); + .anyMatch(name -> !MetadataColumn.KAFKA_METADATA_COLUMN_NAMES.contains(name))); + Arrays.stream(MetadataColumn.values()) + .forEach(element -> Assert.assertNotNull(MetadataColumn.forName(element.getName()))); + } + + @Test public void testGetTaskId() { + String[] + ids = + {"attempt_200707121733_0003_m_000005_0", "attempt_local_0001_m_000005_0", "task_200709221812_0001_m_000005_0", + "task_local_0001_r_000005_0", "task_local_0001_r_000005_2"}; + + String[] + expectedIds = + {"attempt_200707121733_0003_m_000005", "attempt_local_0001_m_000005", "task_200709221812_0001_m_000005", + "task_local_0001_r_000005", "task_local_0001_r_000005"}; + + Object[] actualIds = Arrays.stream(ids).map(id -> { + Configuration configuration = new Configuration(); + configuration.set("mapred.task.id", id); + return configuration; + }).map(KafkaUtils::getTaskId).toArray(); + Assert.assertArrayEquals(expectedIds, actualIds); } } diff --git kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaRecordWritableTest.java kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaWritableTest.java similarity index 69% rename from kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaRecordWritableTest.java rename to kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaWritableTest.java index 4fb9664780..45bf7912c4 100644 --- kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaRecordWritableTest.java +++ kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaWritableTest.java @@ -27,19 +27,19 @@ import java.io.DataInputStream; import java.io.DataOutputStream; import java.io.IOException; +import java.util.Arrays; /** * Test class for kafka Writable. */ -public class KafkaRecordWritableTest { - public KafkaRecordWritableTest() { +@SuppressWarnings("unchecked") public class KafkaWritableTest { + public KafkaWritableTest() { } @Test public void testWriteReadFields() throws IOException { ConsumerRecord record = new ConsumerRecord("topic", 0, 3L, "key".getBytes(), "value".getBytes()); - KafkaRecordWritable - kafkaRecordWritable = - new KafkaRecordWritable(record.partition(), + KafkaWritable kafkaWritable = + new KafkaWritable(record.partition(), record.offset(), record.timestamp(), record.value(), @@ -48,37 +48,47 @@ public KafkaRecordWritableTest() { null); ByteArrayOutputStream baos = new ByteArrayOutputStream(); DataOutputStream w = new DataOutputStream(baos); - kafkaRecordWritable.write(w); + kafkaWritable.write(w); w.flush(); ByteArrayInputStream input = new ByteArrayInputStream(baos.toByteArray()); DataInputStream inputStream = new DataInputStream(input); - KafkaRecordWritable actualKafkaRecordWritable = new KafkaRecordWritable(); - actualKafkaRecordWritable.readFields(inputStream); - Assert.assertEquals(kafkaRecordWritable, actualKafkaRecordWritable); + KafkaWritable actualKafkaWritable = new KafkaWritable(); + actualKafkaWritable.readFields(inputStream); + Assert.assertEquals(kafkaWritable, actualKafkaWritable); } @Test public void testWriteReadFields2() throws IOException { ConsumerRecord record = new ConsumerRecord("topic", 0, 3L, "key".getBytes(), "value".getBytes()); - KafkaRecordWritable - kafkaRecordWritable = - new KafkaRecordWritable(record.partition(), + KafkaWritable kafkaWritable = + new KafkaWritable(record.partition(), record.offset(), record.timestamp(), record.value(), 0L, 100L, - "thiskey".getBytes()); + "thisKey".getBytes()); ByteArrayOutputStream baos = new ByteArrayOutputStream(); DataOutputStream w = new DataOutputStream(baos); - kafkaRecordWritable.write(w); + kafkaWritable.write(w); w.flush(); ByteArrayInputStream input = new ByteArrayInputStream(baos.toByteArray()); DataInputStream inputStream = new DataInputStream(input); - KafkaRecordWritable actualKafkaRecordWritable = new KafkaRecordWritable(); - actualKafkaRecordWritable.readFields(inputStream); - Assert.assertEquals(kafkaRecordWritable, actualKafkaRecordWritable); + KafkaWritable actualKafkaWritable = new KafkaWritable(); + actualKafkaWritable.readFields(inputStream); + Assert.assertEquals(kafkaWritable, actualKafkaWritable); + } + + @Test public void getHiveWritableDoesHandleAllCases() { + KafkaWritable kafkaWritable = new KafkaWritable(5, + 1000L, + 1L, + "value".getBytes(), + 0L, + 10000L, + "key".getBytes()); + Arrays.stream(MetadataColumn.values()).forEach(kafkaWritable::getHiveWritable); } } diff --git kafka-handler/src/test/org/apache/hadoop/hive/kafka/SimpleKafkaWriterTest.java kafka-handler/src/test/org/apache/hadoop/hive/kafka/SimpleKafkaWriterTest.java new file mode 100644 index 0000000000..d8168e02a0 --- /dev/null +++ kafka-handler/src/test/org/apache/hadoop/hive/kafka/SimpleKafkaWriterTest.java @@ -0,0 +1,194 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.hadoop.hive.kafka; + +import org.apache.hadoop.conf.Configuration; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.serialization.ByteArrayDeserializer; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.nio.charset.Charset; +import java.time.Duration; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Properties; +import java.util.Set; +import java.util.UUID; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +/** + * Test class for Kafka simple writer. + */ +@RunWith(Parameterized.class) public class SimpleKafkaWriterTest { + private static final Logger LOG = LoggerFactory.getLogger(SimpleKafkaWriterTest.class); + + private static final int RECORD_NUMBER = 17384; + private static final byte[] KEY_BYTES = "KEY".getBytes(Charset.forName("UTF-8")); + private static final KafkaBrokerResource KAFKA_BROKER_RESOURCE = new KafkaBrokerResource(); + private static final List RECORDS_WRITABLES = IntStream + .range(0, RECORD_NUMBER) + .mapToObj(number -> { + final byte[] value = ("VALUE-" + Integer.toString(number)).getBytes(Charset.forName("UTF-8")); + return new KafkaWritable(0, (long) number, value, KEY_BYTES); + }).collect(Collectors.toList()); + private final Configuration conf = new Configuration(); + private final KafkaOutputFormat.WriteSemantic writeSemantic; + private KafkaConsumer consumer; + + public SimpleKafkaWriterTest(KafkaOutputFormat.WriteSemantic writeSemantic) { + this.writeSemantic = writeSemantic; + } + + @Parameterized.Parameters public static Iterable data() { + return Arrays.asList(new Object[][] {{KafkaOutputFormat.WriteSemantic.BEST_EFFORT}, + {KafkaOutputFormat.WriteSemantic.AT_LEAST_ONCE}}); + } + + @BeforeClass public static void setupCluster() throws Throwable { + KAFKA_BROKER_RESOURCE.before(); + } + + @AfterClass public static void tearDownCluster() { + KAFKA_BROKER_RESOURCE.after(); + } + + @Before public void setUp() { + LOG.info("setting up Config"); + setupConsumer(); + } + + @After public void tearDown() { + consumer.close(); + LOG.info("tearDown"); + } + + private void setupConsumer() { + this.conf.set("kafka.bootstrap.servers", KafkaBrokerResource.BROKER_IP_PORT); + Properties consumerProps = new Properties(); + consumerProps.setProperty("enable.auto.commit", "false"); + consumerProps.setProperty("auto.offset.reset", "none"); + consumerProps.setProperty("bootstrap.servers", KafkaBrokerResource.BROKER_IP_PORT); + consumerProps.setProperty("key.deserializer", ByteArrayDeserializer.class.getName()); + consumerProps.setProperty("value.deserializer", ByteArrayDeserializer.class.getName()); + consumerProps.setProperty("request.timeout.ms", "3002"); + consumerProps.setProperty("fetch.max.wait.ms", "3001"); + consumerProps.setProperty("session.timeout.ms", "3001"); + consumerProps.setProperty("metadata.max.age.ms", "100"); + this.consumer = new KafkaConsumer<>(consumerProps); + } + + @Test(expected = IllegalStateException.class) public void testMissingBrokerString() { + new SimpleKafkaWriter("t", + null, + writeSemantic.equals(KafkaOutputFormat.WriteSemantic.AT_LEAST_ONCE), + new Properties()); + } + + @Test public void testCheckWriterId() { + Properties properties = new Properties(); + properties.setProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9290"); + SimpleKafkaWriter + writer = + new SimpleKafkaWriter("t", + null, + writeSemantic.equals(KafkaOutputFormat.WriteSemantic.AT_LEAST_ONCE), + properties); + Assert.assertNotNull(writer.getWriterId()); + } + + @Test public void testSendToNoWhere() throws IOException { + String notValidBroker = "localhost:6090"; + Properties properties = new Properties(); + properties.setProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, notValidBroker); + properties.setProperty("request.timeout.ms", "100"); + properties.setProperty("metadata.max.age.ms", "100"); + properties.setProperty("max.block.ms", "1000"); + KafkaWritable record = new KafkaWritable(-1, -1, "value".getBytes(), null); + SimpleKafkaWriter writer = new SimpleKafkaWriter("t", null, false, properties); + writer.write(record); + writer.close(false); + Assert.assertEquals("Expect sent records not matching", 1, writer.getSentRecords()); + Assert.assertEquals("Expect lost records is not matching", 1, writer.getLostRecords()); + writer = new SimpleKafkaWriter("t", null, true, properties); + Exception exception = null; + try { + writer.write(record); + writer.close(false); + } catch (IOException e) { + exception = e; + } + Assert.assertNotNull("Must rethrow exception", exception); + Assert.assertEquals("Expect sent records not matching", 1, writer.getSentRecords()); + Assert.assertEquals("Expect lost records is not matching", 1, writer.getLostRecords()); + } + + @Test public void testSend() throws IOException { + String topic = UUID.randomUUID().toString(); + Properties properties = new Properties(); + properties.setProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, KafkaBrokerResource.BROKER_IP_PORT); + SimpleKafkaWriter + writer = + new SimpleKafkaWriter(topic, + null, + writeSemantic.equals(KafkaOutputFormat.WriteSemantic.AT_LEAST_ONCE), + properties); + RECORDS_WRITABLES.forEach(kafkaRecordWritable -> { + try { + writer.write(kafkaRecordWritable); + } catch (IOException e) { + throw new RuntimeException(e); + } + }); + writer.close(false); + Assert.assertEquals(RECORD_NUMBER, writer.getSentRecords()); + Assert.assertEquals(0, writer.getLostRecords()); + Set assignment = Collections.singleton(new TopicPartition(topic, 0)); + consumer.assign(assignment); + consumer.seekToBeginning(assignment); + long numRecords = 0; + while (numRecords < RECORD_NUMBER) { + ConsumerRecords records = consumer.poll(Duration.ofMillis(1000)); + + Assert.assertFalse(records.records(new TopicPartition(topic, 0)) + .stream() + .anyMatch(consumerRecord -> !RECORDS_WRITABLES.contains(new KafkaWritable(0, + consumerRecord.timestamp(), + consumerRecord.value(), + consumerRecord.key())))); + + numRecords += records.count(); + } + Assert.assertEquals(RECORD_NUMBER, numRecords); + } +} diff --git kafka-handler/src/test/org/apache/hadoop/hive/kafka/TransactionalKafkaWriterTest.java kafka-handler/src/test/org/apache/hadoop/hive/kafka/TransactionalKafkaWriterTest.java new file mode 100644 index 0000000000..86ed8661c7 --- /dev/null +++ kafka-handler/src/test/org/apache/hadoop/hive/kafka/TransactionalKafkaWriterTest.java @@ -0,0 +1,246 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you 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 org.apache.hadoop.hive.kafka; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.api.MetaException; +import org.apache.hadoop.hive.metastore.api.StorageDescriptor; +import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.ProducerFencedException; +import org.apache.kafka.common.serialization.ByteArrayDeserializer; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.mockito.Mockito; + +import java.io.IOException; +import java.nio.charset.Charset; +import java.time.Duration; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.Set; +import java.util.UUID; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +/** + * Test Transactional Writer. + */ +public class TransactionalKafkaWriterTest { + + private static final String TOPIC = "TOPIC_TEST"; + @Rule public TemporaryFolder temporaryFolder = new TemporaryFolder(); + + private static final KafkaBrokerResource KAFKA_BROKER_RESOURCE = new KafkaBrokerResource(); + + private static final int RECORD_NUMBER = 1000; + private static final byte[] KEY_BYTES = "key".getBytes(); + private static final List RECORDS_WRITABLES = IntStream + .range(0, RECORD_NUMBER) + .mapToObj(number -> { + final byte[] value = ("VALUE-" + Integer.toString(number)).getBytes(Charset.forName("UTF-8")); + return new KafkaWritable(0, (long) number, value, KEY_BYTES); + }) + .collect(Collectors.toList()); + + private final Configuration configuration = new Configuration(); + private final FileSystem fs = FileSystem.getLocal(configuration); + private final String queryId = UUID.randomUUID().toString(); + private final Map parameters = new HashMap<>(); + private Path queryWorkingPath; + private KafkaStorageHandler kafkaStorageHandler; + private final Table table = Mockito.mock(Table.class); + private KafkaConsumer consumer; + private Properties properties; + + private void setupConsumer() { + Properties consumerProps = new Properties(); + consumerProps.setProperty("enable.auto.commit", "false"); + consumerProps.setProperty("auto.offset.reset", "none"); + consumerProps.setProperty("bootstrap.servers", KafkaBrokerResource.BROKER_IP_PORT); + consumerProps.setProperty("key.deserializer", ByteArrayDeserializer.class.getName()); + consumerProps.setProperty("value.deserializer", ByteArrayDeserializer.class.getName()); + consumerProps.setProperty("request.timeout.ms", "3002"); + consumerProps.setProperty("fetch.max.wait.ms", "3001"); + consumerProps.setProperty("session.timeout.ms", "3001"); + consumerProps.setProperty("metadata.max.age.ms", "100"); + consumerProps.setProperty("max.poll.interval.ms", "300"); + consumerProps.setProperty("max.block.ms", "1000"); + consumerProps.setProperty("isolation.level", "read_committed"); + this.consumer = new KafkaConsumer<>(consumerProps); + } + + public TransactionalKafkaWriterTest() throws IOException { + } + + @BeforeClass public static void setupCluster() throws Throwable { + KAFKA_BROKER_RESOURCE.before(); + } + + @AfterClass public static void tearDownCluster() { + KAFKA_BROKER_RESOURCE.after(); + } + + @Before public void setup() throws IOException { + setupConsumer(); + temporaryFolder.create(); + Path tableLocation = new Path(temporaryFolder.newFolder().toURI()); + queryWorkingPath = new Path(tableLocation, queryId); + configuration.set(HiveConf.ConfVars.HIVEQUERYID.varname, queryId); + String taskId = "attempt_m_0001_0"; + configuration.set("mapred.task.id", taskId); + configuration.set(KafkaTableProperties.HIVE_KAFKA_BOOTSTRAP_SERVERS.getName(), KafkaBrokerResource.BROKER_IP_PORT); + Arrays.stream(KafkaTableProperties.values()) + .filter(kafkaTableProperties -> !kafkaTableProperties.isMandatory()) + .forEach(key -> { + configuration.set(key.getName(), key.getDefaultValue()); + parameters.put(key.getName(), key.getDefaultValue()); + }); + parameters.put(KafkaTableProperties.WRITE_SEMANTIC_PROPERTY.getName(), + KafkaOutputFormat.WriteSemantic.EXACTLY_ONCE.name()); + configuration.set(KafkaTableProperties.WRITE_SEMANTIC_PROPERTY.getName(), + KafkaOutputFormat.WriteSemantic.EXACTLY_ONCE.name()); + parameters.put(KafkaTableProperties.HIVE_KAFKA_TOPIC.getName(), TOPIC); + parameters.put(KafkaTableProperties.HIVE_KAFKA_BOOTSTRAP_SERVERS.getName(), KafkaBrokerResource.BROKER_IP_PORT); + parameters.put(KafkaTableProperties.HIVE_KAFKA_OPTIMISTIC_COMMIT.getName(), "false"); + kafkaStorageHandler = new KafkaStorageHandler(); + Mockito.when(table.getParameters()).thenReturn(parameters); + StorageDescriptor sd = new StorageDescriptor(); + sd.setLocation(tableLocation.toString()); + Mockito.when(table.getSd()).thenReturn(sd); + kafkaStorageHandler.setConf(configuration); + properties = KafkaUtils.producerProperties(configuration); + } + + @After public void tearAfterTest() { + KAFKA_BROKER_RESOURCE.deleteTopic(TOPIC); + consumer.close(); + consumer = null; + } + + @Test public void writeAndCommit() throws IOException, MetaException { + TransactionalKafkaWriter + zombieWriter = + new TransactionalKafkaWriter(TOPIC, properties, queryWorkingPath, fs, false); + RECORDS_WRITABLES.forEach(kafkaRecordWritable -> { + try { + zombieWriter.write(kafkaRecordWritable); + } catch (IOException e) { + throw new RuntimeException(e); + } + }); + Assert.assertEquals(RECORD_NUMBER, zombieWriter.getSentRecords()); + + TransactionalKafkaWriter + writer = + new TransactionalKafkaWriter(TOPIC, properties, queryWorkingPath, fs, false); + + RECORDS_WRITABLES.forEach(kafkaRecordWritable -> { + try { + writer.write(kafkaRecordWritable); + } catch (IOException e) { + throw new RuntimeException(e); + } + }); + + //TEST zombie id is the same as current writer and epoch is greater + Assert.assertEquals(writer.getProducerId(), zombieWriter.getProducerId()); + Assert.assertTrue(writer.getProducerEpoch() > zombieWriter.getProducerEpoch()); + + zombieWriter.close(false); + writer.close(false); + + kafkaStorageHandler.commitInsertTable(table, false); + checkData(); + } + + @Test(expected = java.lang.AssertionError.class) public void writeAndNoCommit() throws IOException { + TransactionalKafkaWriter + writer = + new TransactionalKafkaWriter(TOPIC, properties, queryWorkingPath, fs, false); + RECORDS_WRITABLES.forEach(kafkaRecordWritable -> { + try { + writer.write(kafkaRecordWritable); + } catch (IOException e) { + throw new RuntimeException(e); + } + }); + writer.close(false); + Assert.assertEquals(writer.getSentRecords(), RECORD_NUMBER); + //DATA is not committed + checkData(); + } + + @Test(expected = IOException.class) public void writerFencedOut() throws IOException { + TransactionalKafkaWriter + writer = + new TransactionalKafkaWriter(TOPIC, properties, queryWorkingPath, fs, false); + + //noinspection unused this is actually used, the contstructor start the TX that is what we need + TransactionalKafkaWriter + newWriter = + new TransactionalKafkaWriter(TOPIC, properties, queryWorkingPath, fs, false); + + try { + for (KafkaWritable record : RECORDS_WRITABLES) { + writer.write(record); + } + } catch (IOException e) { + Assert.assertTrue(e.getCause() instanceof ProducerFencedException); + throw e; + } + } + + private void checkData() { + Set assignment = Collections.singleton(new TopicPartition(TOPIC, 0)); + consumer.assign(assignment); + consumer.seekToBeginning(assignment); + long numRecords = 0; + boolean emptyPoll = false; + while (numRecords < RECORD_NUMBER && !emptyPoll) { + ConsumerRecords records = consumer.poll(Duration.ofMillis(1000)); + + Assert.assertFalse(records.records(new TopicPartition(TOPIC, 0)) + .stream() + .anyMatch(consumerRecord -> !RECORDS_WRITABLES.contains(new KafkaWritable(0, + consumerRecord.timestamp(), + consumerRecord.value(), + consumerRecord.key())))); + + emptyPoll = records.isEmpty(); + numRecords += records.count(); + } + Assert.assertEquals(RECORD_NUMBER, numRecords); + } +} diff --git ql/src/test/queries/clientpositive/kafka_storage_handler.q ql/src/test/queries/clientpositive/kafka_storage_handler.q index f10bb90e34..595f0320b6 100644 --- ql/src/test/queries/clientpositive/kafka_storage_handler.q +++ ql/src/test/queries/clientpositive/kafka_storage_handler.q @@ -244,3 +244,55 @@ select cast ((`__timestamp`/1000) as timestamp) as kafka_record_ts, `__timestamp `__partition`, `__start_offset`,`__end_offset`, `__key`, `__offset`, `timestamp`, `user`, `page`, `deleted`, `deltabucket`, `isanonymous`, `commentlength` from wiki_kafka_avro_table where `__timestamp` > 1534750625090; + +CREATE EXTERNAL TABLE kafka_table_insert +(c_name string, c_int int, c_float float) +STORED BY 'org.apache.hadoop.hive.kafka.KafkaStorageHandler' +WITH SERDEPROPERTIES ("timestamp.formats"="yyyy-MM-dd\'T\'HH:mm:ss\'Z\'") +TBLPROPERTIES +("kafka.topic" = "test-topic-write-json", +"kafka.bootstrap.servers"="localhost:9092", +"kafka.serde.class"="org.apache.hadoop.hive.serde2.JsonSerDe") +; + +insert into table kafka_table_insert (c_name,c_int, c_float,`__key`, `__partition`, `__offset`, `__timestamp`, `__start_offset`, `__end_offset`) +values ('test1',5, 4.999,'key',null ,-1,1536449552290,-1,null ); + +insert into table kafka_table_insert (c_name,c_int, c_float, `__key`, `__partition`, `__offset`, `__timestamp`, `__start_offset`, `__end_offset`) +values ('test2',15, 14.9996666, null ,null ,-1,1536449552285,-1,-1 ); + +select * from kafka_table_insert; + + +insert into table wiki_kafka_avro_table select +isrobot as isrobot, channel as channel,`timestamp` as `timestamp`, flags as flags, isunpatrolled as isunpatrolled, page as page, +diffurl as diffurl, added as added, comment as comment, commentlength as commentlength, isnew as isnew, isminor as isminor, +delta as delta, isanonymous as isanonymous, `user` as `user`, deltabucket as detlabucket, deleted as deleted, namespace as namespace, +`__key`, `__partition`, -1 as `__offset`,`__timestamp`, -1 as `__start_offset`, -1 as `__end_offset` +from wiki_kafka_avro_table; + +select cast ((`__timestamp`/1000) as timestamp) as kafka_record_ts, `__partition`, `__offset`, `timestamp`, `user`, `page`, `deleted`, `deltabucket`, `isanonymous`, `commentlength` from wiki_kafka_avro_table; + +select `__key`, count(1) FROM wiki_kafka_avro_table group by `__key` order by `__key`; + +select `__timestamp`, count(1) from wiki_kafka_avro_table group by `__timestamp` order by `__timestamp`; + + +CREATE EXTERNAL TABLE kafka_table_csv +(c_name string, c_int int, c_float float) +STORED BY 'org.apache.hadoop.hive.kafka.KafkaStorageHandler' +TBLPROPERTIES +("kafka.topic" = "test-topic-write-csv", +"kafka.bootstrap.servers"="localhost:9092", +"kafka.serde.class"="org.apache.hadoop.hive.serde2.OpenCSVSerde"); + +ALTER TABLE kafka_table_csv SET TBLPROPERTIES ("hive.kafka.optimistic.commit"="false", "kafka.write.semantic"="EXACTLY_ONCE"); +insert into table kafka_table_csv select c_name, c_int, c_float, `__key`, `__partition`, -1 as `__offset`, `__timestamp`, -1, -1 from kafka_table_insert; + +insert into table kafka_table_csv (c_name,c_int, c_float,`__key`, `__partition`, `__offset`, `__timestamp`, `__start_offset`, `__end_offset`) +values ('test4',-5, -4.999,'key-2',null ,-1,1536449552291,-1,null ); + +insert into table kafka_table_csv (c_name,c_int, c_float, `__key`, `__partition`, `__offset`, `__timestamp`, `__start_offset`, `__end_offset`) +values ('test5',-15, -14.9996666, 'key-3' ,null ,-1,1536449552284,-1,-1 ); + +select * from kafka_table_csv; \ No newline at end of file diff --git ql/src/test/results/clientpositive/druid/kafka_storage_handler.q.out ql/src/test/results/clientpositive/druid/kafka_storage_handler.q.out index 593cd07b93..73f0f293d9 100644 --- ql/src/test/results/clientpositive/druid/kafka_storage_handler.q.out +++ ql/src/test/results/clientpositive/druid/kafka_storage_handler.q.out @@ -590,16 +590,16 @@ FROM kafka_table_2 POSTHOOK: type: QUERY POSTHOOK: Input: default@kafka_table_2 POSTHOOK: Output: hdfs://### HDFS PATH ### -0 0 2013-08-30 18:02:33.0 US/Pacific Gypsy Danger nuclear en United States North America article true true false false 57 200 -143 -0 1 2013-08-30 20:32:45.0 US/Pacific Striker Eureka speed en Australia Australia wikipedia true false false true 459 129 330 -0 2 2013-08-31 00:11:21.0 US/Pacific Cherno Alpha masterYi ru Russia Asia article true false false true 123 12 111 -0 3 2013-08-31 04:58:39.0 US/Pacific Crimson Typhoon triplets zh China Asia wikipedia false true false true 905 5 900 -0 4 2013-08-31 05:41:27.0 US/Pacific Coyote Tango stringer ja Japan Asia wikipedia false true false true 1 10 -9 -0 5 2013-08-31 18:02:33.0 US/Pacific Gypsy Danger nuclear en United States North America article true true false false 57 200 -143 -0 6 2013-08-31 20:32:45.0 US/Pacific Striker Eureka speed en Australia Australia wikipedia true false false true 459 129 330 -0 7 2013-09-01 00:11:21.0 US/Pacific Cherno Alpha masterYi ru Russia Asia article true false false true 123 12 111 -0 8 2013-09-01 04:58:39.0 US/Pacific Crimson Typhoon triplets zh China Asia wikipedia false true false true 905 5 900 -0 9 2013-09-01 05:41:27.0 US/Pacific Coyote Tango stringer ja Japan Asia wikipedia false true false true 1 10 -9 +0 0 NULL Gypsy Danger nuclear en United States North America article true true false false 57 200 -143 +0 1 NULL Striker Eureka speed en Australia Australia wikipedia true false false true 459 129 330 +0 2 NULL Cherno Alpha masterYi ru Russia Asia article true false false true 123 12 111 +0 3 NULL Crimson Typhoon triplets zh China Asia wikipedia false true false true 905 5 900 +0 4 NULL Coyote Tango stringer ja Japan Asia wikipedia false true false true 1 10 -9 +0 5 NULL Gypsy Danger nuclear en United States North America article true true false false 57 200 -143 +0 6 NULL Striker Eureka speed en Australia Australia wikipedia true false false true 459 129 330 +0 7 NULL Cherno Alpha masterYi ru Russia Asia article true false false true 123 12 111 +0 8 NULL Crimson Typhoon triplets zh China Asia wikipedia false true false true 905 5 900 +0 9 NULL Coyote Tango stringer ja Japan Asia wikipedia false true false true 1 10 -9 PREHOOK: query: Select count(*) FROM kafka_table_2 PREHOOK: type: QUERY PREHOOK: Input: default@kafka_table_2 @@ -895,3 +895,209 @@ POSTHOOK: Output: hdfs://### HDFS PATH ### 2018-08-20 11:37:05.09 1534765025090 0 5 11 key-8 8 08/20/2018 04:37:05 test-user-8 page is 800 -8 803.2 true 8 2018-08-20 12:37:05.09 1534768625090 0 5 11 key-9 9 08/20/2018 05:37:05 test-user-9 page is 900 -9 903.6 false 9 2018-08-20 13:37:05.09 1534772225090 0 5 11 key-10 10 08/20/2018 06:37:05 test-user-10 page is 1000 -10 1004.0 true 10 +PREHOOK: query: CREATE EXTERNAL TABLE kafka_table_insert +(c_name string, c_int int, c_float float) +STORED BY 'org.apache.hadoop.hive.kafka.KafkaStorageHandler' +WITH SERDEPROPERTIES ("timestamp.formats"="yyyy-MM-dd\'T\'HH:mm:ss\'Z\'") +TBLPROPERTIES +("kafka.topic" = "test-topic-write-json", +"kafka.bootstrap.servers"="localhost:9092", +"kafka.serde.class"="org.apache.hadoop.hive.serde2.JsonSerDe") +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@kafka_table_insert +POSTHOOK: query: CREATE EXTERNAL TABLE kafka_table_insert +(c_name string, c_int int, c_float float) +STORED BY 'org.apache.hadoop.hive.kafka.KafkaStorageHandler' +WITH SERDEPROPERTIES ("timestamp.formats"="yyyy-MM-dd\'T\'HH:mm:ss\'Z\'") +TBLPROPERTIES +("kafka.topic" = "test-topic-write-json", +"kafka.bootstrap.servers"="localhost:9092", +"kafka.serde.class"="org.apache.hadoop.hive.serde2.JsonSerDe") +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@kafka_table_insert +PREHOOK: query: insert into table kafka_table_insert (c_name,c_int, c_float,`__key`, `__partition`, `__offset`, `__timestamp`, `__start_offset`, `__end_offset`) +values ('test1',5, 4.999,'key',null ,-1,1536449552290,-1,null ) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@kafka_table_insert +POSTHOOK: query: insert into table kafka_table_insert (c_name,c_int, c_float,`__key`, `__partition`, `__offset`, `__timestamp`, `__start_offset`, `__end_offset`) +values ('test1',5, 4.999,'key',null ,-1,1536449552290,-1,null ) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@kafka_table_insert +PREHOOK: query: insert into table kafka_table_insert (c_name,c_int, c_float, `__key`, `__partition`, `__offset`, `__timestamp`, `__start_offset`, `__end_offset`) +values ('test2',15, 14.9996666, null ,null ,-1,1536449552285,-1,-1 ) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@kafka_table_insert +POSTHOOK: query: insert into table kafka_table_insert (c_name,c_int, c_float, `__key`, `__partition`, `__offset`, `__timestamp`, `__start_offset`, `__end_offset`) +values ('test2',15, 14.9996666, null ,null ,-1,1536449552285,-1,-1 ) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@kafka_table_insert +PREHOOK: query: select * from kafka_table_insert +PREHOOK: type: QUERY +PREHOOK: Input: default@kafka_table_insert +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select * from kafka_table_insert +POSTHOOK: type: QUERY +POSTHOOK: Input: default@kafka_table_insert +POSTHOOK: Output: hdfs://### HDFS PATH ### +test1 5 4.999 key 0 0 1536449552290 0 2 +test2 15 14.999666 NULL 0 1 1536449552285 0 2 +PREHOOK: query: insert into table wiki_kafka_avro_table select +isrobot as isrobot, channel as channel,`timestamp` as `timestamp`, flags as flags, isunpatrolled as isunpatrolled, page as page, +diffurl as diffurl, added as added, comment as comment, commentlength as commentlength, isnew as isnew, isminor as isminor, +delta as delta, isanonymous as isanonymous, `user` as `user`, deltabucket as detlabucket, deleted as deleted, namespace as namespace, +`__key`, `__partition`, -1 as `__offset`,`__timestamp`, -1 as `__start_offset`, -1 as `__end_offset` +from wiki_kafka_avro_table +PREHOOK: type: QUERY +PREHOOK: Input: default@wiki_kafka_avro_table +PREHOOK: Output: default@wiki_kafka_avro_table +POSTHOOK: query: insert into table wiki_kafka_avro_table select +isrobot as isrobot, channel as channel,`timestamp` as `timestamp`, flags as flags, isunpatrolled as isunpatrolled, page as page, +diffurl as diffurl, added as added, comment as comment, commentlength as commentlength, isnew as isnew, isminor as isminor, +delta as delta, isanonymous as isanonymous, `user` as `user`, deltabucket as detlabucket, deleted as deleted, namespace as namespace, +`__key`, `__partition`, -1 as `__offset`,`__timestamp`, -1 as `__start_offset`, -1 as `__end_offset` +from wiki_kafka_avro_table +POSTHOOK: type: QUERY +POSTHOOK: Input: default@wiki_kafka_avro_table +POSTHOOK: Output: default@wiki_kafka_avro_table +PREHOOK: query: select cast ((`__timestamp`/1000) as timestamp) as kafka_record_ts, `__partition`, `__offset`, `timestamp`, `user`, `page`, `deleted`, `deltabucket`, `isanonymous`, `commentlength` from wiki_kafka_avro_table +PREHOOK: type: QUERY +PREHOOK: Input: default@wiki_kafka_avro_table +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select cast ((`__timestamp`/1000) as timestamp) as kafka_record_ts, `__partition`, `__offset`, `timestamp`, `user`, `page`, `deleted`, `deltabucket`, `isanonymous`, `commentlength` from wiki_kafka_avro_table +POSTHOOK: type: QUERY +POSTHOOK: Input: default@wiki_kafka_avro_table +POSTHOOK: Output: hdfs://### HDFS PATH ### +2018-08-20 03:37:05.09 0 0 08/19/2018 20:37:05 test-user-0 page is 0 0 0.0 false 0 +2018-08-20 04:37:05.09 0 1 08/19/2018 21:37:05 test-user-1 page is 100 -1 100.4 true 1 +2018-08-20 05:37:05.09 0 2 08/19/2018 22:37:05 test-user-2 page is 200 -2 200.8 true 2 +2018-08-20 06:37:05.09 0 3 08/19/2018 23:37:05 test-user-3 page is 300 -3 301.20000000000005 false 3 +2018-08-20 07:37:05.09 0 4 08/20/2018 00:37:05 test-user-4 page is 400 -4 401.6 true 4 +2018-08-20 08:37:05.09 0 5 08/20/2018 01:37:05 test-user-5 page is 500 -5 502.0 true 5 +2018-08-20 09:37:05.09 0 6 08/20/2018 02:37:05 test-user-6 page is 600 -6 602.4000000000001 false 6 +2018-08-20 10:37:05.09 0 7 08/20/2018 03:37:05 test-user-7 page is 700 -7 702.8000000000001 true 7 +2018-08-20 11:37:05.09 0 8 08/20/2018 04:37:05 test-user-8 page is 800 -8 803.2 true 8 +2018-08-20 12:37:05.09 0 9 08/20/2018 05:37:05 test-user-9 page is 900 -9 903.6 false 9 +2018-08-20 13:37:05.09 0 10 08/20/2018 06:37:05 test-user-10 page is 1000 -10 1004.0 true 10 +2018-08-20 03:37:05.09 0 11 08/19/2018 20:37:05 test-user-0 page is 0 0 0.0 false 0 +2018-08-20 04:37:05.09 0 12 08/19/2018 21:37:05 test-user-1 page is 100 -1 100.4 true 1 +2018-08-20 05:37:05.09 0 13 08/19/2018 22:37:05 test-user-2 page is 200 -2 200.8 true 2 +2018-08-20 06:37:05.09 0 14 08/19/2018 23:37:05 test-user-3 page is 300 -3 301.20000000000005 false 3 +2018-08-20 07:37:05.09 0 15 08/20/2018 00:37:05 test-user-4 page is 400 -4 401.6 true 4 +2018-08-20 08:37:05.09 0 16 08/20/2018 01:37:05 test-user-5 page is 500 -5 502.0 true 5 +2018-08-20 09:37:05.09 0 17 08/20/2018 02:37:05 test-user-6 page is 600 -6 602.4000000000001 false 6 +2018-08-20 10:37:05.09 0 18 08/20/2018 03:37:05 test-user-7 page is 700 -7 702.8000000000001 true 7 +2018-08-20 11:37:05.09 0 19 08/20/2018 04:37:05 test-user-8 page is 800 -8 803.2 true 8 +2018-08-20 12:37:05.09 0 20 08/20/2018 05:37:05 test-user-9 page is 900 -9 903.6 false 9 +2018-08-20 13:37:05.09 0 21 08/20/2018 06:37:05 test-user-10 page is 1000 -10 1004.0 true 10 +PREHOOK: query: select `__key`, count(1) FROM wiki_kafka_avro_table group by `__key` order by `__key` +PREHOOK: type: QUERY +PREHOOK: Input: default@wiki_kafka_avro_table +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select `__key`, count(1) FROM wiki_kafka_avro_table group by `__key` order by `__key` +POSTHOOK: type: QUERY +POSTHOOK: Input: default@wiki_kafka_avro_table +POSTHOOK: Output: hdfs://### HDFS PATH ### +key-0 2 +key-1 2 +key-10 2 +key-2 2 +key-3 2 +key-4 2 +key-5 2 +key-6 2 +key-7 2 +key-8 2 +key-9 2 +PREHOOK: query: select `__timestamp`, count(1) from wiki_kafka_avro_table group by `__timestamp` order by `__timestamp` +PREHOOK: type: QUERY +PREHOOK: Input: default@wiki_kafka_avro_table +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select `__timestamp`, count(1) from wiki_kafka_avro_table group by `__timestamp` order by `__timestamp` +POSTHOOK: type: QUERY +POSTHOOK: Input: default@wiki_kafka_avro_table +POSTHOOK: Output: hdfs://### HDFS PATH ### +1534736225090 2 +1534739825090 2 +1534743425090 2 +1534747025090 2 +1534750625090 2 +1534754225090 2 +1534757825090 2 +1534761425090 2 +1534765025090 2 +1534768625090 2 +1534772225090 2 +PREHOOK: query: CREATE EXTERNAL TABLE kafka_table_csv +(c_name string, c_int int, c_float float) +STORED BY 'org.apache.hadoop.hive.kafka.KafkaStorageHandler' +TBLPROPERTIES +("kafka.topic" = "test-topic-write-csv", +"kafka.bootstrap.servers"="localhost:9092", +"kafka.serde.class"="org.apache.hadoop.hive.serde2.OpenCSVSerde") +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@kafka_table_csv +POSTHOOK: query: CREATE EXTERNAL TABLE kafka_table_csv +(c_name string, c_int int, c_float float) +STORED BY 'org.apache.hadoop.hive.kafka.KafkaStorageHandler' +TBLPROPERTIES +("kafka.topic" = "test-topic-write-csv", +"kafka.bootstrap.servers"="localhost:9092", +"kafka.serde.class"="org.apache.hadoop.hive.serde2.OpenCSVSerde") +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@kafka_table_csv +PREHOOK: query: ALTER TABLE kafka_table_csv SET TBLPROPERTIES ("hive.kafka.optimistic.commit"="false", "kafka.write.semantic"="EXACTLY_ONCE") +PREHOOK: type: ALTERTABLE_PROPERTIES +PREHOOK: Input: default@kafka_table_csv +PREHOOK: Output: default@kafka_table_csv +POSTHOOK: query: ALTER TABLE kafka_table_csv SET TBLPROPERTIES ("hive.kafka.optimistic.commit"="false", "kafka.write.semantic"="EXACTLY_ONCE") +POSTHOOK: type: ALTERTABLE_PROPERTIES +POSTHOOK: Input: default@kafka_table_csv +POSTHOOK: Output: default@kafka_table_csv +PREHOOK: query: insert into table kafka_table_csv select c_name, c_int, c_float, `__key`, `__partition`, -1 as `__offset`, `__timestamp`, -1, -1 from kafka_table_insert +PREHOOK: type: QUERY +PREHOOK: Input: default@kafka_table_insert +PREHOOK: Output: default@kafka_table_csv +POSTHOOK: query: insert into table kafka_table_csv select c_name, c_int, c_float, `__key`, `__partition`, -1 as `__offset`, `__timestamp`, -1, -1 from kafka_table_insert +POSTHOOK: type: QUERY +POSTHOOK: Input: default@kafka_table_insert +POSTHOOK: Output: default@kafka_table_csv +PREHOOK: query: insert into table kafka_table_csv (c_name,c_int, c_float,`__key`, `__partition`, `__offset`, `__timestamp`, `__start_offset`, `__end_offset`) +values ('test4',-5, -4.999,'key-2',null ,-1,1536449552291,-1,null ) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@kafka_table_csv +POSTHOOK: query: insert into table kafka_table_csv (c_name,c_int, c_float,`__key`, `__partition`, `__offset`, `__timestamp`, `__start_offset`, `__end_offset`) +values ('test4',-5, -4.999,'key-2',null ,-1,1536449552291,-1,null ) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@kafka_table_csv +PREHOOK: query: insert into table kafka_table_csv (c_name,c_int, c_float, `__key`, `__partition`, `__offset`, `__timestamp`, `__start_offset`, `__end_offset`) +values ('test5',-15, -14.9996666, 'key-3' ,null ,-1,1536449552284,-1,-1 ) +PREHOOK: type: QUERY +PREHOOK: Input: _dummy_database@_dummy_table +PREHOOK: Output: default@kafka_table_csv +POSTHOOK: query: insert into table kafka_table_csv (c_name,c_int, c_float, `__key`, `__partition`, `__offset`, `__timestamp`, `__start_offset`, `__end_offset`) +values ('test5',-15, -14.9996666, 'key-3' ,null ,-1,1536449552284,-1,-1 ) +POSTHOOK: type: QUERY +POSTHOOK: Input: _dummy_database@_dummy_table +POSTHOOK: Output: default@kafka_table_csv +PREHOOK: query: select * from kafka_table_csv +PREHOOK: type: QUERY +PREHOOK: Input: default@kafka_table_csv +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select * from kafka_table_csv +POSTHOOK: type: QUERY +POSTHOOK: Input: default@kafka_table_csv +POSTHOOK: Output: hdfs://### HDFS PATH ### +test1 5 4.999 key 0 0 1536449552290 0 7 +test2 15 14.999666 NULL 0 1 1536449552285 0 7 +test4 -5 -4.999 key-2 0 3 1536449552291 0 7 +test5 -15 -14.9996666 key-3 0 5 1536449552284 0 7