From 3996b5b2e86f7b3cd4ca319227163852e268aec6 Mon Sep 17 00:00:00 2001 From: Slim Bouguerra Date: Thu, 19 Jul 2018 09:30:48 -0700 Subject: [PATCH 01/33] Working version of kafka storage handler Change-Id: Ief161074d151917c3a7ed443cf78374bcaf7bcfc trying random things fix typo working version for demo only change might be getRecordreader call adding kafka Trimmer first cut Change-Id: If9bf7f561b867e80ab31f5c8c3c40730128986af link the code and add some docs Change-Id: I7e15d90de772fffef8ee0930352069742af12ac7 add static function to avoid dep on hive exec Change-Id: Ib61901cd45027d1469d72a890e26f73997402974 working version of filter pushdown partition and offsets Change-Id: I3e3d157438fcc965491380e3f1afa9c81a6cd75b fix case when offset filter is out of range and add some code comments Change-Id: Ic3321a14cc9a06b8eeb85cc20ea89f21e2765a93 add filter push down based on ts Change-Id: I5a1da7634cfc80990036add3463a28810ce642c2 clean code Change-Id: I753cfe4e9f0a69396635b7a9fe16c29f7bb0675e case filter is null can imply no filter Change-Id: I68fa9afe09009329ebe8ac9e5d3f7923a2003ebb some refactor for how to build consumer Change-Id: If007a89d93c215542027aec7ea0aff2fa6dc83fb major refactor: removed and/or methods switch logic of ANDing filters Change-Id: Id36f45842db7c4579edaa1e6062bddd291d53bc9 adding tests and refactor fetch by time Change-Id: I6329ca3fa8c6f29e90d0001034064d2a603d2580 adding more tests Change-Id: Ic9d1f1a86ffd0f99b298965db5c5d1f36670b15b more tests Change-Id: I4aada643b67e3e1207a504267a50dc75bc0176e3 small refactor to where column names are stored Change-Id: I0635c9627520f786e4f45e501eb63158f915d2a0 Start working on GenericSerde Change-Id: I1adddfb639da7d6a64dde06c46bc67c072806469 first cut that uses default_json Change-Id: Icad9eb9eb36b76eea4ba1342193c649a9d419287 cleaning some tests stuff Change-Id: I889c4788250590f6cc7f0d9d7a1756f71e9e5cc5 setting the default serde for storage handler Change-Id: I00179ec97d43d0a955500a23ebdab770d26930a7 fix how reading the property Change-Id: I1bba7e19defda3316b4ddb0e0721e45cf11be063 working version of generic serde Change-Id: I370a87aaf55f599db1695775ec2737e54af81270 fixup squash with previous Change-Id: I6adfc93efed84c38aa1ad7092660e4cca49bc29a use nanos for logging Change-Id: I9055e0813b3b4bcbf72db0900ab9d2cb480c8f8b major refactor plus tests working q files tests Change-Id: I5ffc1cfcb4708e7a89163c371027e92782f2e4fc adding q files test Change-Id: I887fff5e3fdcdb0322770e52f9a8ab732a8dbe86 --- hive-kafka-reader/pom.xml | 161 ++++ .../hadoop/hive/kafka/GenericKafkaSerDe.java | 195 +++++ .../hadoop/hive/kafka/KafkaJsonSerDe.java | 270 +++++++ .../hive/kafka/KafkaPullerInputFormat.java | 234 ++++++ .../hive/kafka/KafkaPullerInputSplit.java | 231 ++++++ .../hive/kafka/KafkaPullerRecordReader.java | 172 +++++ .../hive/kafka/KafkaRecordIterator.java | 183 +++++ .../hive/kafka/KafkaRecordWritable.java | 121 +++ .../hadoop/hive/kafka/KafkaScanTrimmer.java | 523 +++++++++++++ .../hive/kafka/KafkaStorageHandler.java | 182 +++++ .../hive/kafka/KafkaStreamingUtils.java | 69 ++ .../hive/kafka/KafkaPullerInputSplitTest.java | 217 ++++++ .../hive/kafka/KafkaRecordIteratorTest.java | 376 +++++++++ .../hive/kafka/KafkaRecordWritableTest.java | 37 + .../hive/kafka/KafkaScanTrimmerTest.java | 730 ++++++++++++++++++ .../hive/kafka/KafkaStreamingUtilsTest.java | 29 + itests/qtest-druid/pom.xml | 2 +- itests/qtest/pom.xml | 7 +- .../hive/llap/cli/LlapServiceDriver.java | 2 +- packaging/pom.xml | 5 + pom.xml | 1 + .../hive/ql/index/IndexPredicateAnalyzer.java | 24 +- .../clientpositive/kafka_storage_handler.q | 46 ++ .../druid/kafka_storage_handler.q.out | 184 +++++ 24 files changed, 3986 insertions(+), 15 deletions(-) create mode 100644 hive-kafka-reader/pom.xml create mode 100644 hive-kafka-reader/src/java/org/apache/hadoop/hive/kafka/GenericKafkaSerDe.java create mode 100644 hive-kafka-reader/src/java/org/apache/hadoop/hive/kafka/KafkaJsonSerDe.java create mode 100644 hive-kafka-reader/src/java/org/apache/hadoop/hive/kafka/KafkaPullerInputFormat.java create mode 100644 hive-kafka-reader/src/java/org/apache/hadoop/hive/kafka/KafkaPullerInputSplit.java create mode 100644 hive-kafka-reader/src/java/org/apache/hadoop/hive/kafka/KafkaPullerRecordReader.java create mode 100644 hive-kafka-reader/src/java/org/apache/hadoop/hive/kafka/KafkaRecordIterator.java create mode 100644 hive-kafka-reader/src/java/org/apache/hadoop/hive/kafka/KafkaRecordWritable.java create mode 100644 hive-kafka-reader/src/java/org/apache/hadoop/hive/kafka/KafkaScanTrimmer.java create mode 100644 hive-kafka-reader/src/java/org/apache/hadoop/hive/kafka/KafkaStorageHandler.java create mode 100644 hive-kafka-reader/src/java/org/apache/hadoop/hive/kafka/KafkaStreamingUtils.java create mode 100644 hive-kafka-reader/src/test/org/apache/hadoop/hive/kafka/KafkaPullerInputSplitTest.java create mode 100644 hive-kafka-reader/src/test/org/apache/hadoop/hive/kafka/KafkaRecordIteratorTest.java create mode 100644 hive-kafka-reader/src/test/org/apache/hadoop/hive/kafka/KafkaRecordWritableTest.java create mode 100644 hive-kafka-reader/src/test/org/apache/hadoop/hive/kafka/KafkaScanTrimmerTest.java create mode 100644 hive-kafka-reader/src/test/org/apache/hadoop/hive/kafka/KafkaStreamingUtilsTest.java create mode 100644 ql/src/test/queries/clientpositive/kafka_storage_handler.q create mode 100644 ql/src/test/results/clientpositive/druid/kafka_storage_handler.q.out diff --git a/hive-kafka-reader/pom.xml b/hive-kafka-reader/pom.xml new file mode 100644 index 000000000000..86dbcde8f4f9 --- /dev/null +++ b/hive-kafka-reader/pom.xml @@ -0,0 +1,161 @@ + + + + + + org.apache.hive + hive + 4.0.0-SNAPSHOT + ../pom.xml + + 4.0.0 + + + .. + 1.0.1 + + + hive-kafka-reader + jar + Hive Kafka Streaming Ingestion Handler + + + + + + org.apache.hive + hive-exec + provided + ${project.version} + + + org.apache.hadoop + hadoop-common + + + org.apache.hadoop + hadoop-client + + + org.apache.kafka + kafka-clients + ${kafka.version} + + + + junit + junit + ${junit.version} + test + + + org.apache.kafka + kafka-clients + ${kafka.version} + test + test + + + + org.apache.kafka + kafka_2.11 + ${kafka.version} + test + test + + + org.apache.kafka + kafka_2.11 + ${kafka.version} + test + + + + + + dev-fast-build + + + skipShade + !true + + + + + + org.apache.maven.plugins + maven-shade-plugin + ${maven.shade.plugin.version} + + + + + package + + shade + + + true + false + + + org.apache.kafka:* + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + static/ + + + + + + + + + + + + + ${basedir}/src/java + ${basedir}/src/test + + + org.apache.maven.plugins + maven-jar-plugin + + + + test-jar + + + + + + + \ No newline at end of file diff --git a/hive-kafka-reader/src/java/org/apache/hadoop/hive/kafka/GenericKafkaSerDe.java b/hive-kafka-reader/src/java/org/apache/hadoop/hive/kafka/GenericKafkaSerDe.java new file mode 100644 index 000000000000..827b4475e275 --- /dev/null +++ b/hive-kafka-reader/src/java/org/apache/hadoop/hive/kafka/GenericKafkaSerDe.java @@ -0,0 +1,195 @@ +/* + * 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.Lists; +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.JsonSerDe; +import org.apache.hadoop.hive.serde2.SerDeException; +import org.apache.hadoop.hive.serde2.SerDeStats; +import org.apache.hadoop.hive.serde2.SerDeUtils; +import org.apache.hadoop.hive.serde2.avro.AvroGenericRecordWritable; +import org.apache.hadoop.hive.serde2.avro.AvroSerDe; +import org.apache.hadoop.hive.serde2.columnar.BytesRefWritable; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory; +import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; +import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; +import org.apache.hadoop.io.IntWritable; +import org.apache.hadoop.io.LongWritable; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.io.Writable; +import org.apache.hive.common.util.ReflectionUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; +import java.io.ByteArrayInputStream; +import java.io.DataInputStream; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Properties; +import java.util.stream.Collectors; +; + +public class GenericKafkaSerDe extends AbstractSerDe +{ + private static final Logger log = LoggerFactory.getLogger(GenericKafkaSerDe.class); + // ORDER of fields and types matters here + public static final ImmutableList METADATA_COLUMN_NAMES = ImmutableList.of( + KafkaStorageHandler.__PARTITION, + KafkaStorageHandler.__OFFSET, + KafkaStorageHandler.__TIMESTAMP + ); + public static final ImmutableList METADATA_PRIMITIVE_TYPE_INFO = ImmutableList.of( + TypeInfoFactory.intTypeInfo, + TypeInfoFactory.longTypeInfo, + TypeInfoFactory.longTypeInfo + ); + + private AbstractSerDe delegateSerDe; + private ObjectInspector objectInspector; + private final List columnNames = Lists.newArrayList(); + StructObjectInspector delegateObjectInspector; + + @Override + public void initialize(@Nullable Configuration conf, Properties tbl) throws SerDeException + { + final String className = tbl.getProperty(KafkaStorageHandler.SERDE_CLASS_NAME, KafkaJsonSerDe.class.getName()); + delegateSerDe = createDelegate(className); + delegateSerDe.initialize(conf, tbl); + log.info("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(); + + final List inspectors; + // Get column names and types + String columnNameProperty = tbl.getProperty(serdeConstants.LIST_COLUMNS); + final String columnNameDelimiter = tbl.containsKey(serdeConstants.COLUMN_NAME_DELIMITER) ? tbl + .getProperty(serdeConstants.COLUMN_NAME_DELIMITER) : String.valueOf(SerDeUtils.COMMA); + // all table column names + if (!columnNameProperty.isEmpty()) { + columnNames.addAll(Arrays.asList(columnNameProperty.split(columnNameDelimiter))); + } + + columnNames.addAll(METADATA_COLUMN_NAMES); + + if (log.isDebugEnabled()) { + log.debug("columns: {}, {}", columnNameProperty, columnNames); + } + + inspectors = new ArrayList<>(columnNames.size()); + inspectors.addAll(delegateObjectInspector.getAllStructFieldRefs() + .stream() + .map(structField -> structField.getFieldObjectInspector()) + .collect(Collectors.toList())); + inspectors.addAll(METADATA_PRIMITIVE_TYPE_INFO.stream() + .map(KafkaJsonSerDe.typeInfoToObjectInspector) + .collect(Collectors.toList())); + + objectInspector = ObjectInspectorFactory.getStandardStructObjectInspector(columnNames, inspectors); + } + + private AbstractSerDe createDelegate(String className) + { + final Class clazz; + try { + clazz = (Class) Class.forName(className); + } + catch (ClassNotFoundException e) { + log.error("Failed a loading delegate SerDe {}", className); + throw new RuntimeException(e); + } + // we are not setting conf thus null is okay + return ReflectionUtil.newInstance(clazz, null); + } + + @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(); + try { + ByteArrayInputStream byteArrayInputStream = new ByteArrayInputStream(record.getValue()); + avroGenericRecordWritable.readFields(new DataInputStream(byteArrayInputStream)); + } + 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 BytesRefWritable(record.getValue())); + } + + return columnNames.stream().map(name -> { + switch (name) { + case KafkaStorageHandler.__PARTITION: + return new IntWritable(record.getPartition()); + case KafkaStorageHandler.__OFFSET: + return new LongWritable(record.getOffset()); + case KafkaStorageHandler.__TIMESTAMP: + return new LongWritable(record.getTimestamp()); + default: + return delegateObjectInspector.getStructFieldData(row, delegateObjectInspector.getStructFieldRef(name)); + } + }).collect(Collectors.toList()); + } + + @Override + public ObjectInspector getObjectInspector() + { + return objectInspector; + } +} diff --git a/hive-kafka-reader/src/java/org/apache/hadoop/hive/kafka/KafkaJsonSerDe.java b/hive-kafka-reader/src/java/org/apache/hadoop/hive/kafka/KafkaJsonSerDe.java new file mode 100644 index 000000000000..3a2d7d77d1f1 --- /dev/null +++ b/hive-kafka-reader/src/java/org/apache/hadoop/hive/kafka/KafkaJsonSerDe.java @@ -0,0 +1,270 @@ +/* + * 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.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.Maps; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.common.type.HiveChar; +import org.apache.hadoop.hive.common.type.HiveDecimal; +import org.apache.hadoop.hive.common.type.HiveVarchar; +import org.apache.hadoop.hive.common.type.TimestampTZ; +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.SerDeStats; +import org.apache.hadoop.hive.serde2.SerDeUtils; +import org.apache.hadoop.hive.serde2.columnar.BytesRefWritable; +import org.apache.hadoop.hive.serde2.io.ByteWritable; +import org.apache.hadoop.hive.serde2.io.DoubleWritable; +import org.apache.hadoop.hive.serde2.io.HiveCharWritable; +import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable; +import org.apache.hadoop.hive.serde2.io.HiveVarcharWritable; +import org.apache.hadoop.hive.serde2.io.ShortWritable; +import org.apache.hadoop.hive.serde2.io.TimestampLocalTZWritable; +import org.apache.hadoop.hive.serde2.io.TimestampWritable; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory; +import org.apache.hadoop.hive.serde2.typeinfo.CharTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.TimestampLocalTZTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils; +import org.apache.hadoop.io.BooleanWritable; +import org.apache.hadoop.io.FloatWritable; +import org.apache.hadoop.io.IntWritable; +import org.apache.hadoop.io.LongWritable; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.io.Writable; +import org.joda.time.format.DateTimeFormatter; +import org.joda.time.format.DateTimeFormatterBuilder; +import org.joda.time.format.DateTimeParser; +import org.joda.time.format.ISODateTimeFormat; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.time.Instant; +import java.time.ZonedDateTime; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.function.Function; +import java.util.stream.Collectors; + +public class KafkaJsonSerDe extends AbstractSerDe +{ + private static final Logger LOG = LoggerFactory.getLogger(KafkaJsonSerDe.class); + private static final DateTimeFormatter TS_PARSER = createAutoParser(); + public static Function typeInfoToObjectInspector = typeInfo -> PrimitiveObjectInspectorFactory + .getPrimitiveWritableObjectInspector( + TypeInfoFactory.getPrimitiveTypeInfo(typeInfo.getTypeName())); + private List columnNames; + private List columnTypes; + private ObjectInspector inspector; + private final ObjectMapper mapper = new ObjectMapper(); + private long rowCount = 0L; + private long rawDataSize = 0L; + + + @Override + public void initialize( + @Nullable Configuration conf, Properties tbl + ) + { + final List inspectors; + // Get column names and types + String columnNameProperty = tbl.getProperty(serdeConstants.LIST_COLUMNS); + String columnTypeProperty = tbl.getProperty(serdeConstants.LIST_COLUMN_TYPES); + final String columnNameDelimiter = tbl.containsKey(serdeConstants.COLUMN_NAME_DELIMITER) ? tbl + .getProperty(serdeConstants.COLUMN_NAME_DELIMITER) : String.valueOf(SerDeUtils.COMMA); + // all table column names + if (!columnNameProperty.isEmpty()) { + columnNames = Arrays.asList(columnNameProperty.split(columnNameDelimiter)); + } + // all column types + if (!columnTypeProperty.isEmpty()) { + columnTypes = TypeInfoUtils.getTypeInfosFromTypeString(columnTypeProperty); + } + + if (LOG.isDebugEnabled()) { + LOG.debug("columns: {}, {}", columnNameProperty, columnNames); + LOG.debug("types: {}, {} ", columnTypeProperty, columnTypes); + } + + inspectors = columnTypes.stream().map(typeInfoToObjectInspector).collect(Collectors.toList()); + inspector = ObjectInspectorFactory.getStandardStructObjectInspector(columnNames, inspectors); + } + + @Override + public Class getSerializedClass() + { + return BytesRefWritable.class; + } + + @Override + public Writable serialize( + Object obj, ObjectInspector objInspector + ) throws SerDeException + { + throw new SerDeException("unimplemented"); + } + + @Override + public SerDeStats getSerDeStats() + { + SerDeStats serDeStats = new SerDeStats(); + serDeStats.setRawDataSize(rawDataSize); + serDeStats.setRowCount(rowCount); + return serDeStats; + } + + @Override + public Object deserialize(Writable blob) throws SerDeException + { + BytesRefWritable record = (BytesRefWritable) blob; + Map payload; + try { + payload = parseAsJson(record.getData()); + rowCount += 1; + rawDataSize += record.getData().length; + } + catch (IOException e) { + throw new SerDeException(e); + } + + final List output = new ArrayList<>(columnNames.size()); + + for (int i = 0; i < columnNames.size(); i++) { + final String name = columnNames.get(i); + final TypeInfo typeInfo = columnTypes.get(i); + final JsonNode value = payload.get(name); + if (value == null) { + output.add(null); + } else { + switch (columnTypes.get(i).getCategory()) { + case PRIMITIVE: + output.add(parseAsPrimitive(value, typeInfo)); + break; + case MAP: + case LIST: + case UNION: + case STRUCT: + throw new SerDeException("not supported yet"); + } + } + + } + return output; + } + + private Object parseAsPrimitive(JsonNode value, TypeInfo typeInfo) throws SerDeException + { + switch (TypeInfoFactory.getPrimitiveTypeInfo(typeInfo.getTypeName()) + .getPrimitiveCategory()) { + case TIMESTAMP: + TimestampWritable timestampWritable = new TimestampWritable(); + timestampWritable.setTime(TS_PARSER.parseMillis(value.textValue())); + return timestampWritable; + + case TIMESTAMPLOCALTZ: + final long numberOfMillis = TS_PARSER.parseMillis(value.textValue()); + return new TimestampLocalTZWritable(new TimestampTZ(ZonedDateTime + .ofInstant( + Instant.ofEpochMilli(numberOfMillis), + ((TimestampLocalTZTypeInfo) typeInfo).timeZone() + ))); + + case BYTE: + return new ByteWritable((byte) value.intValue()); + case SHORT: + return (new ShortWritable(value.shortValue())); + case INT: + return new IntWritable(value.intValue()); + case LONG: + return (new LongWritable((value.longValue()))); + case FLOAT: + return (new FloatWritable(value.floatValue())); + case DOUBLE: + return (new DoubleWritable(value.doubleValue())); + case DECIMAL: + return (new HiveDecimalWritable(HiveDecimal.create(value.decimalValue()))); + case CHAR: + return (new HiveCharWritable( + new HiveChar(value.textValue(), ((CharTypeInfo) typeInfo).getLength()))); + case VARCHAR: + return (new HiveVarcharWritable( + new HiveVarchar(value.textValue(), ((CharTypeInfo) typeInfo).getLength()))); + case STRING: + return (new Text(value.textValue())); + case BOOLEAN: + return (new BooleanWritable(value.isBoolean() ? value.booleanValue() : Boolean.valueOf(value.textValue()))); + default: + throw new SerDeException("Unknown type: " + typeInfo.getTypeName()); + } + } + + private Map parseAsJson(byte[] value) throws IOException + { + JsonNode document = mapper.readValue(value, JsonNode.class); + //Hive Column names are case insensitive. + Map documentMap = Maps.newTreeMap(String.CASE_INSENSITIVE_ORDER); + document.fields() + .forEachRemaining(field -> documentMap.put(field.getKey().toLowerCase(), field.getValue())); + return documentMap; + } + + @Override + public ObjectInspector getObjectInspector() throws SerDeException + { + if (inspector == null) { + throw new SerDeException("null inspector ??"); + } + return inspector; + } + + private static DateTimeFormatter createAutoParser() + { + final DateTimeFormatter offsetElement = new DateTimeFormatterBuilder() + .appendTimeZoneOffset("Z", true, 2, 4) + .toFormatter(); + + DateTimeParser timeOrOffset = new DateTimeFormatterBuilder() + .append( + null, + new DateTimeParser[]{ + new DateTimeFormatterBuilder().appendLiteral('T').toParser(), + new DateTimeFormatterBuilder().appendLiteral(' ').toParser() + } + ) + .appendOptional(ISODateTimeFormat.timeElementParser().getParser()) + .appendOptional(offsetElement.getParser()) + .toParser(); + + return new DateTimeFormatterBuilder() + .append(ISODateTimeFormat.dateElementParser()) + .appendOptional(timeOrOffset) + .toFormatter(); + } +} diff --git a/hive-kafka-reader/src/java/org/apache/hadoop/hive/kafka/KafkaPullerInputFormat.java b/hive-kafka-reader/src/java/org/apache/hadoop/hive/kafka/KafkaPullerInputFormat.java new file mode 100644 index 000000000000..84a0d621b6a3 --- /dev/null +++ b/hive-kafka-reader/src/java/org/apache/hadoop/hive/kafka/KafkaPullerInputFormat.java @@ -0,0 +1,234 @@ +/* + * 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.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.ql.exec.SerializationUtilities; +import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc; +import org.apache.hadoop.hive.ql.plan.TableScanDesc; +import org.apache.hadoop.io.NullWritable; +import org.apache.hadoop.mapred.InputSplit; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapred.RecordReader; +import org.apache.hadoop.mapred.Reporter; +import org.apache.hadoop.mapreduce.InputFormat; +import org.apache.hadoop.mapreduce.JobContext; +import org.apache.hadoop.mapreduce.TaskAttemptContext; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.TopicPartition; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.stream.Collectors; + + +/** + * Kafka puller input format is in charge of reading a exact set of records from a Kafka Queue + * The input split will contain the set of topic partition and start/end offsets + * Records will be returned as bytes + */ +public class KafkaPullerInputFormat extends InputFormat + implements org.apache.hadoop.mapred.InputFormat +{ + + public static final String HIVE_KAFKA_TOPIC = "kafka.topic"; + public static final String CONSUMER_CONFIGURATION_PREFIX = "kafka.consumer"; + public static final String HIVE_KAFKA_BOOTSTRAP_SERVERS = "kafka.bootstrap.servers"; + public static final String GENERATION_TIMEOUT_MS = "hive.kafka.split.generation.timeout.ms"; + + private static final Logger log = LoggerFactory.getLogger(KafkaPullerInputFormat.class); + + + @Override + public InputSplit[] getSplits( + JobConf jobConf, int i + ) throws IOException + { + List inputSplits = null; + try { + inputSplits = computeSplits(jobConf); + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new IOException(e); + } + InputSplit[] inputSplitsArray = new InputSplit[inputSplits.size()]; + return inputSplits.toArray(inputSplitsArray); + } + + /** + * Build a full scan using Kafka list partition then beginning/end offsets + * This function might block duo to calls like + * org.apache.kafka.clients.consumer.KafkaConsumer#beginningOffsets(java.util.Collection) + * + * @param topic kafka topic + * @param consumer initialized kafka consumer + * @param tablePaths hive table path + * + * @return full scan input split collection based on Kafka metadata APIs + */ + private static List buildFullScanFromKafka( + String topic, + KafkaConsumer consumer, + Path[] tablePaths + ) + { + final Map starOffsetsMap; + final Map endOffsetsMap; + + final List topicPartitions; + topicPartitions = fetchTopicPartitions(topic, consumer); + starOffsetsMap = consumer.beginningOffsets(topicPartitions); + endOffsetsMap = consumer.endOffsets(topicPartitions); + + if (log.isDebugEnabled()) { + log.info( + "Found the following partitions [{}]", + topicPartitions.stream().map(topicPartition -> topicPartition.toString()) + .collect(Collectors.joining(",")) + ); + starOffsetsMap.forEach((tp, start) -> log.info("TPartition [{}],Start offsets [{}]", tp, start)); + endOffsetsMap.forEach((tp, end) -> log.info("TPartition [{}],End offsets [{}]", tp, end)); + } + return topicPartitions.stream().map( + topicPartition -> new KafkaPullerInputSplit( + topicPartition.topic(), + topicPartition.partition(), + starOffsetsMap.get(topicPartition), + endOffsetsMap.get(topicPartition), + tablePaths[0] + )).collect(Collectors.toList()); + } + + private List computeSplits(Configuration configuration) + throws IOException, InterruptedException + { + // this will be used to harness some KAFKA blocking calls + final ExecutorService EXECUTOR = Executors.newSingleThreadExecutor(); + try (KafkaConsumer consumer = new KafkaConsumer(KafkaStreamingUtils.consumerProperties(configuration))) { + final String topic = configuration.get(HIVE_KAFKA_TOPIC); + final long timeoutMs = configuration.getLong(GENERATION_TIMEOUT_MS, 5000); + // hive depends on FileSplits + JobConf jobConf = new JobConf(configuration); + Path[] tablePaths = org.apache.hadoop.mapred.FileInputFormat.getInputPaths(jobConf); + + Future> futureFullHouse = EXECUTOR.submit(() -> buildFullScanFromKafka( + topic, + consumer, + tablePaths + )); + List fullHouse; + try { + fullHouse = futureFullHouse.get(timeoutMs, TimeUnit.MILLISECONDS); + } + catch (TimeoutException | ExecutionException e) { + futureFullHouse.cancel(true); + log.error("can not generate full scan split", e); + // at this point we can not go further fail split generation + throw new IOException(e); + } + + + final ImmutableMap.Builder builder = new ImmutableMap.Builder(); + fullHouse.stream().forEach(input -> builder.put(new TopicPartition( + input.getTopic(), + input.getPartition() + ), input)); + + final KafkaScanTrimmer kafkaScanTrimmer = new KafkaScanTrimmer(builder.build(), consumer); + final String filterExprSerialized = configuration.get(TableScanDesc.FILTER_EXPR_CONF_STR); + + if (filterExprSerialized != null && !filterExprSerialized.isEmpty()) { + ExprNodeGenericFuncDesc filterExpr = SerializationUtilities.deserializeExpression(filterExprSerialized); + log.info("Kafka trimmer working on Filter tree {}", filterExpr.getExprString()); + Callable> trimmerWorker = () -> kafkaScanTrimmer.computeOptimizedScan(filterExpr) + .entrySet() + .stream() + .map(entry -> entry.getValue()) + .collect(Collectors.toList()); + + Future> futureTinyHouse = EXECUTOR.submit(trimmerWorker); + try { + return futureTinyHouse.get(timeoutMs, TimeUnit.MILLISECONDS); + } + catch (ExecutionException | TimeoutException e) { + futureTinyHouse.cancel(true); + log.error("Had issue with trimmer will return full scan ", e); + return fullHouse; + } + } + //Case null: it can be filter evaluated to false or no filter at all thus return full scan + return fullHouse; + } + finally { + EXECUTOR.shutdown(); + } + } + + + private static List fetchTopicPartitions(String topic, KafkaConsumer consumer) + { + // this will block till REQUEST_TIMEOUT_MS_CONFIG = "request.timeout.ms" + // then throws org.apache.kafka.common.errors.TimeoutException if can not fetch metadata + // @TODO add retry logic maybe + List partitions = consumer.partitionsFor(topic); + return partitions.stream().map(p -> new TopicPartition(topic, p.partition())) + .collect(Collectors.toList()); + } + + @Override + public RecordReader getRecordReader( + InputSplit inputSplit, + JobConf jobConf, Reporter reporter + ) throws IOException + { + return new KafkaPullerRecordReader((KafkaPullerInputSplit) inputSplit, jobConf); + } + + @Override + public List getSplits( + JobContext jobContext + ) throws IOException, InterruptedException + { + return computeSplits(jobContext.getConfiguration()).stream() + .map(kafkaPullerInputSplit -> (org.apache.hadoop.mapreduce.InputSplit) kafkaPullerInputSplit) + .collect(Collectors.toList()); + } + + @Override + public org.apache.hadoop.mapreduce.RecordReader createRecordReader( + org.apache.hadoop.mapreduce.InputSplit inputSplit, TaskAttemptContext taskAttemptContext + ) throws IOException, InterruptedException + { + return new KafkaPullerRecordReader(); + } +} diff --git a/hive-kafka-reader/src/java/org/apache/hadoop/hive/kafka/KafkaPullerInputSplit.java b/hive-kafka-reader/src/java/org/apache/hadoop/hive/kafka/KafkaPullerInputSplit.java new file mode 100644 index 000000000000..9c6e866b917a --- /dev/null +++ b/hive-kafka-reader/src/java/org/apache/hadoop/hive/kafka/KafkaPullerInputSplit.java @@ -0,0 +1,231 @@ +/* + * 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.Objects; +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.mapred.FileSplit; + +import javax.annotation.Nullable; +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; +import java.util.Collections; +import java.util.List; + +public class KafkaPullerInputSplit extends FileSplit + implements org.apache.hadoop.mapred.InputSplit +{ + private String topic; + private long startOffset; + private int partition; + private long endOffset; + + public KafkaPullerInputSplit() + { + super((Path) null, 0, 0, (String[]) null); + } + + public KafkaPullerInputSplit( + String topic, int partition, long startOffset, + long endOffset, + Path dummyPath + ) + { + super(dummyPath, 0, 0, (String[]) null); + this.topic = topic; + this.startOffset = startOffset; + this.partition = partition; + this.endOffset = endOffset; + Preconditions.checkArgument( + startOffset >= 0 && startOffset <= endOffset, + "start [%s] has to be positive and >= end [%]", + startOffset, + endOffset + ); + } + + @Override + public long getLength() + { + return 0; + } + + @Override + public String[] getLocations() throws IOException + { + return new String[0]; + } + + @Override + public void write(DataOutput dataOutput) throws IOException + { + super.write(dataOutput); + dataOutput.writeUTF(topic); + dataOutput.writeInt(partition); + dataOutput.writeLong(startOffset); + dataOutput.writeLong(endOffset); + } + + @Override + public void readFields(DataInput dataInput) throws IOException + { + super.readFields(dataInput); + topic = dataInput.readUTF(); + partition = dataInput.readInt(); + startOffset = dataInput.readLong(); + endOffset = dataInput.readLong(); + Preconditions.checkArgument( + startOffset >= 0 && startOffset <= endOffset, + "start [%s] has to be positive and >= end [%]", + startOffset, + endOffset + ); + } + + public String getTopic() + { + return topic; + } + + public int getPartition() + { + return partition; + } + + public long getStartOffset() + { + return startOffset; + } + + public long getEndOffset() + { + return endOffset; + } + + + + + /** + * Compute the intersection of 2 splits. Splits must share the same topic and partition number. + * + * @param split1 + * @param split2 + * + * @return new split that represents range intersection or null if it is not overlapping + */ + @Nullable + public static KafkaPullerInputSplit intersectRange(KafkaPullerInputSplit split1, KafkaPullerInputSplit split2) { + assert (split1.topic == split2.topic); + assert (split1.partition == split2.partition); + final long startOffset = Math.max(split1.getStartOffset(), split2.getStartOffset()); + final long endOffset = Math.min(split1.getEndOffset(), split2.getEndOffset()); + if (startOffset > endOffset) { + // there is no overlapping + return null; + } + return new KafkaPullerInputSplit(split1.topic, split1.partition, startOffset, endOffset, split1.getPath()); + } + + /** + * Compute union of ranges between splits. Splits must share the same topic and partition + * + * @param split1 + * @param split2 + * + * @return new split with a range including both splits. + */ + public static KafkaPullerInputSplit unionRange(KafkaPullerInputSplit split1, KafkaPullerInputSplit split2) { + assert (split1.topic == 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()); + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (!(o instanceof KafkaPullerInputSplit)) { + return false; + } + KafkaPullerInputSplit that = (KafkaPullerInputSplit) o; + return Objects.equal(getTopic(), that.getTopic()) && Objects + .equal(getStartOffset(), that.getStartOffset()) && Objects + .equal(getPartition(), that.getPartition()) && Objects + .equal(getEndOffset(), that.getEndOffset()); + } + + @Override + public int hashCode() + { + return Objects.hashCode(getTopic(), getStartOffset(), getPartition(), getEndOffset()); + } + + @Override + public String toString() + { + return "KafkaPullerInputSplit{" + + "topic='" + topic + '\'' + + ", startOffset=" + startOffset + + ", partition=" + partition + + ", endOffset=" + endOffset + + ", path=" + super.getPath().toString() + + '}'; + } + + public static KafkaPullerInputSplit copyOf(KafkaPullerInputSplit other) + { + return new KafkaPullerInputSplit( + other.getTopic(), + other.getPartition(), + other.getStartOffset(), + other.getEndOffset(), + other.getPath() + ); + } + + public KafkaPullerInputSplit clone() + { + return copyOf(this); + } + + public static List slice(long sliceSize, final KafkaPullerInputSplit split) { + if (split.getEndOffset() - split.getStartOffset() > sliceSize) { + ImmutableList.Builder builder = ImmutableList.builder(); + long start = split.getStartOffset(); + while (start < split.getEndOffset() - sliceSize) { + builder.add(new KafkaPullerInputSplit(split.topic, split.partition, start, start + sliceSize + 1, split.getPath())); + start += sliceSize + 1; + } + // last split + if (start < split.getEndOffset()) { + builder.add(new KafkaPullerInputSplit(split.topic, split.partition, start, split.getEndOffset(), split.getPath())); + } + return builder.build(); + } + + return Collections.singletonList(copyOf(split)); + } +} diff --git a/hive-kafka-reader/src/java/org/apache/hadoop/hive/kafka/KafkaPullerRecordReader.java b/hive-kafka-reader/src/java/org/apache/hadoop/hive/kafka/KafkaPullerRecordReader.java new file mode 100644 index 000000000000..229a1e4ad9fa --- /dev/null +++ b/hive-kafka-reader/src/java/org/apache/hadoop/hive/kafka/KafkaPullerRecordReader.java @@ -0,0 +1,172 @@ +/* + * 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.io.Closer; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.io.NullWritable; +import org.apache.hadoop.mapreduce.RecordReader; +import org.apache.hadoop.mapreduce.TaskAttemptContext; +import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.common.TopicPartition; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.Iterator; +import java.util.Properties; + +public class KafkaPullerRecordReader extends RecordReader + implements org.apache.hadoop.mapred.RecordReader { + + private static final Logger log = LoggerFactory.getLogger(KafkaPullerRecordReader.class); + public static final String HIVE_KAFKA_POLL_TIMEOUT = "hive.kafka.poll.timeout.ms"; + public static final long DEFAULT_CONSUMER_POLL_TIMEOUT_MS = 5000l; + + private final Closer closer = Closer.create(); + private KafkaConsumer consumer = null; + private Configuration config = null; + private KafkaRecordWritable currentWritableValue; + private Iterator> recordsCursor = null; + + private TopicPartition topicPartition; + private long startOffset; + private long endOffset; + + private long totalNumberRecords = 0l; + private long consumedRecords = 0l; + private long readBytes = 0l; + private long pollTimeout; + private volatile boolean started = false; + + public KafkaPullerRecordReader() { + } + + private void initConsumer() { + if (consumer == null) { + log.info("Initializing Kafka Consumer"); + final Properties properties = KafkaStreamingUtils.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); + consumer = new KafkaConsumer(properties); + closer.register(consumer); + } + } + + public KafkaPullerRecordReader(KafkaPullerInputSplit inputSplit, Configuration jobConf) { + initialize(inputSplit, jobConf); + } + + synchronized private void initialize(KafkaPullerInputSplit inputSplit, Configuration jobConf) { + if (!started) { + this.config = jobConf; + computeTopicPartitionOffsets(inputSplit); + initConsumer(); + pollTimeout = config.getLong(HIVE_KAFKA_POLL_TIMEOUT, DEFAULT_CONSUMER_POLL_TIMEOUT_MS); + log.debug("Consumer poll timeout [{}] ms", pollTimeout); + recordsCursor = new KafkaRecordIterator(consumer, topicPartition, startOffset, endOffset, pollTimeout); + started = true; + } + } + + private void computeTopicPartitionOffsets(KafkaPullerInputSplit split) { + String topic = split.getTopic(); + int partition = split.getPartition(); + startOffset = split.getStartOffset(); + endOffset = split.getEndOffset(); + topicPartition = new TopicPartition(topic, partition); + Preconditions.checkState( + startOffset >= 0 && startOffset <= endOffset, + "Start [%s] has to be positive and less or equal than End [%s]", + startOffset, + endOffset + ); + totalNumberRecords += endOffset - startOffset; + } + + @Override synchronized public void initialize(org.apache.hadoop.mapreduce.InputSplit inputSplit, TaskAttemptContext context) { + initialize((KafkaPullerInputSplit) inputSplit, context.getConfiguration()); + } + + @Override public boolean next(NullWritable nullWritable, KafkaRecordWritable bytesWritable) { + if (started && recordsCursor.hasNext()) { + ConsumerRecord record = recordsCursor.next(); + bytesWritable.set(record); + consumedRecords += 1; + readBytes += record.serializedValueSize(); + return true; + } + return false; + } + + @Override public NullWritable createKey() { + return NullWritable.get(); + } + + @Override public KafkaRecordWritable createValue() { + return new KafkaRecordWritable(); + } + + @Override public long getPos() throws IOException { + + return consumedRecords; + } + + @Override public boolean nextKeyValue() throws IOException { + currentWritableValue = new KafkaRecordWritable(); + if (next(NullWritable.get(), currentWritableValue)) { + return true; + } + currentWritableValue = null; + return false; + } + + @Override public NullWritable getCurrentKey() throws IOException, InterruptedException { + return NullWritable.get(); + } + + @Override public KafkaRecordWritable getCurrentValue() throws IOException, InterruptedException { + return Preconditions.checkNotNull(currentWritableValue); + } + + @Override public float getProgress() throws IOException { + if (consumedRecords == 0) { + return 0f; + } + if (consumedRecords >= totalNumberRecords) { + return 1f; + } + return consumedRecords * 1.0f / totalNumberRecords; + } + + @Override public void close() throws IOException { + if (!started) { + return; + } + log.trace("total read bytes [{}]", readBytes); + if (consumer != null) { + consumer.wakeup(); + } + closer.close(); + } +} diff --git a/hive-kafka-reader/src/java/org/apache/hadoop/hive/kafka/KafkaRecordIterator.java b/hive-kafka-reader/src/java/org/apache/hadoop/hive/kafka/KafkaRecordIterator.java new file mode 100644 index 000000000000..a5e1c0b24755 --- /dev/null +++ b/hive-kafka-reader/src/java/org/apache/hadoop/hive/kafka/KafkaRecordIterator.java @@ -0,0 +1,183 @@ +/* + * 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.Stopwatch; +import com.google.common.collect.ImmutableList; +import org.apache.kafka.clients.consumer.Consumer; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.common.TopicPartition; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; +import java.util.Collection; +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. + *

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

+ * When provided with an end offset it will return records up to the record with offset == endOffset - 1, + * Else If end offsets is null it will read up to the current end see {@link org.apache.kafka.clients.consumer.Consumer#endOffsets(Collection)} + */ +public class KafkaRecordIterator implements Iterator> +{ + private static final Logger log = LoggerFactory.getLogger(KafkaRecordIterator.class); + + private final Consumer consumer; + private final TopicPartition topicPartition; + private long endOffset; + private long startOffset; + private final long pollTimeout; + private final Stopwatch stopwatch = Stopwatch.createUnstarted(); + private ConsumerRecords records; + private long currentOffset; + private ConsumerRecord nextRecord; + private boolean hasMore = true; + private Iterator> cursor = null; + + /** + * @param consumer functional kafka consumer + * @param topicPartition kafka topic partition + * @param startOffset start position of stream. + * @param endOffset requested end position. If null will read up to current last + * @param pollTimeout poll time out in ms + */ + public KafkaRecordIterator( + Consumer consumer, TopicPartition topicPartition, + @Nullable Long startOffset, @Nullable Long endOffset, long pollTimeout + ) + { + this.consumer = Preconditions.checkNotNull(consumer, "Consumer can not be null"); + this.topicPartition = Preconditions.checkNotNull(topicPartition, "Topic partition can not be null"); + this.pollTimeout = pollTimeout; + Preconditions.checkState(pollTimeout > 0, "poll timeout has to be positive number"); + this.startOffset = startOffset == null ? -1l : startOffset; + this.endOffset = endOffset == null ? -1l : endOffset; + } + + public KafkaRecordIterator( + Consumer consumer, TopicPartition tp, long pollTimeout + ) + { + this(consumer, tp, null, null, pollTimeout); + } + + private void assignAndSeek() + { + // assign topic partition to consumer + final List topicPartitionList = ImmutableList.of(topicPartition); + if (log.isTraceEnabled()) { + stopwatch.reset().start(); + } + + consumer.assign(topicPartitionList); + // compute offsets and seek to start + if (startOffset > -1) { + log.info("Seeking to offset [{}] of topic partition [{}]", startOffset, topicPartition); + consumer.seek(topicPartition, startOffset); + } else { + log.info("Seeking to beginning of topic partition [{}]", topicPartition); + // seekToBeginning is lazy thus need to call position() or poll(0) + this.consumer.seekToBeginning(Collections.singleton(topicPartition)); + startOffset = consumer.position(topicPartition); + } + if (endOffset == -1) { + this.endOffset = consumer.endOffsets(topicPartitionList).get(topicPartition); + log.debug("EndOffset is {}", endOffset); + } + currentOffset = consumer.position(topicPartition); + Preconditions.checkState(this.endOffset >= currentOffset, + "End offset [%s] need to be greater than start offset [%s]", this.endOffset, currentOffset + ); + log.info("Kafka Iterator ready, assigned TopicPartition [{}]; startOffset [{}]; endOffset [{}]", topicPartition, + currentOffset, this.endOffset + ); + if (log.isTraceEnabled()) { + stopwatch.stop(); + log.trace("Time to assign and seek [{}] ms", stopwatch.elapsed(TimeUnit.MILLISECONDS)); + } + } + + @Override + public boolean hasNext() + { + if (records == null) { + assignAndSeek(); + } + //Init poll OR Need to poll at least one more record since currentOffset + 1 < endOffset + if (records == null || (hasMore == false && currentOffset + 1 < endOffset)) { + pollRecords(); + findNext(); + } + return hasMore; + } + + private void pollRecords() + { + if (log.isTraceEnabled()) { + stopwatch.reset().start(); + } + records = consumer.poll(pollTimeout); + if (log.isTraceEnabled()) { + stopwatch.stop(); + log.trace("Pulled [{}] records in [{}] ms", records.count(), + stopwatch.elapsed(TimeUnit.MILLISECONDS) + ); + } + Preconditions.checkState(!records.isEmpty() || currentOffset == endOffset, + "Current read offset [%s]-TopicPartition:[%s], End offset[%s]." + + "Consumer returned 0 record due to exhausted poll timeout [%s] ms", + currentOffset, topicPartition.toString(), endOffset, pollTimeout + ); + cursor = records.iterator(); + } + + @Override + public ConsumerRecord next() + { + ConsumerRecord value = nextRecord; + Preconditions.checkState(value.offset() < endOffset); + findNext(); + return Preconditions.checkNotNull(value); + } + + private void findNext() + { + if (cursor.hasNext()) { + nextRecord = cursor.next(); + hasMore = true; + if (nextRecord.offset() < endOffset) { + currentOffset = nextRecord.offset(); + return; + } + } + hasMore = false; + nextRecord = null; + } + +} diff --git a/hive-kafka-reader/src/java/org/apache/hadoop/hive/kafka/KafkaRecordWritable.java b/hive-kafka-reader/src/java/org/apache/hadoop/hive/kafka/KafkaRecordWritable.java new file mode 100644 index 000000000000..08cf53be6430 --- /dev/null +++ b/hive-kafka-reader/src/java/org/apache/hadoop/hive/kafka/KafkaRecordWritable.java @@ -0,0 +1,121 @@ +/* + * 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.io.Writable; +import org.apache.kafka.clients.consumer.ConsumerRecord; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; +import java.util.Arrays; +import java.util.Objects; + +/** + * Writable implementation of Kafka ConsumerRecord. + * Serialized in the form + * kafkaRecordTimestamp(long) | kafkaPartition (int) | recordOffset (long) | value.size (int) | value (byte []) + */ +public class KafkaRecordWritable implements Writable { + + private int partition; + private long offset; + private long timestamp; + private byte [] value; + + public static KafkaRecordWritable fromKafkaRecord(ConsumerRecord consumerRecord) { + return new KafkaRecordWritable(consumerRecord.partition(), consumerRecord.offset(), + consumerRecord.timestamp(), consumerRecord.value() + ); + } + + public void set(ConsumerRecord consumerRecord) { + this.partition = consumerRecord.partition(); + this.timestamp = consumerRecord.timestamp(); + this.offset = consumerRecord.offset(); + this.value = consumerRecord.value(); + } + + private KafkaRecordWritable(int partition, long offset, long timestamp, byte[] value) { + this.partition = partition; + this.offset = offset; + this.timestamp = timestamp; + this.value = value; + } + + public KafkaRecordWritable() { + } + + @Override public void write(DataOutput dataOutput) throws IOException { + dataOutput.writeLong(timestamp); + dataOutput.writeInt(partition); + dataOutput.writeLong(offset); + dataOutput.writeInt(value.length); + dataOutput.write(value); + } + + @Override public void readFields(DataInput dataInput) throws IOException { + timestamp = dataInput.readLong(); + partition = dataInput.readInt(); + offset = dataInput.readLong(); + int size = dataInput.readInt(); + if (size > 0) { + value = new byte[size]; + dataInput.readFully(value); + } else { + value = new byte[0]; + } + } + + public int getPartition() { + return partition; + } + + public long getOffset() { + return offset; + } + + public long getTimestamp() { + return timestamp; + } + + public byte[] getValue() { + return value; + } + + @Override public boolean equals(Object o) { + if (this == o) { + return true; + } + if (!(o instanceof KafkaRecordWritable)) { + return false; + } + KafkaRecordWritable that = (KafkaRecordWritable) o; + return getPartition() == that.getPartition() && getOffset() == that.getOffset() + && getTimestamp() == that.getTimestamp() && Arrays.equals(getValue(), that.getValue()); + } + + @Override public int hashCode() { + + int result = Objects.hash(getPartition(), getOffset(), getTimestamp()); + result = 31 * result + Arrays.hashCode(getValue()); + return result; + } + +} diff --git a/hive-kafka-reader/src/java/org/apache/hadoop/hive/kafka/KafkaScanTrimmer.java b/hive-kafka-reader/src/java/org/apache/hadoop/hive/kafka/KafkaScanTrimmer.java new file mode 100644 index 000000000000..4ce82b593933 --- /dev/null +++ b/hive-kafka-reader/src/java/org/apache/hadoop/hive/kafka/KafkaScanTrimmer.java @@ -0,0 +1,523 @@ +/* + * 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.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Maps; +import org.apache.hadoop.hive.ql.exec.FunctionRegistry; +import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf; +import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc; +import org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc; +import org.apache.hadoop.hive.ql.plan.ExprNodeDesc; +import org.apache.hadoop.hive.ql.plan.ExprNodeDescUtils; +import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDF; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDFBaseCompare; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDFBridge; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqual; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqualOrGreaterThan; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqualOrLessThan; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPGreaterThan; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPLessThan; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDFToBinary; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDFToChar; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDFToDate; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDFToDecimal; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDFToUnixTimeStamp; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDFToUtcTimestamp; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDFToVarchar; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.consumer.OffsetAndTimestamp; +import org.apache.kafka.common.TopicPartition; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; +import java.util.function.Predicate; + +/** + * Kafka Range trimmer, takes a full kafka scan and prune the scan based on a filter expression + * it is a Best effort trimmer and it can not replace the filter it self, filtration still takes place in Hive executor + */ +public class KafkaScanTrimmer +{ + private static final Logger log = LoggerFactory.getLogger(KafkaScanTrimmer.class); + 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 + */ + public KafkaScanTrimmer( + Map fullHouse, + KafkaConsumer kafkaConsumer + ) + { + this.fullHouse = fullHouse; + this.kafkaConsumer = kafkaConsumer; + } + + /** + * This might block due to calls like: + * org.apache.kafka.clients.consumer.KafkaConsumer#offsetsForTimes(java.util.Map) + * + * @param filterExpression filter expression to be used for pruning scan + * + * @return tiny house of of the full house based on filter expression + */ + public Map computeOptimizedScan(ExprNodeGenericFuncDesc filterExpression) + { + Map optimizedScan = parse(filterExpression); + + if (log.isDebugEnabled()) { + if (optimizedScan != null) { + log.debug("Optimized scan:"); + optimizedScan.forEach((tp, input) -> log.info( + "Topic-[{}] Partition-[{}] - Split startOffset [{}] :-> endOffset [{}]", + tp.topic(), + tp.partition(), + input.getStartOffset(), input.getEndOffset() + )); + } else { + log.debug("No optimization thus using full scan "); + fullHouse.forEach((tp, input) -> log.info( + "Topic-[{}] Partition-[{}] - Split startOffset [{}] :-> endOffset [{}]", + tp.topic(), + tp.partition(), + input.getStartOffset(), input.getEndOffset() + )); + } + } + return optimizedScan == null ? fullHouse : optimizedScan; + } + + /** + * @param expression filter to parse and trim the full scan + * + * @return Map of optimized kafka range scans or null if it is impossible to optimize. + */ + @Nullable + private Map parse(ExprNodeDesc expression) + { + if (expression.getClass() != ExprNodeGenericFuncDesc.class) { + return null; + } + // get the kind of expression + ExprNodeGenericFuncDesc expr = (ExprNodeGenericFuncDesc) expression; + Class op = expr.getGenericUDF().getClass(); + + // handle the logical operators + if (FunctionRegistry.isOpOr(expr)) { + return pushOrOp(expr); + } + if (FunctionRegistry.isOpAnd(expr)) { + return pushAndOp(expr); + } + + if (op == GenericUDFOPGreaterThan.class) { + return pushLeaf(expr, PredicateLeaf.Operator.LESS_THAN_EQUALS, true); + } else if (op == GenericUDFOPEqualOrGreaterThan.class) { + return pushLeaf(expr, PredicateLeaf.Operator.LESS_THAN, true); + } else if (op == GenericUDFOPLessThan.class) { + return pushLeaf(expr, PredicateLeaf.Operator.LESS_THAN, false); + } else if (op == GenericUDFOPEqualOrLessThan.class) { + return pushLeaf(expr, PredicateLeaf.Operator.LESS_THAN_EQUALS, false); + } else if (op == GenericUDFOPEqual.class) { + return pushLeaf(expr, PredicateLeaf.Operator.EQUALS, false); + // otherwise, we didn't understand it, so bailout + } else { + return null; + } + } + + + /** + * @param expr leaf node to push + * @param operator operator + * @param negation true if it is a negation, this is used to represent: + * GenericUDFOPGreaterThan and GenericUDFOPEqualOrGreaterThan + * using PredicateLeaf.Operator.LESS_THAN and PredicateLeaf.Operator.LESS_THAN_EQUALS + * + * @return leaf scan or null if can not figure out push down + */ + @Nullable + private Map pushLeaf( + ExprNodeGenericFuncDesc expr, PredicateLeaf.Operator operator, boolean negation + ) + { + if (expr.getChildren().size() != 2) { + return null; + } + GenericUDF genericUDF = expr.getGenericUDF(); + if (!(genericUDF instanceof GenericUDFBaseCompare)) { + return null; + } + ExprNodeDesc expr1 = expr.getChildren().get(0); + ExprNodeDesc expr2 = expr.getChildren().get(1); + // We may need to peel off the GenericUDFBridge that is added by CBO or user + if (expr1.getTypeInfo().equals(expr2.getTypeInfo())) { + expr1 = getColumnExpr(expr1); + expr2 = getColumnExpr(expr2); + } + + ExprNodeDesc[] extracted = ExprNodeDescUtils.extractComparePair(expr1, expr2); + if (extracted == null || (extracted.length > 2)) { + return null; + } + + ExprNodeColumnDesc columnDesc; + ExprNodeConstantDesc constantDesc; + final boolean flip; + + if (extracted[0] instanceof ExprNodeColumnDesc) { + columnDesc = (ExprNodeColumnDesc) extracted[0]; + constantDesc = (ExprNodeConstantDesc) extracted[1]; + flip = false; + + } else { + flip = true; + columnDesc = (ExprNodeColumnDesc) extracted[1]; + constantDesc = (ExprNodeConstantDesc) extracted[0]; + } + + if (columnDesc.getColumn().equals(KafkaStorageHandler.__PARTITION)) { + return buildScanFormPartitionPredicate( + fullHouse, + operator, + ((Number) constantDesc.getValue()).intValue(), + flip, + negation + ); + + } + if (columnDesc.getColumn().equals(KafkaStorageHandler.__OFFSET)) { + return buildScanFromOffsetPredicate( + fullHouse, + operator, + ((Number) constantDesc.getValue()).longValue(), + flip, + negation + ); + } + + if (columnDesc.getColumn().equals(KafkaStorageHandler.__TIMESTAMP)) { + long timestamp = ((Number) constantDesc.getValue()).longValue(); + return buildScanForTimesPredicate(fullHouse, operator, timestamp, flip, negation, kafkaConsumer); + } + return null; + } + + + /** + * Trim kafka scan using a leaf binary predicate on partition column + * + * @param fullScan kafka full scan to be optimized + * @param operator predicate operator, equal, lessThan or lessThanEqual + * @param partitionConst partition constant value + * @param flip true if the position of column and constant is flipped by default assuming column OP constant + * @param negation true if the expression is a negation of the original expression + * + * @return filtered kafka scan + */ + @VisibleForTesting + protected static Map buildScanFormPartitionPredicate( + Map fullScan, + PredicateLeaf.Operator operator, int partitionConst, boolean flip, boolean negation + ) + { + final Predicate predicate; + final Predicate intermediatePredicate; + switch (operator) { + case EQUALS: + predicate = topicPartition -> topicPartition != null && topicPartition.partition() == partitionConst; + break; + case LESS_THAN: + intermediatePredicate = flip + ? topicPartition -> topicPartition != null + && partitionConst < topicPartition.partition() + : topicPartition -> topicPartition != null + && topicPartition.partition() < partitionConst; + + predicate = negation ? intermediatePredicate.negate() : intermediatePredicate; + break; + case LESS_THAN_EQUALS: + intermediatePredicate = flip ? topicPartition -> topicPartition != null + && partitionConst + <= topicPartition.partition() + : topicPartition -> topicPartition != null + && topicPartition.partition() <= partitionConst; + + predicate = negation ? intermediatePredicate.negate() : intermediatePredicate; + break; + default: + //Default to select * for unknown cases + predicate = topicPartition -> true; + } + + 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())); + return builder.build(); + } + + /** + * @param fullScan full kafka scan to be pruned + * @param operator operator kind + * @param offsetConst offset constant value + * @param flip true if position of constant and column were flipped by default assuming COLUMN OP CONSTANT + * @param negation true if the expression is a negation of the original expression + * + * @return optimized kafka scan + */ + @VisibleForTesting + protected static Map buildScanFromOffsetPredicate( + final Map fullScan, + PredicateLeaf.Operator operator, long offsetConst, boolean flip, boolean negation + ) + { + final boolean isEndBound; + final long startOffset; + final long endOffset; + + if (flip == false && negation == false || flip == true && negation == true) { + isEndBound = true; + } else { + isEndBound = false; + } + switch (operator) { + case LESS_THAN_EQUALS: + if (isEndBound) { + startOffset = -1; + endOffset = negation ? offsetConst : offsetConst + 1; + } else { + endOffset = -1; + startOffset = negation ? offsetConst + 1 : offsetConst; + } + break; + case EQUALS: + startOffset = offsetConst; + endOffset = offsetConst + 1; + break; + case LESS_THAN: + if (isEndBound) { + endOffset = negation ? offsetConst + 1 : offsetConst; + startOffset = -1; + } else { + endOffset = -1; + startOffset = negation ? offsetConst : offsetConst + 1; + } + break; + default: + // default to select * + startOffset = -1; + endOffset = -1; + } + + final Map newScan = new HashMap<>(); + + fullScan.forEach((tp, existingInputSplit) -> { + final KafkaPullerInputSplit newInputSplit; + if (startOffset != -1 && endOffset == -1) { + newInputSplit = new KafkaPullerInputSplit( + tp.topic(), + tp.partition(), + // @TODO make sure that this is okay + //if the user as 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(), + //@TODO check this, if user ask for non existing end offset ignore it and position head on start + 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(), + //@TODO check this with team if we have ask for offset out of range what to do ? + // here am seeking to last offset + existingInputSplit.getEndOffset(), + existingInputSplit.getEndOffset(), + existingInputSplit.getPath() + ); + } else { + newInputSplit = new KafkaPullerInputSplit( + tp.topic(), + tp.partition(), + startOffset, + endOffset, + existingInputSplit.getPath() + ); + } + + } else { + newInputSplit = new KafkaPullerInputSplit( + tp.topic(), + tp.partition(), + existingInputSplit.getStartOffset(), + existingInputSplit.getEndOffset(), + existingInputSplit.getPath() + ); + } + + newScan.put(tp, KafkaPullerInputSplit.intersectRange(newInputSplit, existingInputSplit)); + }); + + return newScan; + } + + @Nullable + protected static Map buildScanForTimesPredicate( + final Map fullHouse, + PredicateLeaf.Operator operator, long timestamp, boolean flip, boolean negation, KafkaConsumer consumer + ) + { + long increment = (flip && operator == PredicateLeaf.Operator.LESS_THAN + || negation && operator == PredicateLeaf.Operator.LESS_THAN_EQUALS) ? 1L : 0L; + // only accepted cases are timestamp_column [ > ; >= ; = ]constant + if (operator == PredicateLeaf.Operator.EQUALS || flip ^ negation) { + final Map timePartitionsMap = Maps.toMap(fullHouse.keySet(), tp -> timestamp + increment); + try { + // Based on Kafka docs + // 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); + final Map newScan = Maps.toMap(fullHouse.keySet(), tp -> { + KafkaPullerInputSplit 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( + tp.topic(), + tp.partition(), + startOffset, + existing.getEndOffset(), + existing.getPath() + ); + }); + return newScan; + } + catch (Exception e) { + log.error("Error while looking up offsets for time", e); + //Bailout when can not figure out offsets for times. + return null; + } + + } + return null; + } + + /** + * @param expr And expression to be parsed + * + * @return either full scan or an optimized sub scan. + */ + private Map pushAndOp(ExprNodeGenericFuncDesc expr) + { + Map currentScan = new HashMap<>(); + + fullHouse.forEach((tp, input) -> currentScan.put( + tp, + KafkaPullerInputSplit.copyOf(input) + )); + + for (ExprNodeDesc child : + expr.getChildren()) { + Map scan = parse(child); + if (scan != null) { + Set currentKeys = ImmutableSet.copyOf(currentScan.keySet()); + currentKeys.stream().forEach(key -> { + KafkaPullerInputSplit newSplit = scan.get(key); + KafkaPullerInputSplit oldSplit = currentScan.get(key); + currentScan.remove(key); + if (newSplit != null) { + KafkaPullerInputSplit intersectionSplit = KafkaPullerInputSplit.intersectRange(newSplit, oldSplit); + if (intersectionSplit != null) { + currentScan.put(key, intersectionSplit); + } + } + }); + + } + } + return currentScan; + } + + @Nullable + private Map pushOrOp(ExprNodeGenericFuncDesc expr) + { + final Map currentScan = new HashMap<>(); + for (ExprNodeDesc child : + expr.getChildren()) { + Map scan = parse(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)); + }); + } + return currentScan; + } + + private static ExprNodeDesc getColumnExpr(ExprNodeDesc expr) + { + if (expr instanceof ExprNodeColumnDesc) { + return expr; + } + ExprNodeGenericFuncDesc funcDesc = null; + if (expr instanceof ExprNodeGenericFuncDesc) { + funcDesc = (ExprNodeGenericFuncDesc) expr; + } + if (null == funcDesc) { + return expr; + } + GenericUDF udf = funcDesc.getGenericUDF(); + // check if its a simple cast expression. + if ((udf instanceof GenericUDFBridge || udf instanceof GenericUDFToBinary + || udf instanceof GenericUDFToChar || udf instanceof GenericUDFToVarchar + || udf instanceof GenericUDFToDecimal || udf instanceof GenericUDFToDate + || udf instanceof GenericUDFToUnixTimeStamp || udf instanceof GenericUDFToUtcTimestamp) + && funcDesc.getChildren().size() == 1 + && funcDesc.getChildren().get(0) instanceof ExprNodeColumnDesc) { + return expr.getChildren().get(0); + } + return expr; + } + +} diff --git a/hive-kafka-reader/src/java/org/apache/hadoop/hive/kafka/KafkaStorageHandler.java b/hive-kafka-reader/src/java/org/apache/hadoop/hive/kafka/KafkaStorageHandler.java new file mode 100644 index 000000000000..86b44b998bdb --- /dev/null +++ b/hive-kafka-reader/src/java/org/apache/hadoop/hive/kafka/KafkaStorageHandler.java @@ -0,0 +1,182 @@ +/* + * 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.conf.Configuration; +import org.apache.hadoop.hive.metastore.HiveMetaHook; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.metadata.HiveStorageHandler; +import org.apache.hadoop.hive.ql.plan.TableDesc; +import org.apache.hadoop.hive.ql.security.authorization.DefaultHiveAuthorizationProvider; +import org.apache.hadoop.hive.ql.security.authorization.HiveAuthorizationProvider; +import org.apache.hadoop.hive.serde2.AbstractSerDe; +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.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.HashMap; +import java.util.Map; + +import static org.apache.hadoop.hive.kafka.KafkaPullerInputFormat.CONSUMER_CONFIGURATION_PREFIX; +import static org.apache.hadoop.hive.kafka.KafkaPullerInputFormat.HIVE_KAFKA_BOOTSTRAP_SERVERS; +import static org.apache.hadoop.hive.kafka.KafkaPullerInputFormat.HIVE_KAFKA_TOPIC; + +public class KafkaStorageHandler implements HiveStorageHandler +{ + + public static final String __TIMESTAMP = "__timestamp"; + public static final String __PARTITION = "__partition"; + public static final String __OFFSET = "__offset"; + public static final String SERDE_CLASS_NAME = "kafka.serde.class"; + + private static final Logger log = LoggerFactory.getLogger(KafkaStorageHandler.class); + + Configuration configuration; + + @Override + public Class getInputFormatClass() + { + return KafkaPullerInputFormat.class; + } + + @Override + public Class getOutputFormatClass() + { + return NullOutputFormat.class; + } + + @Override + public Class getSerDeClass() + { + return GenericKafkaSerDe.class; + } + + @Override + public HiveMetaHook getMetaHook() + { + return null; + } + + @Override + public HiveAuthorizationProvider getAuthorizationProvider() throws HiveException + { + return new DefaultHiveAuthorizationProvider(); + } + + @Override + public void configureInputJobProperties( + TableDesc tableDesc, + Map jobProperties + ) + { + jobProperties.put(HIVE_KAFKA_TOPIC, Preconditions + .checkNotNull( + tableDesc.getProperties().getProperty(HIVE_KAFKA_TOPIC), + "kafka topic missing set table property->" + HIVE_KAFKA_TOPIC + )); + log.debug("Table properties: Kafka Topic {}", tableDesc.getProperties().getProperty(HIVE_KAFKA_TOPIC)); + jobProperties.put(HIVE_KAFKA_BOOTSTRAP_SERVERS, Preconditions + .checkNotNull( + tableDesc.getProperties().getProperty(HIVE_KAFKA_BOOTSTRAP_SERVERS), + "Broker address missing set table property->" + HIVE_KAFKA_BOOTSTRAP_SERVERS + )); + log.debug("Table properties: Kafka broker {}", tableDesc.getProperties().getProperty(HIVE_KAFKA_BOOTSTRAP_SERVERS)); + jobProperties.put( + SERDE_CLASS_NAME, + tableDesc.getProperties().getProperty(SERDE_CLASS_NAME, KafkaJsonSerDe.class.getName()) + ); + + log.info("Table properties: SerDe class name {}", jobProperties.get(SERDE_CLASS_NAME)); + + //set extra properties + tableDesc.getProperties() + .entrySet() + .stream() + .filter( + objectObjectEntry -> objectObjectEntry.getKey() + .toString() + .toLowerCase() + .startsWith(CONSUMER_CONFIGURATION_PREFIX)) + .forEach(entry -> { + String key = entry.getKey().toString().substring(CONSUMER_CONFIGURATION_PREFIX.length() + 1); + 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 + ) + { + + } + + @Override + public void configureOutputJobProperties( + TableDesc tableDesc, + Map jobProperties + ) + { + + } + + @Override + public void configureTableJobProperties( + TableDesc tableDesc, + Map jobProperties + ) + { + configureInputJobProperties(tableDesc, jobProperties); + } + + @Override + public void configureJobConf(TableDesc tableDesc, JobConf jobConf) + { + + Map properties = new HashMap<>(); + configureInputJobProperties(tableDesc, properties); + properties.forEach((key, value) -> jobConf.set(key, value)); + } + + @Override + public void setConf(Configuration configuration) + { + this.configuration = configuration; + } + + @Override + public Configuration getConf() + { + return configuration; + } + + @Override + public String toString() + { + return "org.apache.hadoop.hive.kafka.KafkaStorageHandler"; + } +} diff --git a/hive-kafka-reader/src/java/org/apache/hadoop/hive/kafka/KafkaStreamingUtils.java b/hive-kafka-reader/src/java/org/apache/hadoop/hive/kafka/KafkaStreamingUtils.java new file mode 100644 index 000000000000..958c4e42e9b2 --- /dev/null +++ b/hive-kafka-reader/src/java/org/apache/hadoop/hive/kafka/KafkaStreamingUtils.java @@ -0,0 +1,69 @@ +/* + * 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.CommonClientConfigs; +import org.apache.kafka.common.serialization.ByteArrayDeserializer; + +import java.util.Map; +import java.util.Properties; + +import static org.apache.kafka.clients.consumer.ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG; +import static org.apache.kafka.clients.consumer.ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG; + + +/** + * Utilities class + */ +public class KafkaStreamingUtils { + + private KafkaStreamingUtils() {} + + /** + * @param configuration Job configs + * + * @return default consumer properties + */ + public static Properties consumerProperties(Configuration configuration) { + final Properties props = new Properties(); + // those are very important to set to avoid long blocking + props.setProperty("request.timeout.ms", "10001" ); + props.setProperty("fetch.max.wait.ms", "10000" ); + props.setProperty("session.timeout.ms", "10000" ); + // we are managing the commit offset + props.setProperty("enable.auto.commit", "false"); + // we are seeking in the stream so no reset + props.setProperty("auto.offset.reset", "none"); + String brokerEndPoint = configuration.get(KafkaPullerInputFormat.HIVE_KAFKA_BOOTSTRAP_SERVERS); + props.setProperty(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, brokerEndPoint); + props.setProperty(KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName()); + props.setProperty(VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName()); + // user can always override stuff + final Map kafkaProperties = + configuration.getValByRegex("^" + KafkaPullerInputFormat.CONSUMER_CONFIGURATION_PREFIX + "\\..*"); + for (Map.Entry entry : kafkaProperties.entrySet()) { + props.setProperty(entry.getKey().substring( + KafkaPullerInputFormat.CONSUMER_CONFIGURATION_PREFIX.length() + 1), + entry.getValue() + ); + } + return props; + } +} diff --git a/hive-kafka-reader/src/test/org/apache/hadoop/hive/kafka/KafkaPullerInputSplitTest.java b/hive-kafka-reader/src/test/org/apache/hadoop/hive/kafka/KafkaPullerInputSplitTest.java new file mode 100644 index 000000000000..aa3aba7e3ed4 --- /dev/null +++ b/hive-kafka-reader/src/test/org/apache/hadoop/hive/kafka/KafkaPullerInputSplitTest.java @@ -0,0 +1,217 @@ +/* + * 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; + +public class KafkaPullerInputSplitTest +{ + private String topic = "my_topic"; + private KafkaPullerInputSplit expectedInputSplit; + + public KafkaPullerInputSplitTest() + { + this.expectedInputSplit = new KafkaPullerInputSplit(this.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 a/hive-kafka-reader/src/test/org/apache/hadoop/hive/kafka/KafkaRecordIteratorTest.java b/hive-kafka-reader/src/test/org/apache/hadoop/hive/kafka/KafkaRecordIteratorTest.java new file mode 100644 index 000000000000..73629aa5577f --- /dev/null +++ b/hive-kafka-reader/src/test/org/apache/hadoop/hive/kafka/KafkaRecordIteratorTest.java @@ -0,0 +1,376 @@ +// +// Source code recreated from a .class file by IntelliJ IDEA +// (powered by Fernflower decompiler) +// + +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.io.NullWritable; +import org.apache.hadoop.mapred.InputSplit; +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.ConsumerRecord; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.record.TimestampType; +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.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.nio.charset.Charset; +import java.nio.file.Files; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.Properties; +import java.util.stream.Collectors; + +public class KafkaRecordIteratorTest +{ + private static final Logger log = LoggerFactory.getLogger(KafkaRecordIteratorTest.class); + private static final String topic = "my_topic2"; + private static final List> RECORDS = new ArrayList(); + private static final int RECORD_NUMBER = 100; + private static final TopicPartition TOPIC_PARTITION = new TopicPartition("my_topic2", 0); + public static final byte[] KEY_BYTES = "KEY".getBytes(Charset.forName("UTF-8")); + private static ZkUtils zkUtils; + private static ZkClient zkClient; + private static KafkaProducer producer; + private static KafkaServer kafkaServer; + private static String zkConnect; + private KafkaConsumer consumer = null; + private KafkaRecordIterator kafkaRecordIterator = null; + private Configuration conf = new Configuration(); + + + public KafkaRecordIteratorTest() + { + } + + @BeforeClass + public static void setupCluster() throws IOException, InterruptedException + { + log.info("init embedded Zookeeper"); + EmbeddedZookeeper zkServer = new EmbeddedZookeeper(); + zkConnect = "127.0.0.1:" + zkServer.port(); + zkClient = new ZkClient(zkConnect, 30000, 30000, 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.dirs", Files.createTempDirectory("kafka-").toAbsolutePath().toString()); + brokerProps.setProperty("listeners", "PLAINTEXT://127.0.0.1:9092"); + brokerProps.setProperty("offsets.topic.replication.factor", "1"); + brokerProps.setProperty("log.retention.ms", "1000000"); + KafkaConfig config = new KafkaConfig(brokerProps); + Time mock = new MockTime(); + kafkaServer = TestUtils.createServer(config, mock); + log.info("Creating kafka topic [{}]", "my_topic2"); + AdminUtils.createTopic(zkUtils, topic, 1, 1, new Properties(), RackAwareMode.Disabled$.MODULE$); + setupProducer(); + sendData(); + } + + @Before + public void setUp() + { + log.info("setting up consumer"); + this.setupConsumer(); + this.kafkaRecordIterator = null; + } + + @Test + public void testHasNextAbsoluteStartEnd() + { + this.kafkaRecordIterator = new KafkaRecordIterator(this.consumer, TOPIC_PARTITION, 0L, (long) RECORDS.size(), 100L); + this.compareIterator(RECORDS, this.kafkaRecordIterator); + } + + @Test + public void testHasNextGivenStartEnd() + { + long startOffset = 2L; + long lastOffset = 4L; + this.kafkaRecordIterator = new KafkaRecordIterator(this.consumer, TOPIC_PARTITION, 2L, 4L, 100L); + this.compareIterator((List) RECORDS.stream().filter((consumerRecord) -> { + return consumerRecord.offset() >= 2L && consumerRecord.offset() < 4L; + }).collect(Collectors.toList()), this.kafkaRecordIterator); + } + + @Test + public void testHasNextNoOffsets() + { + this.kafkaRecordIterator = new KafkaRecordIterator(this.consumer, TOPIC_PARTITION, 100L); + this.compareIterator(RECORDS, this.kafkaRecordIterator); + } + + @Test + public void testHasNextLastRecord() + { + long startOffset = (long) (RECORDS.size() - 1); + long lastOffset = (long) RECORDS.size(); + this.kafkaRecordIterator = new KafkaRecordIterator(this.consumer, TOPIC_PARTITION, startOffset, lastOffset, 100L); + this.compareIterator((List) RECORDS.stream().filter((consumerRecord) -> { + return consumerRecord.offset() >= startOffset && consumerRecord.offset() < lastOffset; + }).collect(Collectors.toList()), this.kafkaRecordIterator); + } + + @Test + public void testHasNextFirstRecord() + { + long startOffset = 0L; + long lastOffset = 1L; + this.kafkaRecordIterator = new KafkaRecordIterator(this.consumer, TOPIC_PARTITION, 0L, 1L, 100L); + this.compareIterator((List) RECORDS.stream().filter((consumerRecord) -> { + return consumerRecord.offset() >= 0L && consumerRecord.offset() < 1L; + }).collect(Collectors.toList()), this.kafkaRecordIterator); + } + + @Test + public void testHasNextNoStart() + { + long startOffset = 0L; + long lastOffset = 10L; + this.kafkaRecordIterator = new KafkaRecordIterator(this.consumer, TOPIC_PARTITION, (Long) null, 10L, 100L); + this.compareIterator((List) RECORDS.stream().filter((consumerRecord) -> { + return consumerRecord.offset() >= 0L && consumerRecord.offset() < 10L; + }).collect(Collectors.toList()), this.kafkaRecordIterator); + } + + @Test + public void testHasNextNoEnd() + { + long startOffset = 5L; + long lastOffset = (long) RECORDS.size(); + this.kafkaRecordIterator = new KafkaRecordIterator(this.consumer, TOPIC_PARTITION, 5L, (Long) null, 100L); + this.compareIterator((List) RECORDS.stream().filter((consumerRecord) -> { + return consumerRecord.offset() >= 5L && consumerRecord.offset() < lastOffset; + }).collect(Collectors.toList()), this.kafkaRecordIterator); + } + + @Test + public void testRecordReader() throws IOException, InterruptedException + { + InputSplit inputSplits = new KafkaPullerInputSplit("my_topic2", 0, 0L, 50L, null); + KafkaPullerRecordReader recordReader = new KafkaPullerRecordReader((KafkaPullerInputSplit) inputSplits, this.conf); + List serRecords = (List) RECORDS.stream().map((recordx) -> { + return KafkaRecordWritable.fromKafkaRecord(recordx); + }).collect(Collectors.toList()); + + for (int i = 0; i < 50; ++i) { + KafkaRecordWritable record = new KafkaRecordWritable(); + Assert.assertTrue(recordReader.next((NullWritable) null, record)); + Assert.assertEquals(serRecords.get(i), record); + } + + recordReader.close(); + recordReader = new KafkaPullerRecordReader(); + TaskAttemptContext context = new TaskAttemptContextImpl(this.conf, new TaskAttemptID()); + recordReader.initialize(new KafkaPullerInputSplit("my_topic2", 0, 50L, 100L, null), context); + + for (int i = 50; i < 100; ++i) { + KafkaRecordWritable record = new KafkaRecordWritable(); + Assert.assertTrue(recordReader.next((NullWritable) null, record)); + Assert.assertEquals(serRecords.get(i), record); + } + + recordReader.close(); + } + + @Test( + expected = IllegalStateException.class + ) + public void testPullingBeyondLimit() + { + this.kafkaRecordIterator = new KafkaRecordIterator( + this.consumer, + TOPIC_PARTITION, + 0L, + (long) RECORDS.size() + 1L, + 100L + ); + this.compareIterator(RECORDS, this.kafkaRecordIterator); + } + + @Test( + expected = IllegalStateException.class + ) + public void testPullingStartGreaterThanEnd() + { + this.kafkaRecordIterator = new KafkaRecordIterator(this.consumer, TOPIC_PARTITION, 10L, 1L, 100L); + this.compareIterator(RECORDS, this.kafkaRecordIterator); + } + + @Test( + expected = IllegalStateException.class + ) + public void testPullingFromEmptyTopic() + { + this.kafkaRecordIterator = new KafkaRecordIterator(this.consumer, new TopicPartition("noHere", 0), 0L, 100L, 100L); + this.compareIterator(RECORDS, this.kafkaRecordIterator); + } + + @Test( + expected = IllegalStateException.class + ) + public void testPullingFromEmptyPartition() + { + this.kafkaRecordIterator = new KafkaRecordIterator( + this.consumer, + new TopicPartition("my_topic2", 1), + 0L, + 100L, + 100L + ); + this.compareIterator(RECORDS, this.kafkaRecordIterator); + } + + @Test + public void testStartIsEqualEnd() + { + this.kafkaRecordIterator = new KafkaRecordIterator(this.consumer, TOPIC_PARTITION, 10L, 10L, 100L); + this.compareIterator(ImmutableList.of(), this.kafkaRecordIterator); + } + + + @Test + public void testStartIsTheLastOffset() + { + this.kafkaRecordIterator = new KafkaRecordIterator( + this.consumer, + TOPIC_PARTITION, + new Long(RECORD_NUMBER), + new Long(RECORD_NUMBER), + 100L + ); + this.compareIterator(ImmutableList.of(), this.kafkaRecordIterator); + } + + private void compareIterator( + List> expected, + Iterator> kafkaRecordIterator + ) + { + expected.stream().forEachOrdered((expectedRecord) -> { + Assert.assertTrue("record with offset " + expectedRecord.offset(), kafkaRecordIterator.hasNext()); + ConsumerRecord record = kafkaRecordIterator.next(); + Assert.assertTrue(record.topic().equals(topic)); + Assert.assertTrue(record.partition() == 0); + 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.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"); + this.conf.set("kafka.bootstrap.servers", "127.0.0.1:9092"); + 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); + } + + private static void sendData() throws InterruptedException + { + log.info("Sending {} records", RECORD_NUMBER); + RECORDS.clear(); + for (int i = 0; i < RECORD_NUMBER; ++i) { + + final byte[] value = ("VALUE-" + Integer.toString(i)).getBytes(Charset.forName("UTF-8")); + //noinspection unchecked + producer.send(new ProducerRecord( + topic, + 0, + 0L, + KEY_BYTES, + value + )); + + //noinspection unchecked + RECORDS.add(new ConsumerRecord( + topic, + 0, + (long) i, + 0L, + (TimestampType) null, + 0L, + 0, + 0, + KEY_BYTES, + value + )); + } + + producer.close(); + } + + @After + public void tearDown() + { + this.kafkaRecordIterator = null; + if (this.consumer != null) { + this.consumer.close(); + } + } + + @AfterClass + public static void tearDownCluster() + { + if (kafkaServer != null) { + kafkaServer.shutdown(); + kafkaServer.awaitShutdown(); + } + + zkClient.close(); + zkUtils.close(); + } +} diff --git a/hive-kafka-reader/src/test/org/apache/hadoop/hive/kafka/KafkaRecordWritableTest.java b/hive-kafka-reader/src/test/org/apache/hadoop/hive/kafka/KafkaRecordWritableTest.java new file mode 100644 index 000000000000..68113514396c --- /dev/null +++ b/hive-kafka-reader/src/test/org/apache/hadoop/hive/kafka/KafkaRecordWritableTest.java @@ -0,0 +1,37 @@ +// +// Source code recreated from a .class file by IntelliJ IDEA +// (powered by Fernflower decompiler) +// + +package org.apache.hadoop.hive.kafka; + +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.junit.Assert; +import org.junit.Test; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; + +public class KafkaRecordWritableTest { + public KafkaRecordWritableTest() { + } + + @Test + public void testWriteReadFields() throws IOException { + ConsumerRecord record = new ConsumerRecord("topic", 0, 3L, "key".getBytes(), "value".getBytes()); + KafkaRecordWritable kafkaRecordWritable = KafkaRecordWritable.fromKafkaRecord(record); + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + DataOutputStream w = new DataOutputStream(baos); + kafkaRecordWritable.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); + } +} diff --git a/hive-kafka-reader/src/test/org/apache/hadoop/hive/kafka/KafkaScanTrimmerTest.java b/hive-kafka-reader/src/test/org/apache/hadoop/hive/kafka/KafkaScanTrimmerTest.java new file mode 100644 index 000000000000..232bfcbb96b1 --- /dev/null +++ b/hive-kafka-reader/src/test/org/apache/hadoop/hive/kafka/KafkaScanTrimmerTest.java @@ -0,0 +1,730 @@ +/* + * 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 com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.ql.exec.SerializationUtilities; +import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf; +import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc; +import org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc; +import org.apache.hadoop.hive.ql.plan.ExprNodeDesc; +import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPAnd; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqual; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqualOrGreaterThan; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqualOrLessThan; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPGreaterThan; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPLessThan; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPOr; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; +import org.apache.kafka.common.TopicPartition; +import org.junit.Assert; +import org.junit.Test; + +import java.util.List; +import java.util.Map; +import java.util.Objects; + +import static org.junit.Assert.assertNotNull; + +public class KafkaScanTrimmerTest +{ + private static final Path PATH = new Path("/tmp"); + + private ExprNodeDesc zeroInt = ConstantExprBuilder.build(0); + private ExprNodeDesc threeInt = ConstantExprBuilder.build(3); + private ExprNodeDesc thirtyLong = ConstantExprBuilder.build(30L); + private ExprNodeDesc thirtyFiveLong = ConstantExprBuilder.build(35L); + private ExprNodeDesc seventyFiveLong = ConstantExprBuilder.build(75L); + private ExprNodeDesc fortyLong = ConstantExprBuilder.build(40L); + + private ExprNodeDesc partitionColumn = new ExprNodeColumnDesc( + TypeInfoFactory.intTypeInfo, + KafkaStorageHandler.__PARTITION, + null, + false + ); + private ExprNodeDesc offsetColumn = new ExprNodeColumnDesc( + TypeInfoFactory.longTypeInfo, + KafkaStorageHandler.__OFFSET, + null, + false + ); + /*private ExprNodeDesc timestampColumn = new ExprNodeColumnDesc( + TypeInfoFactory.longTypeInfo, + KafkaJsonSerDe.__TIMESTAMP, + null, + false + );*/ + + private String topic = "my_topic"; + private Map fullHouse = ImmutableMap.of( + new TopicPartition(topic, 0), + new KafkaPullerInputSplit( + topic, + 0, + 0, + 45, + PATH + ), + new TopicPartition(topic, 1), + new KafkaPullerInputSplit( + topic, + 1, + 5, + 1005, + PATH + ), + new TopicPartition(topic, 2), + new KafkaPullerInputSplit( + topic, + 2, + 9, + 100, + PATH + ), + new TopicPartition(topic, 3), + new KafkaPullerInputSplit( + topic, + 3, + 0, + 100, + PATH + ) + ); + + @Test + public void computeOptimizedScanPartitionBinaryOpFilter() + { + KafkaScanTrimmer kafkaScanTrimmer = new KafkaScanTrimmer(fullHouse, null); + int partitionId = 2; + ExprNodeDesc constant = ConstantExprBuilder.build(partitionId); + final List children = Lists.newArrayList(partitionColumn, constant); + + ExprNodeGenericFuncDesc node = EQ(children); + assertNotNull(node); + + Map actual = kafkaScanTrimmer.computeOptimizedScan(SerializationUtilities.deserializeExpression( + SerializationUtilities.serializeExpression(node))); + Map expected = Maps.filterValues(fullHouse, tp -> Objects.requireNonNull(tp).getPartition() == partitionId); + Assert.assertEquals(expected, actual); + + ExprNodeGenericFuncDesc lessNode = LESS_THAN(children); + assertNotNull(lessNode); + actual = kafkaScanTrimmer.computeOptimizedScan(SerializationUtilities.deserializeExpression(SerializationUtilities.serializeExpression( + lessNode))); + expected = Maps.filterValues(fullHouse, tp -> Objects.requireNonNull(tp).getPartition() < partitionId); + Assert.assertEquals(expected, actual); + + + ExprNodeGenericFuncDesc lessEqNode = LESS_THAN_EQ(children); + + assertNotNull(lessEqNode); + actual = kafkaScanTrimmer.computeOptimizedScan(SerializationUtilities.deserializeExpression(SerializationUtilities.serializeExpression( + lessEqNode))); + expected = Maps.filterValues(fullHouse, tp -> Objects.requireNonNull(tp).getPartition() <= partitionId); + Assert.assertEquals(expected, actual); + + } + + + @Test + public void computeOptimizedScanFalseFilter() + { + KafkaScanTrimmer kafkaScanTrimmer = new KafkaScanTrimmer(fullHouse, null); + ExprNodeGenericFuncDesc falseFilter = AND(Lists.newArrayList( + EQ(Lists.newArrayList(partitionColumn, zeroInt)), + EQ(Lists.newArrayList(partitionColumn, threeInt)) + )); + + assertNotNull(falseFilter); + Map actual = kafkaScanTrimmer.computeOptimizedScan(SerializationUtilities.deserializeExpression( + SerializationUtilities.serializeExpression(falseFilter))); + Assert.assertTrue(actual.isEmpty()); + + ExprNodeGenericFuncDesc falseFilter2 = AND(Lists.newArrayList( + EQ(Lists.newArrayList(offsetColumn, thirtyFiveLong)), + EQ(Lists.newArrayList(offsetColumn, fortyLong)) + )); + + assertNotNull(falseFilter2); + actual = kafkaScanTrimmer.computeOptimizedScan(SerializationUtilities.deserializeExpression( + SerializationUtilities.serializeExpression(falseFilter2))); + Assert.assertTrue(actual.isEmpty()); + + ExprNodeGenericFuncDesc filter3 = OR(Lists.newArrayList(falseFilter, falseFilter2)); + + assertNotNull(filter3); + actual = kafkaScanTrimmer.computeOptimizedScan(SerializationUtilities.deserializeExpression( + SerializationUtilities.serializeExpression(filter3))); + Assert.assertTrue(actual.isEmpty()); + + ExprNodeGenericFuncDesc filter4 = AND(Lists.newArrayList( + filter3, + EQ(Lists.newArrayList(partitionColumn, zeroInt)) + )); + assertNotNull(filter4); + actual = kafkaScanTrimmer.computeOptimizedScan(SerializationUtilities.deserializeExpression( + SerializationUtilities.serializeExpression(filter4))); + Assert.assertTrue(actual.isEmpty()); + } + + @Test + public void computeOptimizedScanOrAndCombinedFilter() + { + KafkaScanTrimmer kafkaScanTrimmer = new KafkaScanTrimmer(fullHouse, null); + // partition = 0 and 30 <= offset < 35 or partition = 3 and 35 <= offset < 75 or (partition = 0 and offset = 40) + + + ExprNodeGenericFuncDesc part1 = AND(Lists.newArrayList( + GREATER_THAN_EQ(Lists.newArrayList(offsetColumn, thirtyLong)), + EQ(Lists.newArrayList(partitionColumn, zeroInt)), + LESS_THAN(Lists.newArrayList(offsetColumn, thirtyFiveLong)) + )); + + ExprNodeGenericFuncDesc part2 = AND(Lists.newArrayList( + GREATER_THAN_EQ(Lists.newArrayList(offsetColumn, thirtyFiveLong)), + EQ(Lists.newArrayList(partitionColumn, threeInt)), + LESS_THAN(Lists.newArrayList(offsetColumn, seventyFiveLong)) + )); + + ExprNodeGenericFuncDesc part3 = AND(Lists.newArrayList( + EQ(Lists.newArrayList(offsetColumn, fortyLong)), + EQ(Lists.newArrayList(partitionColumn, zeroInt)) + )); + + ExprNodeGenericFuncDesc orExpression = OR(Lists.newArrayList(part1, part2, part3)); + + assertNotNull(orExpression); + Map actual = kafkaScanTrimmer.computeOptimizedScan(SerializationUtilities.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); + + Map expected = ImmutableMap.of(tpZero, split1, toThree, split2); + Assert.assertEquals(expected, actual); + + + } + + @Test + public void computeOptimizedScanPartitionOrAndCombinedFilter() + { + KafkaScanTrimmer kafkaScanTrimmer = new KafkaScanTrimmer(fullHouse, null); + + // partition = 1 or (partition >2 and <= 3) + ExprNodeGenericFuncDesc eq = EQ(Lists.newArrayList(partitionColumn, ConstantExprBuilder.build(1))); + ExprNodeGenericFuncDesc lessEq = LESS_THAN_EQ(Lists.newArrayList(partitionColumn, ConstantExprBuilder.build(3))); + ExprNodeGenericFuncDesc greater = GREATER_THAN(Lists.newArrayList(partitionColumn, ConstantExprBuilder.build(2))); + ExprNodeGenericFuncDesc orNode = OR(Lists.newArrayList(AND(Lists.newArrayList(lessEq, greater)), eq)); + + Map actual = kafkaScanTrimmer.computeOptimizedScan(SerializationUtilities.deserializeExpression( + SerializationUtilities.serializeExpression(orNode))); + Map expected = Maps.filterValues( + fullHouse, + tp -> Objects.requireNonNull(tp).getPartition() == 1 || tp.getPartition() == 3 + ); + Assert.assertEquals(expected, actual); + assertNotNull(orNode); + } + + + @Test + public void buildScanFormPartitionPredicateEq() + { + Map actual = KafkaScanTrimmer.buildScanFormPartitionPredicate( + fullHouse, + PredicateLeaf.Operator.EQUALS, + 3, + false, + false + ); + TopicPartition topicPartition = new TopicPartition(topic, 3); + Assert.assertEquals(fullHouse.get(topicPartition), actual.get(topicPartition)); + } + + @Test + public void buildScanFormPartitionPredicateLess() + { + // partitionConst < partitionColumn (flip true) + int partitionConst = 2; + Map actual = KafkaScanTrimmer.buildScanFormPartitionPredicate( + fullHouse, + PredicateLeaf.Operator.LESS_THAN, + partitionConst, + true, + false + ); + + Map expected = Maps.filterEntries( + fullHouse, + entry -> Objects.requireNonNull(entry).getKey().partition() > partitionConst + ); + Assert.assertEquals(expected, actual); + Assert.assertFalse(actual.isEmpty()); + + // partitionConst >= partitionColumn (flip true, negation true) + actual = KafkaScanTrimmer.buildScanFormPartitionPredicate( + fullHouse, + PredicateLeaf.Operator.LESS_THAN, + partitionConst, + true, + true + ); + + expected = Maps.filterEntries( + fullHouse, + entry -> partitionConst >= Objects.requireNonNull(entry).getKey().partition() + ); + Assert.assertEquals(expected, actual); + Assert.assertFalse(actual.isEmpty()); + + // partitionColumn >= partitionConst (negation true) + actual = KafkaScanTrimmer.buildScanFormPartitionPredicate( + fullHouse, + PredicateLeaf.Operator.LESS_THAN, + partitionConst, + false, + true + ); + + expected = Maps.filterEntries( + fullHouse, + entry -> Objects.requireNonNull(entry).getKey().partition() >= partitionConst + ); + Assert.assertEquals(expected, actual); + Assert.assertFalse(actual.isEmpty()); + + // partitionColumn < partitionConst (negation true) + actual = KafkaScanTrimmer.buildScanFormPartitionPredicate( + fullHouse, + PredicateLeaf.Operator.LESS_THAN, + partitionConst, + false, + false + ); + + expected = Maps.filterEntries( + fullHouse, + entry -> Objects.requireNonNull(entry).getKey().partition() < partitionConst + ); + Assert.assertEquals(expected, actual); + Assert.assertFalse(actual.isEmpty()); + } + + @Test + public void buildScanFormPartitionPredicateLessEq() + { + // partitionConst <= partitionColumn (flip true) + int partitionConst = 2; + Map actual = KafkaScanTrimmer.buildScanFormPartitionPredicate( + fullHouse, + PredicateLeaf.Operator.LESS_THAN_EQUALS, + partitionConst, + true, + false + ); + + Map expected = Maps.filterEntries( + fullHouse, + entry -> Objects.requireNonNull(entry).getKey().partition() >= partitionConst + ); + Assert.assertEquals(expected, actual); + Assert.assertFalse(actual.isEmpty()); + + // partitionConst > partitionColumn (flip true, negation true) + actual = KafkaScanTrimmer.buildScanFormPartitionPredicate( + fullHouse, + PredicateLeaf.Operator.LESS_THAN_EQUALS, + partitionConst, + true, + true + ); + + expected = Maps.filterEntries( + fullHouse, + entry -> partitionConst > Objects.requireNonNull(entry).getKey().partition() + ); + Assert.assertEquals(expected, actual); + Assert.assertFalse(actual.isEmpty()); + + + // partitionColumn > partitionConst (negation true) + actual = KafkaScanTrimmer.buildScanFormPartitionPredicate( + fullHouse, + PredicateLeaf.Operator.LESS_THAN_EQUALS, + partitionConst, + false, + true + ); + + expected = Maps.filterEntries(fullHouse, entry -> Objects.requireNonNull(entry).getKey().partition() > partitionConst); + Assert.assertEquals(expected, actual); + Assert.assertFalse(actual.isEmpty()); + + // partitionColumn <= partitionConst (negation true) + actual = KafkaScanTrimmer.buildScanFormPartitionPredicate( + fullHouse, + PredicateLeaf.Operator.LESS_THAN_EQUALS, + partitionConst, + false, + false + ); + + expected = Maps.filterEntries( + fullHouse, + entry -> Objects.requireNonNull(entry).getKey().partition() <= partitionConst + ); + Assert.assertEquals(expected, actual); + Assert.assertFalse(actual.isEmpty()); + } + + + @Test + public void buildScanFromOffsetPredicateEq() + { + long constantOffset = 30; + Map actual = KafkaScanTrimmer.buildScanFromOffsetPredicate( + fullHouse, + PredicateLeaf.Operator.EQUALS, + constantOffset, + false, + false + ); + Map expected = Maps.transformValues( + fullHouse, + entry -> new KafkaPullerInputSplit( + Objects.requireNonNull(entry).getTopic(), + entry.getPartition(), + constantOffset, + constantOffset + 1, + entry.getPath() + ) + ); + + Assert.assertEquals(expected, actual); + + // seek to end if offset is out of reach + actual = KafkaScanTrimmer.buildScanFromOffsetPredicate( + fullHouse, + PredicateLeaf.Operator.EQUALS, + 3000000L, + false, + false + ); + expected = Maps.transformValues( + fullHouse, + entry -> new KafkaPullerInputSplit( + Objects.requireNonNull(entry).getTopic(), + entry.getPartition(), + entry.getEndOffset(), + entry.getEndOffset(), + entry.getPath() + ) + ); + Assert.assertEquals(expected, actual); + + // seek to end if offset is out of reach + actual = KafkaScanTrimmer.buildScanFromOffsetPredicate( + fullHouse, + PredicateLeaf.Operator.EQUALS, + 0L, + false, + false + ); + + expected = Maps.transformValues( + fullHouse, + entry -> new KafkaPullerInputSplit( + Objects.requireNonNull(entry).getTopic(), + entry.getPartition(), + entry.getStartOffset() > 0 ? entry.getEndOffset() : 0, + entry.getStartOffset() > 0 ? entry.getEndOffset() : 1, + entry.getPath() + ) + ); + Assert.assertEquals(expected, actual); + + + } + + @Test + public void buildScanFromOffsetPredicateLess() + { + long constantOffset = 50; + // columnOffset < constant + Map actual = KafkaScanTrimmer.buildScanFromOffsetPredicate( + fullHouse, + PredicateLeaf.Operator.LESS_THAN, + constantOffset, + false, + false + ); + + Map expected = Maps.transformValues( + fullHouse, + entry -> new KafkaPullerInputSplit( + Objects.requireNonNull(entry).getTopic(), + entry.getPartition(), + entry.getStartOffset(), + Math.min(constantOffset, entry.getEndOffset()), + entry.getPath() + ) + ); + Assert.assertEquals(expected, actual); + + + // columnOffset > constant + actual = KafkaScanTrimmer.buildScanFromOffsetPredicate( + fullHouse, + PredicateLeaf.Operator.LESS_THAN, + constantOffset, + true, + false + ); + + expected = Maps.transformValues( + fullHouse, + entry -> new KafkaPullerInputSplit( + Objects.requireNonNull(entry).getTopic(), + entry.getPartition(), + Math.min(entry.getEndOffset(), Math.max(entry.getStartOffset(), constantOffset + 1)), + entry.getEndOffset(), + entry.getPath() + ) + ); + Assert.assertEquals(expected, actual); + + // columnOffset >= constant + actual = KafkaScanTrimmer.buildScanFromOffsetPredicate( + fullHouse, + PredicateLeaf.Operator.LESS_THAN, + constantOffset, + false, + true + ); + + expected = Maps.transformValues( + fullHouse, + entry -> new KafkaPullerInputSplit( + Objects.requireNonNull(entry).getTopic(), + entry.getPartition(), + Math.min(entry.getEndOffset(), Math.max(entry.getStartOffset(), constantOffset)), + entry.getEndOffset(), + entry.getPath() + ) + ); + Assert.assertEquals(expected, actual); + + +// columnOffset <= constant + actual = KafkaScanTrimmer.buildScanFromOffsetPredicate( + fullHouse, + PredicateLeaf.Operator.LESS_THAN, + constantOffset, + true, + true + ); + + expected = Maps.transformValues( + fullHouse, + entry -> new KafkaPullerInputSplit( + Objects.requireNonNull(entry).getTopic(), + entry.getPartition(), + entry.getStartOffset(), + Math.min(constantOffset + 1, entry.getEndOffset()), + entry.getPath() + ) + ); + Assert.assertEquals(expected, actual); + + } + + @Test + public void buildScanFromOffsetPredicateLessEq() + { + long constantOffset = 50; + // columnOffset < constant + Map actual = KafkaScanTrimmer.buildScanFromOffsetPredicate( + fullHouse, + PredicateLeaf.Operator.LESS_THAN_EQUALS, + constantOffset, + false, + false + ); + + Map expected = Maps.transformValues( + fullHouse, + entry -> new KafkaPullerInputSplit( + Objects.requireNonNull(entry).getTopic(), + entry.getPartition(), + entry.getStartOffset(), + Math.min(constantOffset + 1, entry.getEndOffset()), + entry.getPath() + ) + ); + Assert.assertEquals(expected, actual); + + // columnOffset >= constant + actual = KafkaScanTrimmer.buildScanFromOffsetPredicate( + fullHouse, + PredicateLeaf.Operator.LESS_THAN_EQUALS, + constantOffset, + true, + false + ); + + expected = Maps.transformValues( + fullHouse, + entry -> new KafkaPullerInputSplit( + Objects.requireNonNull(entry).getTopic(), + entry.getPartition(), + Math.min(entry.getEndOffset(), Math.max(entry.getStartOffset(), constantOffset)), + entry.getEndOffset(), + entry.getPath() + ) + ); + Assert.assertEquals(expected, actual); + + // columnOffset > constant + actual = KafkaScanTrimmer.buildScanFromOffsetPredicate( + fullHouse, + PredicateLeaf.Operator.LESS_THAN_EQUALS, + constantOffset, + false, + true + ); + + expected = Maps.transformValues( + fullHouse, + entry -> new KafkaPullerInputSplit( + Objects.requireNonNull(entry).getTopic(), + entry.getPartition(), + Math.min(entry.getEndOffset(), Math.max(entry.getStartOffset(), constantOffset + 1)), + entry.getEndOffset(), + entry.getPath() + ) + ); + Assert.assertEquals(expected, actual); + + // columnOffset < constant + actual = KafkaScanTrimmer.buildScanFromOffsetPredicate( + fullHouse, + PredicateLeaf.Operator.LESS_THAN_EQUALS, + constantOffset, + true, + true + ); + + expected = Maps.transformValues( + fullHouse, + entry -> new KafkaPullerInputSplit( + Objects.requireNonNull(entry).getTopic(), + entry.getPartition(), + entry.getStartOffset(), + Math.min(constantOffset, entry.getEndOffset()), + entry.getPath() + ) + ); + Assert.assertEquals(expected, actual); + } + + private static class ConstantExprBuilder + { + static ExprNodeDesc build(long constant) + { + return new ExprNodeConstantDesc(TypeInfoFactory.longTypeInfo, constant); + } + + static ExprNodeDesc build(int constant) + { + return new ExprNodeConstantDesc(TypeInfoFactory.longTypeInfo, constant); + } + } + + + private static ExprNodeGenericFuncDesc OR(List children) + { + return new ExprNodeGenericFuncDesc( + TypeInfoFactory.booleanTypeInfo, + new GenericUDFOPOr(), + children + ); + } + + private static ExprNodeGenericFuncDesc AND(List children) + { + return new ExprNodeGenericFuncDesc( + TypeInfoFactory.booleanTypeInfo, + new GenericUDFOPAnd(), + children + ); + } + + private static ExprNodeGenericFuncDesc EQ(List children) + { + return new ExprNodeGenericFuncDesc( + children.get(0).getTypeInfo(), + new GenericUDFOPEqual(), + children + ); + } + + private static ExprNodeGenericFuncDesc LESS_THAN(List children) + { + return new ExprNodeGenericFuncDesc( + children.get(0).getTypeInfo(), + new GenericUDFOPLessThan(), + children + ); + } + + private static ExprNodeGenericFuncDesc LESS_THAN_EQ(List children) + { + return new ExprNodeGenericFuncDesc( + children.get(0).getTypeInfo(), + new GenericUDFOPEqualOrLessThan(), + children + ); + } + + private static ExprNodeGenericFuncDesc GREATER_THAN(List children) + { + return new ExprNodeGenericFuncDesc( + children.get(0).getTypeInfo(), + new GenericUDFOPGreaterThan(), + children + ); + } + + private static ExprNodeGenericFuncDesc GREATER_THAN_EQ(List children) + { + return new ExprNodeGenericFuncDesc( + children.get(0).getTypeInfo(), + new GenericUDFOPEqualOrGreaterThan(), + children + ); + } +} \ No newline at end of file diff --git a/hive-kafka-reader/src/test/org/apache/hadoop/hive/kafka/KafkaStreamingUtilsTest.java b/hive-kafka-reader/src/test/org/apache/hadoop/hive/kafka/KafkaStreamingUtilsTest.java new file mode 100644 index 000000000000..add3f7137ee2 --- /dev/null +++ b/hive-kafka-reader/src/test/org/apache/hadoop/hive/kafka/KafkaStreamingUtilsTest.java @@ -0,0 +1,29 @@ +// +// Source code recreated from a .class file by IntelliJ IDEA +// (powered by Fernflower decompiler) +// + +package org.apache.hadoop.hive.kafka; + +import org.apache.hadoop.conf.Configuration; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Properties; + +public class KafkaStreamingUtilsTest { + public KafkaStreamingUtilsTest() { + } + + @Test + public void testConsumerProperties() { + Configuration configuration = new Configuration(); + 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 = KafkaStreamingUtils.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")); + } +} diff --git a/itests/qtest-druid/pom.xml b/itests/qtest-druid/pom.xml index 79a0fb3f2f08..e566fcf4d706 100644 --- a/itests/qtest-druid/pom.xml +++ b/itests/qtest-druid/pom.xml @@ -43,7 +43,7 @@ 10.11.1.1 16.0.1 4.1.0 - 0.10.2.0 + 1.0.1 diff --git a/itests/qtest/pom.xml b/itests/qtest/pom.xml index 576780601748..a0cfcaf8c1f1 100644 --- a/itests/qtest/pom.xml +++ b/itests/qtest/pom.xml @@ -138,7 +138,12 @@ ${project.version} test - + + org.apache.hive + hive-kafka-reader + ${project.version} + test + diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/cli/LlapServiceDriver.java b/llap-server/src/java/org/apache/hadoop/hive/llap/cli/LlapServiceDriver.java index 0f1d5eea4cc8..ae1de26adc47 100644 --- a/llap-server/src/java/org/apache/hadoop/hive/llap/cli/LlapServiceDriver.java +++ b/llap-server/src/java/org/apache/hadoop/hive/llap/cli/LlapServiceDriver.java @@ -91,7 +91,7 @@ public class LlapServiceDriver { private static final String[] DEFAULT_AUX_CLASSES = new String[] { "org.apache.hive.hcatalog.data.JsonSerDe","org.apache.hadoop.hive.druid.DruidStorageHandler", "org.apache.hive.storage.jdbc.JdbcStorageHandler", "org.apache.commons.dbcp.BasicDataSourceFactory", - "org.apache.commons.pool.impl.GenericObjectPool" + "org.apache.commons.pool.impl.GenericObjectPool", "org.apache.hadoop.hive.kafka.KafkaStorageHandler" }; private static final String HBASE_SERDE_CLASS = "org.apache.hadoop.hive.hbase.HBaseSerDe"; private static final String[] NEEDED_CONFIGS = LlapDaemonConfiguration.DAEMON_CONFIGS; diff --git a/packaging/pom.xml b/packaging/pom.xml index 5c859acfadb5..07f2382b0368 100644 --- a/packaging/pom.xml +++ b/packaging/pom.xml @@ -213,6 +213,11 @@ hive-druid-handler ${project.version} + + org.apache.hive + hive-kafka-reader + ${project.version} + org.apache.hive hive-jdbc-handler diff --git a/pom.xml b/pom.xml index 7503cff5322a..be14e94f2441 100644 --- a/pom.xml +++ b/pom.xml @@ -63,6 +63,7 @@ packaging standalone-metastore upgrade-acid + hive-kafka-reader diff --git a/ql/src/java/org/apache/hadoop/hive/ql/index/IndexPredicateAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/index/IndexPredicateAnalyzer.java index da31f4d9a220..f39ba87a800e 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/index/IndexPredicateAnalyzer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/index/IndexPredicateAnalyzer.java @@ -17,16 +17,6 @@ */ package org.apache.hadoop.hive.ql.index; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.HashMap; -import java.util.HashSet; -import java.util.LinkedHashMap; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.Stack; - import org.apache.hadoop.hive.ql.exec.FunctionRegistry; import org.apache.hadoop.hive.ql.lib.DefaultGraphWalker; import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher; @@ -44,6 +34,7 @@ import org.apache.hadoop.hive.ql.plan.ExprNodeFieldDesc; import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc; import org.apache.hadoop.hive.ql.udf.generic.GenericUDF; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDFBaseCompare; import org.apache.hadoop.hive.ql.udf.generic.GenericUDFBridge; import org.apache.hadoop.hive.ql.udf.generic.GenericUDFToBinary; import org.apache.hadoop.hive.ql.udf.generic.GenericUDFToChar; @@ -54,7 +45,16 @@ import org.apache.hadoop.hive.ql.udf.generic.GenericUDFToUtcTimestamp; import org.apache.hadoop.hive.ql.udf.generic.GenericUDFToVarchar; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; -import org.apache.hadoop.hive.ql.udf.generic.GenericUDFBaseCompare; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.Stack; /** * IndexPredicateAnalyzer decomposes predicates, separating the parts @@ -181,7 +181,7 @@ public Object process(Node nd, Stack stack, //Check if ExprNodeColumnDesc is wrapped in expr. //If so, peel off. Otherwise return itself. - private ExprNodeDesc getColumnExpr(ExprNodeDesc expr) { + private static ExprNodeDesc getColumnExpr(ExprNodeDesc expr) { if (expr instanceof ExprNodeColumnDesc) { return expr; } diff --git a/ql/src/test/queries/clientpositive/kafka_storage_handler.q b/ql/src/test/queries/clientpositive/kafka_storage_handler.q new file mode 100644 index 000000000000..e5e7261e119b --- /dev/null +++ b/ql/src/test/queries/clientpositive/kafka_storage_handler.q @@ -0,0 +1,46 @@ +SET hive.vectorized.execution.enabled=false; + +CREATE EXTERNAL TABLE kafka_table +(`__time` timestamp , `page` string, `user` string, `language` string, +`country` string,`continent` string, `namespace` string, `newPage` boolean, `unpatrolled` boolean, +`anonymous` boolean, `robot` boolean, added int, deleted int, delta bigint) +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", +"kafka.bootstrap.servers"="localhost:9092", +"kafka.serde.class"="org.apache.hadoop.hive.serde2.JsonSerDe") +; + +DESCRIBE EXTENDED kafka_table; + +Select * FROM kafka_table; + +Select count(*) FROM kafka_table; + +Select `__partition`, `__offset`,`__time`, `page`, `user`, `language`, `country`,`continent`, `namespace`, `newPage` , +`unpatrolled` , `anonymous` , `robot` , added , deleted , delta +from kafka_table where `__timestamp` > 1533960760123; +Select `__partition`, `__offset`, `__time`, `page`, `user`, `language`, `country`,`continent`, `namespace`, `newPage` , +`unpatrolled` , `anonymous` , `robot` , added , deleted , delta +from kafka_table where `__timestamp` > 533960760123; + +Select `__partition`, `__offset`,`__time`, `page`, `user`, `language`, `country`,`continent`, `namespace`, `newPage` , +`unpatrolled` , `anonymous` , `robot` , added , deleted , delta +from kafka_table where `__offset` > 7 and `__partition` = 0 OR +`__offset` = 4 and `__partition` = 0 OR `__offset` <= 1 and `__partition` = 0; + +CREATE EXTERNAL TABLE kafka_table_2 +(`__time` timestamp with local time zone , `page` string, `user` string, `language` string, +`country` string,`continent` string, `namespace` string, `newPage` boolean, `unpatrolled` boolean, +`anonymous` boolean, `robot` boolean, added int, deleted int, delta bigint) +STORED BY 'org.apache.hadoop.hive.kafka.KafkaStorageHandler' +TBLPROPERTIES +("kafka.topic" = "test-topic", +"kafka.bootstrap.servers"="localhost:9092"); + +Select `__partition`, `__offset`, `__time`, `page`, `user`, `language`, `country`,`continent`, `namespace`, `newPage` , +`unpatrolled` , `anonymous` , `robot` , added , deleted , delta +FROM kafka_table_2; + +Select count(*) FROM kafka_table_2; \ No newline at end of file diff --git a/ql/src/test/results/clientpositive/druid/kafka_storage_handler.q.out b/ql/src/test/results/clientpositive/druid/kafka_storage_handler.q.out new file mode 100644 index 000000000000..4b76504ca890 --- /dev/null +++ b/ql/src/test/results/clientpositive/druid/kafka_storage_handler.q.out @@ -0,0 +1,184 @@ +PREHOOK: query: CREATE EXTERNAL TABLE kafka_table +(`__time` timestamp , `page` string, `user` string, `language` string, +`country` string,`continent` string, `namespace` string, `newPage` boolean, `unpatrolled` boolean, +`anonymous` boolean, `robot` boolean, added int, deleted int, delta bigint) +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", +"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 +POSTHOOK: query: CREATE EXTERNAL TABLE kafka_table +(`__time` timestamp , `page` string, `user` string, `language` string, +`country` string,`continent` string, `namespace` string, `newPage` boolean, `unpatrolled` boolean, +`anonymous` boolean, `robot` boolean, added int, deleted int, delta bigint) +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", +"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 +PREHOOK: query: DESCRIBE EXTENDED kafka_table +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@kafka_table +POSTHOOK: query: DESCRIBE EXTENDED kafka_table +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@kafka_table +__time timestamp from deserializer +page string from deserializer +user string from deserializer +language string from deserializer +country string from deserializer +continent string from deserializer +namespace string from deserializer +newpage boolean from deserializer +unpatrolled boolean from deserializer +anonymous boolean from deserializer +robot boolean from deserializer +added int from deserializer +deleted int from deserializer +delta bigint from deserializer +__partition int from deserializer +__offset bigint from deserializer +__timestamp bigint from deserializer + +#### A masked pattern was here #### +PREHOOK: query: Select * FROM kafka_table +PREHOOK: type: QUERY +PREHOOK: Input: default@kafka_table +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: Select * FROM kafka_table +POSTHOOK: type: QUERY +POSTHOOK: Input: default@kafka_table +POSTHOOK: Output: hdfs://### HDFS PATH ### +NULL Gypsy Danger nuclear en United States North America article true true false false 57 200 -143 0 0 1533945692531 +NULL Striker Eureka speed en Australia Australia wikipedia true false false true 459 129 330 0 1 1533945692556 +NULL Cherno Alpha masterYi ru Russia Asia article true false false true 123 12 111 0 2 1533945692556 +NULL Crimson Typhoon triplets zh China Asia wikipedia false true false true 905 5 900 0 3 1533945692557 +NULL Coyote Tango stringer ja Japan Asia wikipedia false true false true 1 10 -9 0 4 1533945692557 +NULL Gypsy Danger nuclear en United States North America article true true false false 57 200 -143 0 5 1533945692557 +NULL Striker Eureka speed en Australia Australia wikipedia true false false true 459 129 330 0 6 1533945692562 +NULL Cherno Alpha masterYi ru Russia Asia article true false false true 123 12 111 0 7 1533945692562 +NULL Crimson Typhoon triplets zh China Asia wikipedia false true false true 905 5 900 0 8 1533945692562 +NULL Coyote Tango stringer ja Japan Asia wikipedia false true false true 1 10 -9 0 9 1533945692562 +PREHOOK: query: Select count(*) FROM kafka_table +PREHOOK: type: QUERY +PREHOOK: Input: default@kafka_table +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: Select count(*) FROM kafka_table +POSTHOOK: type: QUERY +POSTHOOK: Input: default@kafka_table +POSTHOOK: Output: hdfs://### HDFS PATH ### +10 +PREHOOK: query: Select `__partition`, `__offset`,`__time`, `page`, `user`, `language`, `country`,`continent`, `namespace`, `newPage` , +`unpatrolled` , `anonymous` , `robot` , added , deleted , delta +from kafka_table where `__timestamp` > 1533960760123 +PREHOOK: type: QUERY +PREHOOK: Input: default@kafka_table +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: Select `__partition`, `__offset`,`__time`, `page`, `user`, `language`, `country`,`continent`, `namespace`, `newPage` , +`unpatrolled` , `anonymous` , `robot` , added , deleted , delta +from kafka_table where `__timestamp` > 1533960760123 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@kafka_table +POSTHOOK: Output: hdfs://### HDFS PATH ### +PREHOOK: query: Select `__partition`, `__offset`, `__time`, `page`, `user`, `language`, `country`,`continent`, `namespace`, `newPage` , +`unpatrolled` , `anonymous` , `robot` , added , deleted , delta +from kafka_table where `__timestamp` > 533960760123 +PREHOOK: type: QUERY +PREHOOK: Input: default@kafka_table +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: Select `__partition`, `__offset`, `__time`, `page`, `user`, `language`, `country`,`continent`, `namespace`, `newPage` , +`unpatrolled` , `anonymous` , `robot` , added , deleted , delta +from kafka_table where `__timestamp` > 533960760123 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@kafka_table +POSTHOOK: Output: hdfs://### HDFS PATH ### +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 `__partition`, `__offset`,`__time`, `page`, `user`, `language`, `country`,`continent`, `namespace`, `newPage` , +`unpatrolled` , `anonymous` , `robot` , added , deleted , delta +from kafka_table where `__offset` > 7 and `__partition` = 0 OR +`__offset` = 4 and `__partition` = 0 OR `__offset` <= 1 and `__partition` = 0 +PREHOOK: type: QUERY +PREHOOK: Input: default@kafka_table +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: Select `__partition`, `__offset`,`__time`, `page`, `user`, `language`, `country`,`continent`, `namespace`, `newPage` , +`unpatrolled` , `anonymous` , `robot` , added , deleted , delta +from kafka_table where `__offset` > 7 and `__partition` = 0 OR +`__offset` = 4 and `__partition` = 0 OR `__offset` <= 1 and `__partition` = 0 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@kafka_table +POSTHOOK: Output: hdfs://### HDFS PATH ### +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 4 NULL Coyote Tango stringer ja Japan Asia wikipedia false true false true 1 10 -9 +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: CREATE EXTERNAL TABLE kafka_table_2 +(`__time` timestamp with local time zone , `page` string, `user` string, `language` string, +`country` string,`continent` string, `namespace` string, `newPage` boolean, `unpatrolled` boolean, +`anonymous` boolean, `robot` boolean, added int, deleted int, delta bigint) +STORED BY 'org.apache.hadoop.hive.kafka.KafkaStorageHandler' +TBLPROPERTIES +("kafka.topic" = "test-topic", +"kafka.bootstrap.servers"="localhost:9092") +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@kafka_table_2 +POSTHOOK: query: CREATE EXTERNAL TABLE kafka_table_2 +(`__time` timestamp with local time zone , `page` string, `user` string, `language` string, +`country` string,`continent` string, `namespace` string, `newPage` boolean, `unpatrolled` boolean, +`anonymous` boolean, `robot` boolean, added int, deleted int, delta bigint) +STORED BY 'org.apache.hadoop.hive.kafka.KafkaStorageHandler' +TBLPROPERTIES +("kafka.topic" = "test-topic", +"kafka.bootstrap.servers"="localhost:9092") +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@kafka_table_2 +PREHOOK: query: Select `__partition`, `__offset`, `__time`, `page`, `user`, `language`, `country`,`continent`, `namespace`, `newPage` , +`unpatrolled` , `anonymous` , `robot` , added , deleted , delta +FROM kafka_table_2 +PREHOOK: type: QUERY +PREHOOK: Input: default@kafka_table_2 +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: Select `__partition`, `__offset`, `__time`, `page`, `user`, `language`, `country`,`continent`, `namespace`, `newPage` , +`unpatrolled` , `anonymous` , `robot` , added , deleted , delta +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 +PREHOOK: query: Select count(*) FROM kafka_table_2 +PREHOOK: type: QUERY +PREHOOK: Input: default@kafka_table_2 +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: Select count(*) FROM kafka_table_2 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@kafka_table_2 +POSTHOOK: Output: hdfs://### HDFS PATH ### +10 From d28f1c94956b65374d58f3cda94fbbee5ed3e6b4 Mon Sep 17 00:00:00 2001 From: Slim Bouguerra Date: Mon, 13 Aug 2018 11:00:55 -0700 Subject: [PATCH 02/33] re activate the test Change-Id: I7f8ef8a44271286abbd5a36a92ccde87d2ef8839 --- .../hive/cli/TestMiniDruidKafkaCliDriver.java | 1 - .../resources/testconfiguration.properties | 3 +- .../hadoop/hive/cli/control/CliConfigs.java | 1 + .../clientpositive/kafka_storage_handler.q | 3 +- .../druid/kafka_storage_handler.q.out | 36 ++++++++++++------- 5 files changed, 29 insertions(+), 15 deletions(-) diff --git a/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestMiniDruidKafkaCliDriver.java b/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestMiniDruidKafkaCliDriver.java index c54b2bf63ac5..b526c29cee09 100644 --- a/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestMiniDruidKafkaCliDriver.java +++ b/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestMiniDruidKafkaCliDriver.java @@ -56,7 +56,6 @@ public TestMiniDruidKafkaCliDriver(String name, File qfile) { this.qfile = qfile; } - @Ignore("HIVE-19509: Disable tests that are failing continuously") @Test public void testCliDriver() throws Exception { adapter.runTest(name, qfile); diff --git a/itests/src/test/resources/testconfiguration.properties b/itests/src/test/resources/testconfiguration.properties index af7f089975f7..9ec28f50e26a 100644 --- a/itests/src/test/resources/testconfiguration.properties +++ b/itests/src/test/resources/testconfiguration.properties @@ -1725,7 +1725,8 @@ druid.query.files=druidmini_test1.q,\ druid.llap.local.query.files=druidmini_noop.q -druid.kafka.query.files=druidkafkamini_basic.q +druid.kafka.query.files=druidkafkamini_basic.q \ + kafka_storage_handler.q # tests to be run by TestErasureCodingHDFSCliDriver and TestCliDriver erasurecoding.shared.query.files=erasure_commands.q diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CliConfigs.java b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CliConfigs.java index 92919e9daf0a..d512ce079f4a 100644 --- a/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CliConfigs.java +++ b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CliConfigs.java @@ -202,6 +202,7 @@ public MiniDruidKafkaCliConfig() { setQueryDir("ql/src/test/queries/clientpositive"); includesFrom(testConfigProps, "druid.kafka.query.files"); + excludeQuery("druidkafkamini_basic.q") // HIVE-19509 setResultsDir("ql/src/test/results/clientpositive/druid"); setLogDir("itests/qtest/target/tmp/log"); diff --git a/ql/src/test/queries/clientpositive/kafka_storage_handler.q b/ql/src/test/queries/clientpositive/kafka_storage_handler.q index e5e7261e119b..3efdb859ad4c 100644 --- a/ql/src/test/queries/clientpositive/kafka_storage_handler.q +++ b/ql/src/test/queries/clientpositive/kafka_storage_handler.q @@ -14,7 +14,8 @@ TBLPROPERTIES DESCRIBE EXTENDED kafka_table; -Select * FROM kafka_table; +Select `__partition` , `__offset`,`__time`, `page`, `user`, `language`, `country`,`continent`, `namespace`, `newPage` , +`unpatrolled` , `anonymous` , `robot` , added , deleted , delta FROM kafka_table; Select count(*) FROM kafka_table; diff --git a/ql/src/test/results/clientpositive/druid/kafka_storage_handler.q.out b/ql/src/test/results/clientpositive/druid/kafka_storage_handler.q.out index 4b76504ca890..a936a26aa7b5 100644 --- a/ql/src/test/results/clientpositive/druid/kafka_storage_handler.q.out +++ b/ql/src/test/results/clientpositive/druid/kafka_storage_handler.q.out @@ -49,24 +49,26 @@ __offset bigint from deserializer __timestamp bigint from deserializer #### A masked pattern was here #### -PREHOOK: query: Select * FROM kafka_table +PREHOOK: query: Select `__partition` , `__offset`,`__time`, `page`, `user`, `language`, `country`,`continent`, `namespace`, `newPage` , +`unpatrolled` , `anonymous` , `robot` , added , deleted , delta FROM kafka_table PREHOOK: type: QUERY PREHOOK: Input: default@kafka_table PREHOOK: Output: hdfs://### HDFS PATH ### -POSTHOOK: query: Select * FROM kafka_table +POSTHOOK: query: Select `__partition` , `__offset`,`__time`, `page`, `user`, `language`, `country`,`continent`, `namespace`, `newPage` , +`unpatrolled` , `anonymous` , `robot` , added , deleted , delta FROM kafka_table POSTHOOK: type: QUERY POSTHOOK: Input: default@kafka_table POSTHOOK: Output: hdfs://### HDFS PATH ### -NULL Gypsy Danger nuclear en United States North America article true true false false 57 200 -143 0 0 1533945692531 -NULL Striker Eureka speed en Australia Australia wikipedia true false false true 459 129 330 0 1 1533945692556 -NULL Cherno Alpha masterYi ru Russia Asia article true false false true 123 12 111 0 2 1533945692556 -NULL Crimson Typhoon triplets zh China Asia wikipedia false true false true 905 5 900 0 3 1533945692557 -NULL Coyote Tango stringer ja Japan Asia wikipedia false true false true 1 10 -9 0 4 1533945692557 -NULL Gypsy Danger nuclear en United States North America article true true false false 57 200 -143 0 5 1533945692557 -NULL Striker Eureka speed en Australia Australia wikipedia true false false true 459 129 330 0 6 1533945692562 -NULL Cherno Alpha masterYi ru Russia Asia article true false false true 123 12 111 0 7 1533945692562 -NULL Crimson Typhoon triplets zh China Asia wikipedia false true false true 905 5 900 0 8 1533945692562 -NULL Coyote Tango stringer ja Japan Asia wikipedia false true false true 1 10 -9 0 9 1533945692562 +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 PREHOOK: type: QUERY PREHOOK: Input: default@kafka_table @@ -88,6 +90,16 @@ from kafka_table where `__timestamp` > 1533960760123 POSTHOOK: type: QUERY POSTHOOK: Input: default@kafka_table POSTHOOK: Output: hdfs://### HDFS PATH ### +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 `__partition`, `__offset`, `__time`, `page`, `user`, `language`, `country`,`continent`, `namespace`, `newPage` , `unpatrolled` , `anonymous` , `robot` , added , deleted , delta from kafka_table where `__timestamp` > 533960760123 From f71f68d8f7458409d5f341c1ab262d6a894ebcc0 Mon Sep 17 00:00:00 2001 From: Slim Bouguerra Date: Mon, 13 Aug 2018 13:02:07 -0700 Subject: [PATCH 03/33] refactor names and added copy jars Change-Id: I11ec3aa4f9e96efc81ca8e9994c7409625384764 --- hive-kafka-reader/pom.xml | 161 ------------------ itests/qtest/pom.xml | 2 +- .../hadoop/hive/cli/control/CliConfigs.java | 2 +- kafka-handler/pom.xml | 156 +++++++++++++++++ .../hadoop/hive/kafka/GenericKafkaSerDe.java | 0 .../hadoop/hive/kafka/KafkaJsonSerDe.java | 0 .../hive/kafka/KafkaPullerInputFormat.java | 12 +- .../hive/kafka/KafkaPullerInputSplit.java | 0 .../hive/kafka/KafkaPullerRecordReader.java | 65 ++++--- .../hive/kafka/KafkaRecordIterator.java | 0 .../hive/kafka/KafkaRecordWritable.java | 0 .../hadoop/hive/kafka/KafkaScanTrimmer.java | 0 .../hive/kafka/KafkaStorageHandler.java | 19 ++- .../hive/kafka/KafkaStreamingUtils.java | 43 ++++- .../hive/kafka/KafkaPullerInputSplitTest.java | 0 .../hive/kafka/KafkaRecordIteratorTest.java | 0 .../hive/kafka/KafkaRecordWritableTest.java | 0 .../hive/kafka/KafkaScanTrimmerTest.java | 0 .../hive/kafka/KafkaStreamingUtilsTest.java | 0 pom.xml | 2 +- 20 files changed, 263 insertions(+), 199 deletions(-) delete mode 100644 hive-kafka-reader/pom.xml create mode 100644 kafka-handler/pom.xml rename {hive-kafka-reader => kafka-handler}/src/java/org/apache/hadoop/hive/kafka/GenericKafkaSerDe.java (100%) rename {hive-kafka-reader => kafka-handler}/src/java/org/apache/hadoop/hive/kafka/KafkaJsonSerDe.java (100%) rename {hive-kafka-reader => kafka-handler}/src/java/org/apache/hadoop/hive/kafka/KafkaPullerInputFormat.java (95%) rename {hive-kafka-reader => kafka-handler}/src/java/org/apache/hadoop/hive/kafka/KafkaPullerInputSplit.java (100%) rename {hive-kafka-reader => kafka-handler}/src/java/org/apache/hadoop/hive/kafka/KafkaPullerRecordReader.java (80%) rename {hive-kafka-reader => kafka-handler}/src/java/org/apache/hadoop/hive/kafka/KafkaRecordIterator.java (100%) rename {hive-kafka-reader => kafka-handler}/src/java/org/apache/hadoop/hive/kafka/KafkaRecordWritable.java (100%) rename {hive-kafka-reader => kafka-handler}/src/java/org/apache/hadoop/hive/kafka/KafkaScanTrimmer.java (100%) rename {hive-kafka-reader => kafka-handler}/src/java/org/apache/hadoop/hive/kafka/KafkaStorageHandler.java (89%) rename {hive-kafka-reader => kafka-handler}/src/java/org/apache/hadoop/hive/kafka/KafkaStreamingUtils.java (61%) rename {hive-kafka-reader => kafka-handler}/src/test/org/apache/hadoop/hive/kafka/KafkaPullerInputSplitTest.java (100%) rename {hive-kafka-reader => kafka-handler}/src/test/org/apache/hadoop/hive/kafka/KafkaRecordIteratorTest.java (100%) rename {hive-kafka-reader => kafka-handler}/src/test/org/apache/hadoop/hive/kafka/KafkaRecordWritableTest.java (100%) rename {hive-kafka-reader => kafka-handler}/src/test/org/apache/hadoop/hive/kafka/KafkaScanTrimmerTest.java (100%) rename {hive-kafka-reader => kafka-handler}/src/test/org/apache/hadoop/hive/kafka/KafkaStreamingUtilsTest.java (100%) diff --git a/hive-kafka-reader/pom.xml b/hive-kafka-reader/pom.xml deleted file mode 100644 index 86dbcde8f4f9..000000000000 --- a/hive-kafka-reader/pom.xml +++ /dev/null @@ -1,161 +0,0 @@ - - - - - - org.apache.hive - hive - 4.0.0-SNAPSHOT - ../pom.xml - - 4.0.0 - - - .. - 1.0.1 - - - hive-kafka-reader - jar - Hive Kafka Streaming Ingestion Handler - - - - - - org.apache.hive - hive-exec - provided - ${project.version} - - - org.apache.hadoop - hadoop-common - - - org.apache.hadoop - hadoop-client - - - org.apache.kafka - kafka-clients - ${kafka.version} - - - - junit - junit - ${junit.version} - test - - - org.apache.kafka - kafka-clients - ${kafka.version} - test - test - - - - org.apache.kafka - kafka_2.11 - ${kafka.version} - test - test - - - org.apache.kafka - kafka_2.11 - ${kafka.version} - test - - - - - - dev-fast-build - - - skipShade - !true - - - - - - org.apache.maven.plugins - maven-shade-plugin - ${maven.shade.plugin.version} - - - - - package - - shade - - - true - false - - - org.apache.kafka:* - - - - - *:* - - META-INF/*.SF - META-INF/*.DSA - META-INF/*.RSA - static/ - - - - - - - - - - - - - ${basedir}/src/java - ${basedir}/src/test - - - org.apache.maven.plugins - maven-jar-plugin - - - - test-jar - - - - - - - \ No newline at end of file diff --git a/itests/qtest/pom.xml b/itests/qtest/pom.xml index a0cfcaf8c1f1..801a43d02fca 100644 --- a/itests/qtest/pom.xml +++ b/itests/qtest/pom.xml @@ -140,7 +140,7 @@ org.apache.hive - hive-kafka-reader + kafka-handler ${project.version} test diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CliConfigs.java b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CliConfigs.java index d512ce079f4a..491b6db581a0 100644 --- a/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CliConfigs.java +++ b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CliConfigs.java @@ -202,7 +202,7 @@ public MiniDruidKafkaCliConfig() { setQueryDir("ql/src/test/queries/clientpositive"); includesFrom(testConfigProps, "druid.kafka.query.files"); - excludeQuery("druidkafkamini_basic.q") // HIVE-19509 + excludeQuery("druidkafkamini_basic.q"); // HIVE-19509 setResultsDir("ql/src/test/results/clientpositive/druid"); setLogDir("itests/qtest/target/tmp/log"); diff --git a/kafka-handler/pom.xml b/kafka-handler/pom.xml new file mode 100644 index 000000000000..cbcb2fb170d2 --- /dev/null +++ b/kafka-handler/pom.xml @@ -0,0 +1,156 @@ + + + + + + org.apache.hive + hive + 4.0.0-SNAPSHOT + ../pom.xml + + 4.0.0 + + + .. + 1.0.1 + + + kafka-handler + jar + Hive Kafka Storage Handler + + + + + org.apache.hive + hive-exec + provided + ${project.version} + + + org.apache.hadoop + hadoop-common + + + org.apache.hadoop + hadoop-client + + + org.apache.kafka + kafka-clients + ${kafka.version} + + + + junit + junit + ${junit.version} + test + + + org.apache.kafka + kafka-clients + ${kafka.version} + test + test + + + + org.apache.kafka + kafka_2.11 + ${kafka.version} + test + test + + + org.apache.kafka + kafka_2.11 + ${kafka.version} + test + + + + + + dev-fast-build + + + skipShade + !true + + + + + + org.apache.maven.plugins + maven-shade-plugin + ${maven.shade.plugin.version} + + + package + + shade + + + true + false + + + org.apache.kafka:* + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + static/ + + + + + + + + + + + + + ${basedir}/src/java + ${basedir}/src/test + + + org.apache.maven.plugins + maven-jar-plugin + + + + test-jar + + + + + + + \ No newline at end of file diff --git a/hive-kafka-reader/src/java/org/apache/hadoop/hive/kafka/GenericKafkaSerDe.java b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/GenericKafkaSerDe.java similarity index 100% rename from hive-kafka-reader/src/java/org/apache/hadoop/hive/kafka/GenericKafkaSerDe.java rename to kafka-handler/src/java/org/apache/hadoop/hive/kafka/GenericKafkaSerDe.java diff --git a/hive-kafka-reader/src/java/org/apache/hadoop/hive/kafka/KafkaJsonSerDe.java b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaJsonSerDe.java similarity index 100% rename from hive-kafka-reader/src/java/org/apache/hadoop/hive/kafka/KafkaJsonSerDe.java rename to kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaJsonSerDe.java diff --git a/hive-kafka-reader/src/java/org/apache/hadoop/hive/kafka/KafkaPullerInputFormat.java b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaPullerInputFormat.java similarity index 95% rename from hive-kafka-reader/src/java/org/apache/hadoop/hive/kafka/KafkaPullerInputFormat.java rename to kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaPullerInputFormat.java index 84a0d621b6a3..2931394ef89f 100644 --- a/hive-kafka-reader/src/java/org/apache/hadoop/hive/kafka/KafkaPullerInputFormat.java +++ b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaPullerInputFormat.java @@ -60,11 +60,6 @@ public class KafkaPullerInputFormat extends InputFormat { - public static final String HIVE_KAFKA_TOPIC = "kafka.topic"; - public static final String CONSUMER_CONFIGURATION_PREFIX = "kafka.consumer"; - public static final String HIVE_KAFKA_BOOTSTRAP_SERVERS = "kafka.bootstrap.servers"; - public static final String GENERATION_TIMEOUT_MS = "hive.kafka.split.generation.timeout.ms"; - private static final Logger log = LoggerFactory.getLogger(KafkaPullerInputFormat.class); @@ -135,8 +130,11 @@ private List computeSplits(Configuration configuration) // this will be used to harness some KAFKA blocking calls final ExecutorService EXECUTOR = Executors.newSingleThreadExecutor(); try (KafkaConsumer consumer = new KafkaConsumer(KafkaStreamingUtils.consumerProperties(configuration))) { - final String topic = configuration.get(HIVE_KAFKA_TOPIC); - final long timeoutMs = configuration.getLong(GENERATION_TIMEOUT_MS, 5000); + final String topic = configuration.get(KafkaStorageHandler.HIVE_KAFKA_TOPIC); + final long timeoutMs = configuration.getLong( + KafkaStorageHandler.HIVE_KAFKA_POLL_TIMEOUT, + KafkaStorageHandler.DEFAULT_CONSUMER_POLL_TIMEOUT_MS + ); // hive depends on FileSplits JobConf jobConf = new JobConf(configuration); Path[] tablePaths = org.apache.hadoop.mapred.FileInputFormat.getInputPaths(jobConf); diff --git a/hive-kafka-reader/src/java/org/apache/hadoop/hive/kafka/KafkaPullerInputSplit.java b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaPullerInputSplit.java similarity index 100% rename from hive-kafka-reader/src/java/org/apache/hadoop/hive/kafka/KafkaPullerInputSplit.java rename to kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaPullerInputSplit.java diff --git a/hive-kafka-reader/src/java/org/apache/hadoop/hive/kafka/KafkaPullerRecordReader.java b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaPullerRecordReader.java similarity index 80% rename from hive-kafka-reader/src/java/org/apache/hadoop/hive/kafka/KafkaPullerRecordReader.java rename to kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaPullerRecordReader.java index 229a1e4ad9fa..0e353dc52506 100644 --- a/hive-kafka-reader/src/java/org/apache/hadoop/hive/kafka/KafkaPullerRecordReader.java +++ b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaPullerRecordReader.java @@ -36,11 +36,10 @@ import java.util.Properties; public class KafkaPullerRecordReader extends RecordReader - implements org.apache.hadoop.mapred.RecordReader { + implements org.apache.hadoop.mapred.RecordReader +{ private static final Logger log = LoggerFactory.getLogger(KafkaPullerRecordReader.class); - public static final String HIVE_KAFKA_POLL_TIMEOUT = "hive.kafka.poll.timeout.ms"; - public static final long DEFAULT_CONSUMER_POLL_TIMEOUT_MS = 5000l; private final Closer closer = Closer.create(); private KafkaConsumer consumer = null; @@ -58,10 +57,12 @@ public class KafkaPullerRecordReader extends RecordReader record = recordsCursor.next(); bytesWritable.set(record); @@ -119,20 +130,28 @@ private void computeTopicPartitionOffsets(KafkaPullerInputSplit split) { return false; } - @Override public NullWritable createKey() { + @Override + public NullWritable createKey() + { return NullWritable.get(); } - @Override public KafkaRecordWritable createValue() { + @Override + public KafkaRecordWritable createValue() + { return new KafkaRecordWritable(); } - @Override public long getPos() throws IOException { + @Override + public long getPos() throws IOException + { return consumedRecords; } - @Override public boolean nextKeyValue() throws IOException { + @Override + public boolean nextKeyValue() throws IOException + { currentWritableValue = new KafkaRecordWritable(); if (next(NullWritable.get(), currentWritableValue)) { return true; @@ -141,15 +160,21 @@ private void computeTopicPartitionOffsets(KafkaPullerInputSplit split) { return false; } - @Override public NullWritable getCurrentKey() throws IOException, InterruptedException { + @Override + public NullWritable getCurrentKey() throws IOException, InterruptedException + { return NullWritable.get(); } - @Override public KafkaRecordWritable getCurrentValue() throws IOException, InterruptedException { + @Override + public KafkaRecordWritable getCurrentValue() throws IOException, InterruptedException + { return Preconditions.checkNotNull(currentWritableValue); } - @Override public float getProgress() throws IOException { + @Override + public float getProgress() throws IOException + { if (consumedRecords == 0) { return 0f; } @@ -159,7 +184,9 @@ private void computeTopicPartitionOffsets(KafkaPullerInputSplit split) { return consumedRecords * 1.0f / totalNumberRecords; } - @Override public void close() throws IOException { + @Override + public void close() throws IOException + { if (!started) { return; } diff --git a/hive-kafka-reader/src/java/org/apache/hadoop/hive/kafka/KafkaRecordIterator.java b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaRecordIterator.java similarity index 100% rename from hive-kafka-reader/src/java/org/apache/hadoop/hive/kafka/KafkaRecordIterator.java rename to kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaRecordIterator.java diff --git a/hive-kafka-reader/src/java/org/apache/hadoop/hive/kafka/KafkaRecordWritable.java b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaRecordWritable.java similarity index 100% rename from hive-kafka-reader/src/java/org/apache/hadoop/hive/kafka/KafkaRecordWritable.java rename to kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaRecordWritable.java diff --git a/hive-kafka-reader/src/java/org/apache/hadoop/hive/kafka/KafkaScanTrimmer.java b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaScanTrimmer.java similarity index 100% rename from hive-kafka-reader/src/java/org/apache/hadoop/hive/kafka/KafkaScanTrimmer.java rename to kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaScanTrimmer.java diff --git a/hive-kafka-reader/src/java/org/apache/hadoop/hive/kafka/KafkaStorageHandler.java b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaStorageHandler.java similarity index 89% rename from hive-kafka-reader/src/java/org/apache/hadoop/hive/kafka/KafkaStorageHandler.java rename to kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaStorageHandler.java index 86b44b998bdb..0639707e733f 100644 --- a/hive-kafka-reader/src/java/org/apache/hadoop/hive/kafka/KafkaStorageHandler.java +++ b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaStorageHandler.java @@ -34,13 +34,10 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.IOException; import java.util.HashMap; import java.util.Map; -import static org.apache.hadoop.hive.kafka.KafkaPullerInputFormat.CONSUMER_CONFIGURATION_PREFIX; -import static org.apache.hadoop.hive.kafka.KafkaPullerInputFormat.HIVE_KAFKA_BOOTSTRAP_SERVERS; -import static org.apache.hadoop.hive.kafka.KafkaPullerInputFormat.HIVE_KAFKA_TOPIC; - public class KafkaStorageHandler implements HiveStorageHandler { @@ -48,6 +45,11 @@ public class KafkaStorageHandler implements HiveStorageHandler public static final String __PARTITION = "__partition"; public static final String __OFFSET = "__offset"; public static final String SERDE_CLASS_NAME = "kafka.serde.class"; + public static final String HIVE_KAFKA_TOPIC = "kafka.topic"; + public static final String CONSUMER_CONFIGURATION_PREFIX = "kafka.consumer"; + public static final String HIVE_KAFKA_BOOTSTRAP_SERVERS = "kafka.bootstrap.servers"; + public static final String HIVE_KAFKA_POLL_TIMEOUT = "hive.kafka.poll.timeout.ms"; + public static final long DEFAULT_CONSUMER_POLL_TIMEOUT_MS = 5000l; // 5 seconds private static final Logger log = LoggerFactory.getLogger(KafkaStorageHandler.class); @@ -121,7 +123,7 @@ public void configureInputJobProperties( String key = entry.getKey().toString().substring(CONSUMER_CONFIGURATION_PREFIX.length() + 1); String value = entry.getValue().toString(); jobProperties.put(key, value); - log.info("Setting extra job properties: key [{}] -> value [{}]", key, value ); + log.info("Setting extra job properties: key [{}] -> value [{}]", key, value); }); } @@ -156,10 +158,15 @@ public void configureTableJobProperties( @Override public void configureJobConf(TableDesc tableDesc, JobConf jobConf) { - Map properties = new HashMap<>(); configureInputJobProperties(tableDesc, properties); properties.forEach((key, value) -> jobConf.set(key, value)); + try { + KafkaStreamingUtils.copyDependencyJars(jobConf, KafkaStorageHandler.class); + } + catch (IOException e) { + throw new RuntimeException(e); + } } @Override diff --git a/hive-kafka-reader/src/java/org/apache/hadoop/hive/kafka/KafkaStreamingUtils.java b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaStreamingUtils.java similarity index 61% rename from hive-kafka-reader/src/java/org/apache/hadoop/hive/kafka/KafkaStreamingUtils.java rename to kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaStreamingUtils.java index 958c4e42e9b2..e5c83f935e57 100644 --- a/hive-kafka-reader/src/java/org/apache/hadoop/hive/kafka/KafkaStreamingUtils.java +++ b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaStreamingUtils.java @@ -19,11 +19,20 @@ 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.ql.exec.Utilities; +import org.apache.hadoop.util.StringUtils; import org.apache.kafka.clients.CommonClientConfigs; 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.Properties; +import java.util.Set; +import java.util.stream.Collectors; import static org.apache.kafka.clients.consumer.ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG; import static org.apache.kafka.clients.consumer.ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG; @@ -51,19 +60,47 @@ public static Properties consumerProperties(Configuration configuration) { props.setProperty("enable.auto.commit", "false"); // we are seeking in the stream so no reset props.setProperty("auto.offset.reset", "none"); - String brokerEndPoint = configuration.get(KafkaPullerInputFormat.HIVE_KAFKA_BOOTSTRAP_SERVERS); + String brokerEndPoint = configuration.get(KafkaStorageHandler.HIVE_KAFKA_BOOTSTRAP_SERVERS); props.setProperty(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, brokerEndPoint); props.setProperty(KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName()); props.setProperty(VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName()); // user can always override stuff final Map kafkaProperties = - configuration.getValByRegex("^" + KafkaPullerInputFormat.CONSUMER_CONFIGURATION_PREFIX + "\\..*"); + configuration.getValByRegex("^" + KafkaStorageHandler.CONSUMER_CONFIGURATION_PREFIX + "\\..*"); for (Map.Entry entry : kafkaProperties.entrySet()) { props.setProperty(entry.getKey().substring( - KafkaPullerInputFormat.CONSUMER_CONFIGURATION_PREFIX.length() + 1), + KafkaStorageHandler.CONSUMER_CONFIGURATION_PREFIX.length() + 1), entry.getValue() ); } return props; } + + public 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.asList(classes).stream().filter(aClass -> aClass != null).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[jars.size()]))); + } } diff --git a/hive-kafka-reader/src/test/org/apache/hadoop/hive/kafka/KafkaPullerInputSplitTest.java b/kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaPullerInputSplitTest.java similarity index 100% rename from hive-kafka-reader/src/test/org/apache/hadoop/hive/kafka/KafkaPullerInputSplitTest.java rename to kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaPullerInputSplitTest.java diff --git a/hive-kafka-reader/src/test/org/apache/hadoop/hive/kafka/KafkaRecordIteratorTest.java b/kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaRecordIteratorTest.java similarity index 100% rename from hive-kafka-reader/src/test/org/apache/hadoop/hive/kafka/KafkaRecordIteratorTest.java rename to kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaRecordIteratorTest.java diff --git a/hive-kafka-reader/src/test/org/apache/hadoop/hive/kafka/KafkaRecordWritableTest.java b/kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaRecordWritableTest.java similarity index 100% rename from hive-kafka-reader/src/test/org/apache/hadoop/hive/kafka/KafkaRecordWritableTest.java rename to kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaRecordWritableTest.java diff --git a/hive-kafka-reader/src/test/org/apache/hadoop/hive/kafka/KafkaScanTrimmerTest.java b/kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaScanTrimmerTest.java similarity index 100% rename from hive-kafka-reader/src/test/org/apache/hadoop/hive/kafka/KafkaScanTrimmerTest.java rename to kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaScanTrimmerTest.java diff --git a/hive-kafka-reader/src/test/org/apache/hadoop/hive/kafka/KafkaStreamingUtilsTest.java b/kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaStreamingUtilsTest.java similarity index 100% rename from hive-kafka-reader/src/test/org/apache/hadoop/hive/kafka/KafkaStreamingUtilsTest.java rename to kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaStreamingUtilsTest.java diff --git a/pom.xml b/pom.xml index be14e94f2441..4e1ab9418627 100644 --- a/pom.xml +++ b/pom.xml @@ -63,7 +63,7 @@ packaging standalone-metastore upgrade-acid - hive-kafka-reader + kafka-handler From 5c442137e3f4b02086fd2c42ce354e4d4dbe4cd3 Mon Sep 17 00:00:00 2001 From: Slim Bouguerra Date: Mon, 13 Aug 2018 13:11:35 -0700 Subject: [PATCH 04/33] fix headers Change-Id: I478c0709ba2ca77a1139011006170e4ad0683617 --- .../hive/kafka/KafkaRecordIteratorTest.java | 21 +++++++++++++++---- .../hive/kafka/KafkaRecordWritableTest.java | 21 +++++++++++++++---- .../hive/kafka/KafkaStreamingUtilsTest.java | 21 +++++++++++++++---- 3 files changed, 51 insertions(+), 12 deletions(-) diff --git a/kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaRecordIteratorTest.java b/kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaRecordIteratorTest.java index 73629aa5577f..bc3141c718ba 100644 --- a/kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaRecordIteratorTest.java +++ b/kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaRecordIteratorTest.java @@ -1,7 +1,20 @@ -// -// Source code recreated from a .class file by IntelliJ IDEA -// (powered by Fernflower decompiler) -// +/* + * 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; diff --git a/kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaRecordWritableTest.java b/kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaRecordWritableTest.java index 68113514396c..1327eb33117f 100644 --- a/kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaRecordWritableTest.java +++ b/kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaRecordWritableTest.java @@ -1,7 +1,20 @@ -// -// Source code recreated from a .class file by IntelliJ IDEA -// (powered by Fernflower decompiler) -// +/* + * 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; diff --git a/kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaStreamingUtilsTest.java b/kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaStreamingUtilsTest.java index add3f7137ee2..9b107f137f1a 100644 --- a/kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaStreamingUtilsTest.java +++ b/kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaStreamingUtilsTest.java @@ -1,7 +1,20 @@ -// -// Source code recreated from a .class file by IntelliJ IDEA -// (powered by Fernflower decompiler) -// +/* + * 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; From dce3a0f8b4eb77a9929ff2b2edf9fde44d364244 Mon Sep 17 00:00:00 2001 From: Slim Bouguerra Date: Mon, 13 Aug 2018 15:25:32 -0700 Subject: [PATCH 05/33] clean code and add comments Change-Id: I1f8b3b748b5ab4f8f7e594bf85433affca83b50e --- .../hive/kafka/KafkaPullerInputFormat.java | 14 +++---- .../hive/kafka/KafkaPullerRecordReader.java | 30 +++++-------- .../hive/kafka/KafkaRecordIterator.java | 42 ++++++++++++------- 3 files changed, 44 insertions(+), 42 deletions(-) diff --git a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaPullerInputFormat.java b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaPullerInputFormat.java index 2931394ef89f..8d5cd5b20bfc 100644 --- a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaPullerInputFormat.java +++ b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaPullerInputFormat.java @@ -128,7 +128,7 @@ private List computeSplits(Configuration configuration) throws IOException, InterruptedException { // this will be used to harness some KAFKA blocking calls - final ExecutorService EXECUTOR = Executors.newSingleThreadExecutor(); + final ExecutorService execService = Executors.newSingleThreadExecutor(); try (KafkaConsumer consumer = new KafkaConsumer(KafkaStreamingUtils.consumerProperties(configuration))) { final String topic = configuration.get(KafkaStorageHandler.HIVE_KAFKA_TOPIC); final long timeoutMs = configuration.getLong( @@ -139,7 +139,7 @@ private List computeSplits(Configuration configuration) JobConf jobConf = new JobConf(configuration); Path[] tablePaths = org.apache.hadoop.mapred.FileInputFormat.getInputPaths(jobConf); - Future> futureFullHouse = EXECUTOR.submit(() -> buildFullScanFromKafka( + Future> futureFullHouse = execService.submit(() -> buildFullScanFromKafka( topic, consumer, tablePaths @@ -156,13 +156,13 @@ private List computeSplits(Configuration configuration) } - final ImmutableMap.Builder builder = new ImmutableMap.Builder(); - fullHouse.stream().forEach(input -> builder.put(new TopicPartition( + final ImmutableMap.Builder fullHouseMapBuilder = new ImmutableMap.Builder(); + fullHouse.stream().forEach(input -> fullHouseMapBuilder.put(new TopicPartition( input.getTopic(), input.getPartition() ), input)); - final KafkaScanTrimmer kafkaScanTrimmer = new KafkaScanTrimmer(builder.build(), consumer); + final KafkaScanTrimmer kafkaScanTrimmer = new KafkaScanTrimmer(fullHouseMapBuilder.build(), consumer); final String filterExprSerialized = configuration.get(TableScanDesc.FILTER_EXPR_CONF_STR); if (filterExprSerialized != null && !filterExprSerialized.isEmpty()) { @@ -174,7 +174,7 @@ private List computeSplits(Configuration configuration) .map(entry -> entry.getValue()) .collect(Collectors.toList()); - Future> futureTinyHouse = EXECUTOR.submit(trimmerWorker); + Future> futureTinyHouse = execService.submit(trimmerWorker); try { return futureTinyHouse.get(timeoutMs, TimeUnit.MILLISECONDS); } @@ -188,7 +188,7 @@ private List computeSplits(Configuration configuration) return fullHouse; } finally { - EXECUTOR.shutdown(); + execService.shutdown(); } } diff --git a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaPullerRecordReader.java b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaPullerRecordReader.java index 0e353dc52506..24f5141592c8 100644 --- a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaPullerRecordReader.java +++ b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaPullerRecordReader.java @@ -83,34 +83,27 @@ synchronized private void initialize(KafkaPullerInputSplit inputSplit, Configura { if (!started) { this.config = jobConf; - computeTopicPartitionOffsets(inputSplit); + this.startOffset = inputSplit.getStartOffset(); + this.endOffset = inputSplit.getEndOffset(); + this.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 + ); + totalNumberRecords += endOffset - startOffset; initConsumer(); pollTimeout = config.getLong( KafkaStorageHandler.HIVE_KAFKA_POLL_TIMEOUT, KafkaStorageHandler.DEFAULT_CONSUMER_POLL_TIMEOUT_MS ); log.debug("Consumer poll timeout [{}] ms", pollTimeout); - recordsCursor = new KafkaRecordIterator(consumer, topicPartition, startOffset, endOffset, pollTimeout); + this.recordsCursor = new KafkaRecordIterator(consumer, topicPartition, startOffset, endOffset, pollTimeout); started = true; } } - private void computeTopicPartitionOffsets(KafkaPullerInputSplit split) - { - String topic = split.getTopic(); - int partition = split.getPartition(); - startOffset = split.getStartOffset(); - endOffset = split.getEndOffset(); - topicPartition = new TopicPartition(topic, partition); - Preconditions.checkState( - startOffset >= 0 && startOffset <= endOffset, - "Start [%s] has to be positive and less or equal than End [%s]", - startOffset, - endOffset - ); - totalNumberRecords += endOffset - startOffset; - } - @Override synchronized public void initialize(org.apache.hadoop.mapreduce.InputSplit inputSplit, TaskAttemptContext context) { @@ -145,7 +138,6 @@ public KafkaRecordWritable createValue() @Override public long getPos() throws IOException { - return consumedRecords; } diff --git a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaRecordIterator.java b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaRecordIterator.java index a5e1c0b24755..d120b663ec24 100644 --- a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaRecordIterator.java +++ b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaRecordIterator.java @@ -43,6 +43,8 @@ *

* When provided with an end offset it will return records up to the record with offset == endOffset - 1, * Else If end offsets is null it will read up to the current end see {@link org.apache.kafka.clients.consumer.Consumer#endOffsets(Collection)} + *

+ * Current implementation of this Iterator will throw and exception if can not poll up to the endOffset - 1 */ public class KafkaRecordIterator implements Iterator> { @@ -52,39 +54,41 @@ public class KafkaRecordIterator implements Iterator records; private long currentOffset; private ConsumerRecord nextRecord; private boolean hasMore = true; - private Iterator> cursor = null; + + //Kafka consumer poll method return an iterator of records. + private Iterator> consumerRecordIterator = null; /** * @param consumer functional kafka consumer * @param topicPartition kafka topic partition * @param startOffset start position of stream. * @param endOffset requested end position. If null will read up to current last - * @param pollTimeout poll time out in ms + * @param pollTimeoutMs poll time out in ms */ public KafkaRecordIterator( Consumer consumer, TopicPartition topicPartition, - @Nullable Long startOffset, @Nullable Long endOffset, long pollTimeout + @Nullable Long startOffset, @Nullable Long endOffset, long pollTimeoutMs ) { this.consumer = Preconditions.checkNotNull(consumer, "Consumer can not be null"); this.topicPartition = Preconditions.checkNotNull(topicPartition, "Topic partition can not be null"); - this.pollTimeout = pollTimeout; - Preconditions.checkState(pollTimeout > 0, "poll timeout has to be positive number"); + this.pollTimeoutMs = pollTimeoutMs; + Preconditions.checkState(this.pollTimeoutMs > 0, "poll timeout has to be positive number"); this.startOffset = startOffset == null ? -1l : startOffset; this.endOffset = endOffset == null ? -1l : endOffset; } public KafkaRecordIterator( - Consumer consumer, TopicPartition tp, long pollTimeout + Consumer consumer, TopicPartition tp, long pollTimeoutMs ) { - this(consumer, tp, null, null, pollTimeout); + this(consumer, tp, null, null, pollTimeoutMs); } private void assignAndSeek() @@ -142,19 +146,25 @@ private void pollRecords() if (log.isTraceEnabled()) { stopwatch.reset().start(); } - records = consumer.poll(pollTimeout); + records = consumer.poll(pollTimeoutMs); if (log.isTraceEnabled()) { stopwatch.stop(); log.trace("Pulled [{}] records in [{}] ms", records.count(), stopwatch.elapsed(TimeUnit.MILLISECONDS) ); } - Preconditions.checkState(!records.isEmpty() || currentOffset == endOffset, - "Current read offset [%s]-TopicPartition:[%s], End offset[%s]." - + "Consumer returned 0 record due to exhausted poll timeout [%s] ms", - currentOffset, topicPartition.toString(), endOffset, pollTimeout + // Fail if we can not poll within one lap of pollTimeoutMs. + Preconditions.checkState( + !records.isEmpty() || currentOffset == endOffset, + "Current offset: [%s]-TopicPartition:[%s], target End offset:[%s]." + + "Consumer returned 0 record due to exhausted poll timeout [%s]ms, try increasing[%s] ", + currentOffset, + topicPartition.toString(), + endOffset, + pollTimeoutMs, + KafkaStorageHandler.HIVE_KAFKA_POLL_TIMEOUT ); - cursor = records.iterator(); + consumerRecordIterator = records.iterator(); } @Override @@ -168,8 +178,8 @@ public ConsumerRecord next() private void findNext() { - if (cursor.hasNext()) { - nextRecord = cursor.next(); + if (consumerRecordIterator.hasNext()) { + nextRecord = consumerRecordIterator.next(); hasMore = true; if (nextRecord.offset() < endOffset) { currentOffset = nextRecord.offset(); From 80d9d5bbd55ea6d1a5d2de8ddffa393e495008ed Mon Sep 17 00:00:00 2001 From: Slim Bouguerra Date: Mon, 13 Aug 2018 17:10:11 -0700 Subject: [PATCH 06/33] fix hive package Change-Id: I2f255590aab7cd0897583ab11ea02961d1114bbd --- packaging/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/packaging/pom.xml b/packaging/pom.xml index 07f2382b0368..0f0037bd6163 100644 --- a/packaging/pom.xml +++ b/packaging/pom.xml @@ -215,7 +215,7 @@ org.apache.hive - hive-kafka-reader + kafka-handler ${project.version} From 172019a6ab06d6026a7ba69f4b2c719f2afb2408 Mon Sep 17 00:00:00 2001 From: Slim Bouguerra Date: Mon, 13 Aug 2018 17:35:57 -0700 Subject: [PATCH 07/33] added more tests Change-Id: I59c8bee67877bb54a4dcf5803b9e15ab2c8f0c42 --- .../clientpositive/kafka_storage_handler.q | 94 +++++ .../druid/kafka_storage_handler.q.out | 358 ++++++++++++++++++ 2 files changed, 452 insertions(+) diff --git a/ql/src/test/queries/clientpositive/kafka_storage_handler.q b/ql/src/test/queries/clientpositive/kafka_storage_handler.q index 3efdb859ad4c..83a5015f0725 100644 --- a/ql/src/test/queries/clientpositive/kafka_storage_handler.q +++ b/ql/src/test/queries/clientpositive/kafka_storage_handler.q @@ -31,6 +31,100 @@ Select `__partition`, `__offset`,`__time`, `page`, `user`, `language`, `country` from kafka_table where `__offset` > 7 and `__partition` = 0 OR `__offset` = 4 and `__partition` = 0 OR `__offset` <= 1 and `__partition` = 0; + +-- non existing partition +Select count(*) from kafka_table where `__partition` = 1; + +-- non existing offset +Select count(*) from kafka_table where `__offset` = 100; + +-- less than non existing offset and partition +Select count(*) from kafka_table where `__offset` <= 100 and `__partition` <= 100; + +Drop table kafka_table_offsets; +create table kafka_table_offsets(partition_id int, max_offset bigint, insert_time timestamp); + +insert overwrite table kafka_table_offsets select `__partition`, min(`__offset`) - 1, CURRENT_TIMESTAMP from kafka_table group by `__partition`, CURRENT_TIMESTAMP ; + +-- check initial state is 0 for partition and 0 offsets +select partition_id, max_offset from kafka_table_offsets; + +Drop table orc_kafka_table; +Create table orc_kafka_table (partition_id int, row_offset bigint, kafka_ts bigint, + `__time` timestamp , `page` string, `user` string, `language` string, +`country` string,`continent` string, `namespace` string, `newPage` boolean, `unpatrolled` boolean, +`anonymous` boolean, `robot` boolean, added int, deleted int, delta bigint +) stored as ORC; + + +From kafka_table 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`, +`__time`, `page`, `user`, `language`, `country`,`continent`, `namespace`, `newPage` , +`unpatrolled` , `anonymous` , `robot` , added , deleted , delta +Insert overwrite table kafka_table_offsets select +`__partition`, max(`__offset`), CURRENT_TIMESTAMP group by `__partition`, CURRENT_TIMESTAMP; + +-- should ingest only first 3 rows +select count(*) from orc_kafka_table; + +-- check max offset is 2 +select partition_id, max_offset from kafka_table_offsets; + +-- 3 rows form 0 to 2 +select `partition_id`, `row_offset`,`__time`, `page`, `user`, `language`, `country`,`continent`, `namespace`, `newPage` , +`unpatrolled` , `anonymous` , `robot` , added , deleted , delta from orc_kafka_table; + + +-- insert the rest using inner join + +From kafka_table 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`, +`__time`, `page`, `user`, `language`, `country`,`continent`, `namespace`, `newPage` , +`unpatrolled` , `anonymous` , `robot` , added , deleted , delta +Insert overwrite table kafka_table_offsets select +`__partition`, max(`__offset`), CURRENT_TIMESTAMP group by `__partition`, CURRENT_TIMESTAMP; + +-- check that max offset is 9 +select partition_id, max_offset from kafka_table_offsets; + +-- 10 rows +select count(*) from orc_kafka_table; + +-- no duplicate or missing data +select `partition_id`, `row_offset`,`__time`, `page`, `user`, `language`, `country`,`continent`, `namespace`, `newPage` , +`unpatrolled` , `anonymous` , `robot` , added , deleted , delta from orc_kafka_table; + +-- LEFT OUTER JOIN if metadata is empty + +Drop table kafka_table_offsets; +create table kafka_table_offsets(partition_id int, max_offset bigint, insert_time timestamp); + +Drop table orc_kafka_table; +Create table orc_kafka_table (partition_id int, row_offset bigint, kafka_ts bigint, + `__time` timestamp , `page` string, `user` string, `language` string, +`country` string,`continent` string, `namespace` string, `newPage` boolean, `unpatrolled` boolean, +`anonymous` boolean, `robot` boolean, added int, deleted int, delta bigint +) stored as ORC; + + +From kafka_table ktable LEFT OUTER 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`, +`__time`, `page`, `user`, `language`, `country`,`continent`, `namespace`, `newPage` , +`unpatrolled` , `anonymous` , `robot` , added , deleted , delta +Insert overwrite table kafka_table_offsets select +`__partition`, max(`__offset`), CURRENT_TIMESTAMP group by `__partition`, CURRENT_TIMESTAMP; + +select count(*) from orc_kafka_table; + +select partition_id, max_offset from kafka_table_offsets; + +select `partition_id`, `row_offset`,`__time`, `page`, `user`, `language`, `country`,`continent`, `namespace`, `newPage` , +`unpatrolled` , `anonymous` , `robot` , added , deleted , delta from orc_kafka_table; + +-- using basic implementation of flat json probably to be removed CREATE EXTERNAL TABLE kafka_table_2 (`__time` timestamp with local time zone , `page` string, `user` string, `language` string, `country` string,`continent` string, `namespace` string, `newPage` boolean, `unpatrolled` boolean, diff --git a/ql/src/test/results/clientpositive/druid/kafka_storage_handler.q.out b/ql/src/test/results/clientpositive/druid/kafka_storage_handler.q.out index a936a26aa7b5..e5a15c2634e0 100644 --- a/ql/src/test/results/clientpositive/druid/kafka_storage_handler.q.out +++ b/ql/src/test/results/clientpositive/druid/kafka_storage_handler.q.out @@ -141,6 +141,364 @@ POSTHOOK: Output: hdfs://### HDFS PATH ### 0 4 NULL Coyote Tango stringer ja Japan Asia wikipedia false true false true 1 10 -9 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 where `__partition` = 1 +PREHOOK: type: QUERY +PREHOOK: Input: default@kafka_table +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: Select count(*) from kafka_table where `__partition` = 1 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@kafka_table +POSTHOOK: Output: hdfs://### HDFS PATH ### +0 +PREHOOK: query: Select count(*) from kafka_table where `__offset` = 100 +PREHOOK: type: QUERY +PREHOOK: Input: default@kafka_table +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: Select count(*) from kafka_table where `__offset` = 100 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@kafka_table +POSTHOOK: Output: hdfs://### HDFS PATH ### +0 +PREHOOK: query: Select count(*) from kafka_table where `__offset` <= 100 and `__partition` <= 100 +PREHOOK: type: QUERY +PREHOOK: Input: default@kafka_table +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: Select count(*) from kafka_table where `__offset` <= 100 and `__partition` <= 100 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@kafka_table +POSTHOOK: Output: hdfs://### HDFS PATH ### +10 +PREHOOK: query: Drop table kafka_table_offsets +PREHOOK: type: DROPTABLE +POSTHOOK: query: Drop table kafka_table_offsets +POSTHOOK: type: DROPTABLE +PREHOOK: query: create table kafka_table_offsets(partition_id int, max_offset bigint, insert_time timestamp) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@kafka_table_offsets +POSTHOOK: query: create table kafka_table_offsets(partition_id int, max_offset bigint, insert_time timestamp) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@kafka_table_offsets +PREHOOK: query: insert overwrite table kafka_table_offsets select `__partition`, min(`__offset`) - 1, CURRENT_TIMESTAMP from kafka_table group by `__partition`, CURRENT_TIMESTAMP +PREHOOK: type: QUERY +PREHOOK: Input: default@kafka_table +PREHOOK: Output: default@kafka_table_offsets +POSTHOOK: query: insert overwrite table kafka_table_offsets select `__partition`, min(`__offset`) - 1, CURRENT_TIMESTAMP from kafka_table group by `__partition`, CURRENT_TIMESTAMP +POSTHOOK: type: QUERY +POSTHOOK: Input: default@kafka_table +POSTHOOK: Output: default@kafka_table_offsets +POSTHOOK: Lineage: kafka_table_offsets.insert_time SIMPLE [] +POSTHOOK: Lineage: kafka_table_offsets.max_offset EXPRESSION [(kafka_table)kafka_table.FieldSchema(name:__offset, type:bigint, comment:from deserializer), ] +POSTHOOK: Lineage: kafka_table_offsets.partition_id SIMPLE [(kafka_table)kafka_table.FieldSchema(name:__partition, type:int, comment:from deserializer), ] +PREHOOK: query: select partition_id, max_offset from kafka_table_offsets +PREHOOK: type: QUERY +PREHOOK: Input: default@kafka_table_offsets +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select partition_id, max_offset from kafka_table_offsets +POSTHOOK: type: QUERY +POSTHOOK: Input: default@kafka_table_offsets +POSTHOOK: Output: hdfs://### HDFS PATH ### +0 -1 +PREHOOK: query: Drop table orc_kafka_table +PREHOOK: type: DROPTABLE +POSTHOOK: query: Drop table orc_kafka_table +POSTHOOK: type: DROPTABLE +PREHOOK: query: Create table orc_kafka_table (partition_id int, row_offset bigint, kafka_ts bigint, + `__time` timestamp , `page` string, `user` string, `language` string, +`country` string,`continent` string, `namespace` string, `newPage` boolean, `unpatrolled` boolean, +`anonymous` boolean, `robot` boolean, added int, deleted int, delta bigint +) stored as ORC +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@orc_kafka_table +POSTHOOK: query: Create table orc_kafka_table (partition_id int, row_offset bigint, kafka_ts bigint, + `__time` timestamp , `page` string, `user` string, `language` string, +`country` string,`continent` string, `namespace` string, `newPage` boolean, `unpatrolled` boolean, +`anonymous` boolean, `robot` boolean, added int, deleted int, delta bigint +) stored as ORC +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@orc_kafka_table +PREHOOK: query: From kafka_table 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`, +`__time`, `page`, `user`, `language`, `country`,`continent`, `namespace`, `newPage` , +`unpatrolled` , `anonymous` , `robot` , added , deleted , delta +Insert overwrite table kafka_table_offsets select +`__partition`, max(`__offset`), CURRENT_TIMESTAMP group by `__partition`, CURRENT_TIMESTAMP +PREHOOK: type: QUERY +PREHOOK: Input: default@kafka_table +PREHOOK: Input: default@kafka_table_offsets +PREHOOK: Output: default@kafka_table_offsets +PREHOOK: Output: default@orc_kafka_table +POSTHOOK: query: From kafka_table 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`, +`__time`, `page`, `user`, `language`, `country`,`continent`, `namespace`, `newPage` , +`unpatrolled` , `anonymous` , `robot` , added , deleted , delta +Insert overwrite table kafka_table_offsets select +`__partition`, max(`__offset`), CURRENT_TIMESTAMP group by `__partition`, CURRENT_TIMESTAMP +POSTHOOK: type: QUERY +POSTHOOK: Input: default@kafka_table +POSTHOOK: Input: default@kafka_table_offsets +POSTHOOK: Output: default@kafka_table_offsets +POSTHOOK: Output: default@orc_kafka_table +POSTHOOK: Lineage: kafka_table_offsets.insert_time EXPRESSION [] +POSTHOOK: Lineage: kafka_table_offsets.max_offset EXPRESSION [(kafka_table)ktable.FieldSchema(name:__offset, type:bigint, comment:from deserializer), ] +POSTHOOK: Lineage: kafka_table_offsets.partition_id SIMPLE [(kafka_table)ktable.FieldSchema(name:__partition, type:int, comment:from deserializer), ] +POSTHOOK: Lineage: orc_kafka_table.__time SIMPLE [(kafka_table)ktable.FieldSchema(name:__time, type:timestamp, comment:from deserializer), ] +POSTHOOK: Lineage: orc_kafka_table.added SIMPLE [(kafka_table)ktable.FieldSchema(name:added, type:int, comment:from deserializer), ] +POSTHOOK: Lineage: orc_kafka_table.anonymous SIMPLE [(kafka_table)ktable.FieldSchema(name:anonymous, type:boolean, comment:from deserializer), ] +POSTHOOK: Lineage: orc_kafka_table.continent SIMPLE [(kafka_table)ktable.FieldSchema(name:continent, type:string, comment:from deserializer), ] +POSTHOOK: Lineage: orc_kafka_table.country SIMPLE [(kafka_table)ktable.FieldSchema(name:country, type:string, comment:from deserializer), ] +POSTHOOK: Lineage: orc_kafka_table.deleted SIMPLE [(kafka_table)ktable.FieldSchema(name:deleted, type:int, comment:from deserializer), ] +POSTHOOK: Lineage: orc_kafka_table.delta SIMPLE [(kafka_table)ktable.FieldSchema(name:delta, type:bigint, comment:from deserializer), ] +POSTHOOK: Lineage: orc_kafka_table.kafka_ts SIMPLE [(kafka_table)ktable.FieldSchema(name:__timestamp, type:bigint, comment:from deserializer), ] +POSTHOOK: Lineage: orc_kafka_table.language SIMPLE [(kafka_table)ktable.FieldSchema(name:language, type:string, comment:from deserializer), ] +POSTHOOK: Lineage: orc_kafka_table.namespace SIMPLE [(kafka_table)ktable.FieldSchema(name:namespace, type:string, comment:from deserializer), ] +POSTHOOK: Lineage: orc_kafka_table.newpage SIMPLE [(kafka_table)ktable.FieldSchema(name:newpage, type:boolean, comment:from deserializer), ] +POSTHOOK: Lineage: orc_kafka_table.page SIMPLE [(kafka_table)ktable.FieldSchema(name:page, type:string, comment:from deserializer), ] +POSTHOOK: Lineage: orc_kafka_table.partition_id SIMPLE [(kafka_table)ktable.FieldSchema(name:__partition, type:int, comment:from deserializer), ] +POSTHOOK: Lineage: orc_kafka_table.robot SIMPLE [(kafka_table)ktable.FieldSchema(name:robot, type:boolean, comment:from deserializer), ] +POSTHOOK: Lineage: orc_kafka_table.row_offset SIMPLE [(kafka_table)ktable.FieldSchema(name:__offset, type:bigint, comment:from deserializer), ] +POSTHOOK: Lineage: orc_kafka_table.unpatrolled SIMPLE [(kafka_table)ktable.FieldSchema(name:unpatrolled, type:boolean, comment:from deserializer), ] +POSTHOOK: Lineage: orc_kafka_table.user SIMPLE [(kafka_table)ktable.FieldSchema(name:user, type:string, comment:from deserializer), ] +PREHOOK: query: select count(*) from orc_kafka_table +PREHOOK: type: QUERY +PREHOOK: Input: default@orc_kafka_table +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select count(*) from orc_kafka_table +POSTHOOK: type: QUERY +POSTHOOK: Input: default@orc_kafka_table +POSTHOOK: Output: hdfs://### HDFS PATH ### +3 +PREHOOK: query: select partition_id, max_offset from kafka_table_offsets +PREHOOK: type: QUERY +PREHOOK: Input: default@kafka_table_offsets +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select partition_id, max_offset from kafka_table_offsets +POSTHOOK: type: QUERY +POSTHOOK: Input: default@kafka_table_offsets +POSTHOOK: Output: hdfs://### HDFS PATH ### +0 2 +PREHOOK: query: select `partition_id`, `row_offset`,`__time`, `page`, `user`, `language`, `country`,`continent`, `namespace`, `newPage` , +`unpatrolled` , `anonymous` , `robot` , added , deleted , delta from orc_kafka_table +PREHOOK: type: QUERY +PREHOOK: Input: default@orc_kafka_table +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select `partition_id`, `row_offset`,`__time`, `page`, `user`, `language`, `country`,`continent`, `namespace`, `newPage` , +`unpatrolled` , `anonymous` , `robot` , added , deleted , delta from orc_kafka_table +POSTHOOK: type: QUERY +POSTHOOK: Input: default@orc_kafka_table +POSTHOOK: Output: hdfs://### HDFS PATH ### +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 +PREHOOK: query: From kafka_table 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`, +`__time`, `page`, `user`, `language`, `country`,`continent`, `namespace`, `newPage` , +`unpatrolled` , `anonymous` , `robot` , added , deleted , delta +Insert overwrite table kafka_table_offsets select +`__partition`, max(`__offset`), CURRENT_TIMESTAMP group by `__partition`, CURRENT_TIMESTAMP +PREHOOK: type: QUERY +PREHOOK: Input: default@kafka_table +PREHOOK: Input: default@kafka_table_offsets +PREHOOK: Output: default@kafka_table_offsets +PREHOOK: Output: default@orc_kafka_table +POSTHOOK: query: From kafka_table 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`, +`__time`, `page`, `user`, `language`, `country`,`continent`, `namespace`, `newPage` , +`unpatrolled` , `anonymous` , `robot` , added , deleted , delta +Insert overwrite table kafka_table_offsets select +`__partition`, max(`__offset`), CURRENT_TIMESTAMP group by `__partition`, CURRENT_TIMESTAMP +POSTHOOK: type: QUERY +POSTHOOK: Input: default@kafka_table +POSTHOOK: Input: default@kafka_table_offsets +POSTHOOK: Output: default@kafka_table_offsets +POSTHOOK: Output: default@orc_kafka_table +POSTHOOK: Lineage: kafka_table_offsets.insert_time EXPRESSION [] +POSTHOOK: Lineage: kafka_table_offsets.max_offset EXPRESSION [(kafka_table)ktable.FieldSchema(name:__offset, type:bigint, comment:from deserializer), ] +POSTHOOK: Lineage: kafka_table_offsets.partition_id SIMPLE [(kafka_table)ktable.FieldSchema(name:__partition, type:int, comment:from deserializer), ] +POSTHOOK: Lineage: orc_kafka_table.__time SIMPLE [(kafka_table)ktable.FieldSchema(name:__time, type:timestamp, comment:from deserializer), ] +POSTHOOK: Lineage: orc_kafka_table.added SIMPLE [(kafka_table)ktable.FieldSchema(name:added, type:int, comment:from deserializer), ] +POSTHOOK: Lineage: orc_kafka_table.anonymous SIMPLE [(kafka_table)ktable.FieldSchema(name:anonymous, type:boolean, comment:from deserializer), ] +POSTHOOK: Lineage: orc_kafka_table.continent SIMPLE [(kafka_table)ktable.FieldSchema(name:continent, type:string, comment:from deserializer), ] +POSTHOOK: Lineage: orc_kafka_table.country SIMPLE [(kafka_table)ktable.FieldSchema(name:country, type:string, comment:from deserializer), ] +POSTHOOK: Lineage: orc_kafka_table.deleted SIMPLE [(kafka_table)ktable.FieldSchema(name:deleted, type:int, comment:from deserializer), ] +POSTHOOK: Lineage: orc_kafka_table.delta SIMPLE [(kafka_table)ktable.FieldSchema(name:delta, type:bigint, comment:from deserializer), ] +POSTHOOK: Lineage: orc_kafka_table.kafka_ts SIMPLE [(kafka_table)ktable.FieldSchema(name:__timestamp, type:bigint, comment:from deserializer), ] +POSTHOOK: Lineage: orc_kafka_table.language SIMPLE [(kafka_table)ktable.FieldSchema(name:language, type:string, comment:from deserializer), ] +POSTHOOK: Lineage: orc_kafka_table.namespace SIMPLE [(kafka_table)ktable.FieldSchema(name:namespace, type:string, comment:from deserializer), ] +POSTHOOK: Lineage: orc_kafka_table.newpage SIMPLE [(kafka_table)ktable.FieldSchema(name:newpage, type:boolean, comment:from deserializer), ] +POSTHOOK: Lineage: orc_kafka_table.page SIMPLE [(kafka_table)ktable.FieldSchema(name:page, type:string, comment:from deserializer), ] +POSTHOOK: Lineage: orc_kafka_table.partition_id SIMPLE [(kafka_table)ktable.FieldSchema(name:__partition, type:int, comment:from deserializer), ] +POSTHOOK: Lineage: orc_kafka_table.robot SIMPLE [(kafka_table)ktable.FieldSchema(name:robot, type:boolean, comment:from deserializer), ] +POSTHOOK: Lineage: orc_kafka_table.row_offset SIMPLE [(kafka_table)ktable.FieldSchema(name:__offset, type:bigint, comment:from deserializer), ] +POSTHOOK: Lineage: orc_kafka_table.unpatrolled SIMPLE [(kafka_table)ktable.FieldSchema(name:unpatrolled, type:boolean, comment:from deserializer), ] +POSTHOOK: Lineage: orc_kafka_table.user SIMPLE [(kafka_table)ktable.FieldSchema(name:user, type:string, comment:from deserializer), ] +PREHOOK: query: select partition_id, max_offset from kafka_table_offsets +PREHOOK: type: QUERY +PREHOOK: Input: default@kafka_table_offsets +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select partition_id, max_offset from kafka_table_offsets +POSTHOOK: type: QUERY +POSTHOOK: Input: default@kafka_table_offsets +POSTHOOK: Output: hdfs://### HDFS PATH ### +0 9 +PREHOOK: query: select count(*) from orc_kafka_table +PREHOOK: type: QUERY +PREHOOK: Input: default@orc_kafka_table +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select count(*) from orc_kafka_table +POSTHOOK: type: QUERY +POSTHOOK: Input: default@orc_kafka_table +POSTHOOK: Output: hdfs://### HDFS PATH ### +10 +PREHOOK: query: select `partition_id`, `row_offset`,`__time`, `page`, `user`, `language`, `country`,`continent`, `namespace`, `newPage` , +`unpatrolled` , `anonymous` , `robot` , added , deleted , delta from orc_kafka_table +PREHOOK: type: QUERY +PREHOOK: Input: default@orc_kafka_table +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select `partition_id`, `row_offset`,`__time`, `page`, `user`, `language`, `country`,`continent`, `namespace`, `newPage` , +`unpatrolled` , `anonymous` , `robot` , added , deleted , delta from orc_kafka_table +POSTHOOK: type: QUERY +POSTHOOK: Input: default@orc_kafka_table +POSTHOOK: Output: hdfs://### HDFS PATH ### +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: Drop table kafka_table_offsets +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@kafka_table_offsets +PREHOOK: Output: default@kafka_table_offsets +POSTHOOK: query: Drop table kafka_table_offsets +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@kafka_table_offsets +POSTHOOK: Output: default@kafka_table_offsets +PREHOOK: query: create table kafka_table_offsets(partition_id int, max_offset bigint, insert_time timestamp) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@kafka_table_offsets +POSTHOOK: query: create table kafka_table_offsets(partition_id int, max_offset bigint, insert_time timestamp) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@kafka_table_offsets +PREHOOK: query: Drop table orc_kafka_table +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@orc_kafka_table +PREHOOK: Output: default@orc_kafka_table +POSTHOOK: query: Drop table orc_kafka_table +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@orc_kafka_table +POSTHOOK: Output: default@orc_kafka_table +PREHOOK: query: Create table orc_kafka_table (partition_id int, row_offset bigint, kafka_ts bigint, + `__time` timestamp , `page` string, `user` string, `language` string, +`country` string,`continent` string, `namespace` string, `newPage` boolean, `unpatrolled` boolean, +`anonymous` boolean, `robot` boolean, added int, deleted int, delta bigint +) stored as ORC +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@orc_kafka_table +POSTHOOK: query: Create table orc_kafka_table (partition_id int, row_offset bigint, kafka_ts bigint, + `__time` timestamp , `page` string, `user` string, `language` string, +`country` string,`continent` string, `namespace` string, `newPage` boolean, `unpatrolled` boolean, +`anonymous` boolean, `robot` boolean, added int, deleted int, delta bigint +) stored as ORC +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@orc_kafka_table +PREHOOK: query: From kafka_table ktable LEFT OUTER 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`, +`__time`, `page`, `user`, `language`, `country`,`continent`, `namespace`, `newPage` , +`unpatrolled` , `anonymous` , `robot` , added , deleted , delta +Insert overwrite table kafka_table_offsets select +`__partition`, max(`__offset`), CURRENT_TIMESTAMP group by `__partition`, CURRENT_TIMESTAMP +PREHOOK: type: QUERY +PREHOOK: Input: default@kafka_table +PREHOOK: Input: default@kafka_table_offsets +PREHOOK: Output: default@kafka_table_offsets +PREHOOK: Output: default@orc_kafka_table +POSTHOOK: query: From kafka_table ktable LEFT OUTER 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`, +`__time`, `page`, `user`, `language`, `country`,`continent`, `namespace`, `newPage` , +`unpatrolled` , `anonymous` , `robot` , added , deleted , delta +Insert overwrite table kafka_table_offsets select +`__partition`, max(`__offset`), CURRENT_TIMESTAMP group by `__partition`, CURRENT_TIMESTAMP +POSTHOOK: type: QUERY +POSTHOOK: Input: default@kafka_table +POSTHOOK: Input: default@kafka_table_offsets +POSTHOOK: Output: default@kafka_table_offsets +POSTHOOK: Output: default@orc_kafka_table +POSTHOOK: Lineage: kafka_table_offsets.insert_time EXPRESSION [] +POSTHOOK: Lineage: kafka_table_offsets.max_offset EXPRESSION [(kafka_table)ktable.FieldSchema(name:__offset, type:bigint, comment:from deserializer), ] +POSTHOOK: Lineage: kafka_table_offsets.partition_id SIMPLE [(kafka_table)ktable.FieldSchema(name:__partition, type:int, comment:from deserializer), ] +POSTHOOK: Lineage: orc_kafka_table.__time SIMPLE [(kafka_table)ktable.FieldSchema(name:__time, type:timestamp, comment:from deserializer), ] +POSTHOOK: Lineage: orc_kafka_table.added SIMPLE [(kafka_table)ktable.FieldSchema(name:added, type:int, comment:from deserializer), ] +POSTHOOK: Lineage: orc_kafka_table.anonymous SIMPLE [(kafka_table)ktable.FieldSchema(name:anonymous, type:boolean, comment:from deserializer), ] +POSTHOOK: Lineage: orc_kafka_table.continent SIMPLE [(kafka_table)ktable.FieldSchema(name:continent, type:string, comment:from deserializer), ] +POSTHOOK: Lineage: orc_kafka_table.country SIMPLE [(kafka_table)ktable.FieldSchema(name:country, type:string, comment:from deserializer), ] +POSTHOOK: Lineage: orc_kafka_table.deleted SIMPLE [(kafka_table)ktable.FieldSchema(name:deleted, type:int, comment:from deserializer), ] +POSTHOOK: Lineage: orc_kafka_table.delta SIMPLE [(kafka_table)ktable.FieldSchema(name:delta, type:bigint, comment:from deserializer), ] +POSTHOOK: Lineage: orc_kafka_table.kafka_ts SIMPLE [(kafka_table)ktable.FieldSchema(name:__timestamp, type:bigint, comment:from deserializer), ] +POSTHOOK: Lineage: orc_kafka_table.language SIMPLE [(kafka_table)ktable.FieldSchema(name:language, type:string, comment:from deserializer), ] +POSTHOOK: Lineage: orc_kafka_table.namespace SIMPLE [(kafka_table)ktable.FieldSchema(name:namespace, type:string, comment:from deserializer), ] +POSTHOOK: Lineage: orc_kafka_table.newpage SIMPLE [(kafka_table)ktable.FieldSchema(name:newpage, type:boolean, comment:from deserializer), ] +POSTHOOK: Lineage: orc_kafka_table.page SIMPLE [(kafka_table)ktable.FieldSchema(name:page, type:string, comment:from deserializer), ] +POSTHOOK: Lineage: orc_kafka_table.partition_id SIMPLE [(kafka_table)ktable.FieldSchema(name:__partition, type:int, comment:from deserializer), ] +POSTHOOK: Lineage: orc_kafka_table.robot SIMPLE [(kafka_table)ktable.FieldSchema(name:robot, type:boolean, comment:from deserializer), ] +POSTHOOK: Lineage: orc_kafka_table.row_offset SIMPLE [(kafka_table)ktable.FieldSchema(name:__offset, type:bigint, comment:from deserializer), ] +POSTHOOK: Lineage: orc_kafka_table.unpatrolled SIMPLE [(kafka_table)ktable.FieldSchema(name:unpatrolled, type:boolean, comment:from deserializer), ] +POSTHOOK: Lineage: orc_kafka_table.user SIMPLE [(kafka_table)ktable.FieldSchema(name:user, type:string, comment:from deserializer), ] +PREHOOK: query: select count(*) from orc_kafka_table +PREHOOK: type: QUERY +PREHOOK: Input: default@orc_kafka_table +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select count(*) from orc_kafka_table +POSTHOOK: type: QUERY +POSTHOOK: Input: default@orc_kafka_table +POSTHOOK: Output: hdfs://### HDFS PATH ### +10 +PREHOOK: query: select partition_id, max_offset from kafka_table_offsets +PREHOOK: type: QUERY +PREHOOK: Input: default@kafka_table_offsets +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select partition_id, max_offset from kafka_table_offsets +POSTHOOK: type: QUERY +POSTHOOK: Input: default@kafka_table_offsets +POSTHOOK: Output: hdfs://### HDFS PATH ### +0 9 +PREHOOK: query: select `partition_id`, `row_offset`,`__time`, `page`, `user`, `language`, `country`,`continent`, `namespace`, `newPage` , +`unpatrolled` , `anonymous` , `robot` , added , deleted , delta from orc_kafka_table +PREHOOK: type: QUERY +PREHOOK: Input: default@orc_kafka_table +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select `partition_id`, `row_offset`,`__time`, `page`, `user`, `language`, `country`,`continent`, `namespace`, `newPage` , +`unpatrolled` , `anonymous` , `robot` , added , deleted , delta from orc_kafka_table +POSTHOOK: type: QUERY +POSTHOOK: Input: default@orc_kafka_table +POSTHOOK: Output: hdfs://### HDFS PATH ### +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: CREATE EXTERNAL TABLE kafka_table_2 (`__time` timestamp with local time zone , `page` string, `user` string, `language` string, `country` string,`continent` string, `namespace` string, `newPage` boolean, `unpatrolled` boolean, From 77700b20b7e2aa604caaad89bcd030dfc8b8e925 Mon Sep 17 00:00:00 2001 From: Slim Bouguerra Date: Mon, 13 Aug 2018 18:01:09 -0700 Subject: [PATCH 08/33] intelliJ friendly warning supress Change-Id: Ib6fb5a8a2fdc2e7cb13dde7f8386bf39b86a8926 --- pom.xml | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 4e1ab9418627..9c15328eccc2 100644 --- a/pom.xml +++ b/pom.xml @@ -81,6 +81,7 @@ + ${maven.test.classpath} file:// ${project.build.directory}/tmp @@ -1481,7 +1482,7 @@ org.apache.maven.plugins maven-javadoc-plugin - false + -Xdoclint:none @@ -1567,6 +1568,7 @@ ${basedir}/${hive.path.to.root}/testutils/hadoop.cmd + ;${env.HADOOP_HOME}/bin ${project.build.directory}/deplibs/* file:///${test.tmp.dir} From 8cfbce6e3cd70a472ebb818f126d72596b39bd91 Mon Sep 17 00:00:00 2001 From: Slim Bouguerra Date: Tue, 14 Aug 2018 13:29:04 -0700 Subject: [PATCH 09/33] fix style issue Change-Id: I36be0353de253e46fbe16e35b52c258c7784a63a --- .../hive/cli/TestMiniDruidKafkaCliDriver.java | 1 - .../hadoop/hive/kafka/GenericKafkaSerDe.java | 106 +-- .../hadoop/hive/kafka/KafkaJsonSerDe.java | 184 ++-- .../hive/kafka/KafkaPullerInputFormat.java | 150 ++- .../hive/kafka/KafkaPullerInputSplit.java | 134 ++- .../hive/kafka/KafkaPullerRecordReader.java | 90 +- .../hive/kafka/KafkaRecordIterator.java | 124 +-- .../hive/kafka/KafkaRecordWritable.java | 19 +- .../hadoop/hive/kafka/KafkaScanTrimmer.java | 311 +++---- .../hive/kafka/KafkaStorageHandler.java | 142 +-- .../hive/kafka/KafkaStreamingUtils.java | 61 +- .../hadoop/hive/kafka/package-info.java | 23 + .../hive/kafka/KafkaPullerInputSplitTest.java | 192 ++-- .../hive/kafka/KafkaRecordIteratorTest.java | 244 ++--- .../hive/kafka/KafkaRecordWritableTest.java | 6 +- .../hive/kafka/KafkaScanTrimmerTest.java | 853 +++++++----------- .../hive/kafka/KafkaStreamingUtilsTest.java | 6 +- .../hadoop/hive/kafka/package-info.java | 23 + .../hive/llap/cli/LlapServiceDriver.java | 87 +- 19 files changed, 1140 insertions(+), 1616 deletions(-) create mode 100644 kafka-handler/src/java/org/apache/hadoop/hive/kafka/package-info.java create mode 100644 kafka-handler/src/test/org/apache/hadoop/hive/kafka/package-info.java diff --git a/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestMiniDruidKafkaCliDriver.java b/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestMiniDruidKafkaCliDriver.java index b526c29cee09..4768975225dd 100644 --- a/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestMiniDruidKafkaCliDriver.java +++ b/itests/qtest/src/test/java/org/apache/hadoop/hive/cli/TestMiniDruidKafkaCliDriver.java @@ -21,7 +21,6 @@ import org.apache.hadoop.hive.cli.control.CliConfigs; import org.junit.ClassRule; -import org.junit.Ignore; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TestRule; diff --git a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/GenericKafkaSerDe.java b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/GenericKafkaSerDe.java index 827b4475e275..a8ccbc1733a9 100644 --- a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/GenericKafkaSerDe.java +++ b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/GenericKafkaSerDe.java @@ -52,39 +52,37 @@ import java.util.List; import java.util.Properties; import java.util.stream.Collectors; -; -public class GenericKafkaSerDe extends AbstractSerDe -{ - private static final Logger log = LoggerFactory.getLogger(GenericKafkaSerDe.class); +/** + * Generic Kafka Serde that allow user to delegate Serde to other class like Avro, Json or any class that supports + * {@link BytesRefWritable}. + */ +public class GenericKafkaSerDe extends AbstractSerDe { + private static final Logger LOG = LoggerFactory.getLogger(GenericKafkaSerDe.class); // ORDER of fields and types matters here - public static final ImmutableList METADATA_COLUMN_NAMES = ImmutableList.of( - KafkaStorageHandler.__PARTITION, - KafkaStorageHandler.__OFFSET, - KafkaStorageHandler.__TIMESTAMP - ); - public static final ImmutableList METADATA_PRIMITIVE_TYPE_INFO = ImmutableList.of( - TypeInfoFactory.intTypeInfo, - TypeInfoFactory.longTypeInfo, - TypeInfoFactory.longTypeInfo - ); + public static final ImmutableList + METADATA_COLUMN_NAMES = + ImmutableList.of(KafkaStorageHandler.PARTITION_COLUMN, + KafkaStorageHandler.OFFSET_COLUMN, + KafkaStorageHandler.TIMESTAMP_COLUMN); + public static final ImmutableList + METADATA_PRIMITIVE_TYPE_INFO = + ImmutableList.of(TypeInfoFactory.intTypeInfo, TypeInfoFactory.longTypeInfo, TypeInfoFactory.longTypeInfo); private AbstractSerDe delegateSerDe; private ObjectInspector objectInspector; private final List columnNames = Lists.newArrayList(); StructObjectInspector delegateObjectInspector; - @Override - public void initialize(@Nullable Configuration conf, Properties tbl) throws SerDeException - { + @Override public void initialize(@Nullable Configuration conf, Properties tbl) throws SerDeException { final String className = tbl.getProperty(KafkaStorageHandler.SERDE_CLASS_NAME, KafkaJsonSerDe.class.getName()); delegateSerDe = createDelegate(className); delegateSerDe.initialize(conf, tbl); - log.info("Using SerDe instance {}", delegateSerDe.getClass().getCanonicalName()); + LOG.info("Using SerDe instance {}", delegateSerDe.getClass().getCanonicalName()); if (!(delegateSerDe.getObjectInspector() instanceof StructObjectInspector)) { throw new SerDeException("Was expecting StructObject Inspector but have " + delegateSerDe.getObjectInspector() - .getClass() - .getName()); + .getClass() + .getName()); } delegateObjectInspector = (StructObjectInspector) delegateSerDe.getObjectInspector(); @@ -92,8 +90,11 @@ public void initialize(@Nullable Configuration conf, Properties tbl) throws SerD final List inspectors; // Get column names and types String columnNameProperty = tbl.getProperty(serdeConstants.LIST_COLUMNS); - final String columnNameDelimiter = tbl.containsKey(serdeConstants.COLUMN_NAME_DELIMITER) ? tbl - .getProperty(serdeConstants.COLUMN_NAME_DELIMITER) : String.valueOf(SerDeUtils.COMMA); + final String + columnNameDelimiter = + tbl.containsKey(serdeConstants.COLUMN_NAME_DELIMITER) ? + tbl.getProperty(serdeConstants.COLUMN_NAME_DELIMITER) : + String.valueOf(SerDeUtils.COMMA); // all table column names if (!columnNameProperty.isEmpty()) { columnNames.addAll(Arrays.asList(columnNameProperty.split(columnNameDelimiter))); @@ -101,57 +102,47 @@ public void initialize(@Nullable Configuration conf, Properties tbl) throws SerD columnNames.addAll(METADATA_COLUMN_NAMES); - if (log.isDebugEnabled()) { - log.debug("columns: {}, {}", columnNameProperty, columnNames); + if (LOG.isDebugEnabled()) { + LOG.debug("columns: {}, {}", columnNameProperty, columnNames); } inspectors = new ArrayList<>(columnNames.size()); inspectors.addAll(delegateObjectInspector.getAllStructFieldRefs() - .stream() - .map(structField -> structField.getFieldObjectInspector()) - .collect(Collectors.toList())); + .stream() + .map(structField -> structField.getFieldObjectInspector()) + .collect(Collectors.toList())); inspectors.addAll(METADATA_PRIMITIVE_TYPE_INFO.stream() - .map(KafkaJsonSerDe.typeInfoToObjectInspector) - .collect(Collectors.toList())); + .map(KafkaJsonSerDe.typeInfoToObjectInspector) + .collect(Collectors.toList())); objectInspector = ObjectInspectorFactory.getStandardStructObjectInspector(columnNames, inspectors); } - private AbstractSerDe createDelegate(String className) - { + private AbstractSerDe createDelegate(String className) { final Class clazz; try { clazz = (Class) Class.forName(className); - } - catch (ClassNotFoundException e) { - log.error("Failed a loading delegate SerDe {}", className); + } catch (ClassNotFoundException e) { + LOG.error("Failed a loading delegate SerDe {}", className); throw new RuntimeException(e); } // we are not setting conf thus null is okay return ReflectionUtil.newInstance(clazz, null); } - @Override - public Class getSerializedClass() - { + @Override public Class getSerializedClass() { return delegateSerDe.getSerializedClass(); } - @Override - public Writable serialize(Object obj, ObjectInspector objInspector) throws SerDeException - { + @Override public Writable serialize(Object obj, ObjectInspector objInspector) throws SerDeException { return delegateSerDe.serialize(obj, objInspector); } - @Override - public SerDeStats getSerDeStats() - { + @Override public SerDeStats getSerDeStats() { return delegateSerDe.getSerDeStats(); } - @Override - public Object deserialize(Writable blob) throws SerDeException - { + @Override public Object deserialize(Writable blob) throws SerDeException { KafkaRecordWritable record = (KafkaRecordWritable) blob; // switch case the serde nature final Object row; @@ -163,8 +154,7 @@ public Object deserialize(Writable blob) throws SerDeException try { ByteArrayInputStream byteArrayInputStream = new ByteArrayInputStream(record.getValue()); avroGenericRecordWritable.readFields(new DataInputStream(byteArrayInputStream)); - } - catch (IOException e) { + } catch (IOException e) { throw new SerDeException(e); } row = delegateSerDe.deserialize(avroGenericRecordWritable); @@ -175,21 +165,19 @@ public Object deserialize(Writable blob) throws SerDeException return columnNames.stream().map(name -> { switch (name) { - case KafkaStorageHandler.__PARTITION: - return new IntWritable(record.getPartition()); - case KafkaStorageHandler.__OFFSET: - return new LongWritable(record.getOffset()); - case KafkaStorageHandler.__TIMESTAMP: - return new LongWritable(record.getTimestamp()); - default: - return delegateObjectInspector.getStructFieldData(row, delegateObjectInspector.getStructFieldRef(name)); + case KafkaStorageHandler.PARTITION_COLUMN: + return new IntWritable(record.getPartition()); + case KafkaStorageHandler.OFFSET_COLUMN: + return new LongWritable(record.getOffset()); + case KafkaStorageHandler.TIMESTAMP_COLUMN: + return new LongWritable(record.getTimestamp()); + default: + return delegateObjectInspector.getStructFieldData(row, delegateObjectInspector.getStructFieldRef(name)); } }).collect(Collectors.toList()); } - @Override - public ObjectInspector getObjectInspector() - { + @Override public ObjectInspector getObjectInspector() { return objectInspector; } } diff --git a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaJsonSerDe.java b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaJsonSerDe.java index 3a2d7d77d1f1..b44aceab44a5 100644 --- a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaJsonSerDe.java +++ b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaJsonSerDe.java @@ -7,7 +7,7 @@ * "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 + * 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, @@ -73,12 +73,15 @@ import java.util.function.Function; import java.util.stream.Collectors; -public class KafkaJsonSerDe extends AbstractSerDe -{ +/** + * Basic JsonSerde to make use of such storage handler smooth and easy. For production please use Hive native JsonSerde + */ +public class KafkaJsonSerDe extends AbstractSerDe { private static final Logger LOG = LoggerFactory.getLogger(KafkaJsonSerDe.class); private static final DateTimeFormatter TS_PARSER = createAutoParser(); - public static Function typeInfoToObjectInspector = typeInfo -> PrimitiveObjectInspectorFactory - .getPrimitiveWritableObjectInspector( + protected static Function + typeInfoToObjectInspector = typeInfo -> + PrimitiveObjectInspectorFactory.getPrimitiveWritableObjectInspector( TypeInfoFactory.getPrimitiveTypeInfo(typeInfo.getTypeName())); private List columnNames; private List columnTypes; @@ -87,18 +90,16 @@ public class KafkaJsonSerDe extends AbstractSerDe private long rowCount = 0L; private long rawDataSize = 0L; - - @Override - public void initialize( - @Nullable Configuration conf, Properties tbl - ) - { + @Override public void initialize(@Nullable Configuration conf, Properties tbl) { final List inspectors; // Get column names and types String columnNameProperty = tbl.getProperty(serdeConstants.LIST_COLUMNS); String columnTypeProperty = tbl.getProperty(serdeConstants.LIST_COLUMN_TYPES); - final String columnNameDelimiter = tbl.containsKey(serdeConstants.COLUMN_NAME_DELIMITER) ? tbl - .getProperty(serdeConstants.COLUMN_NAME_DELIMITER) : String.valueOf(SerDeUtils.COMMA); + final String + columnNameDelimiter = + tbl.containsKey(serdeConstants.COLUMN_NAME_DELIMITER) ? + tbl.getProperty(serdeConstants.COLUMN_NAME_DELIMITER) : + String.valueOf(SerDeUtils.COMMA); // all table column names if (!columnNameProperty.isEmpty()) { columnNames = Arrays.asList(columnNameProperty.split(columnNameDelimiter)); @@ -117,40 +118,29 @@ public void initialize( inspector = ObjectInspectorFactory.getStandardStructObjectInspector(columnNames, inspectors); } - @Override - public Class getSerializedClass() - { + @Override public Class getSerializedClass() { return BytesRefWritable.class; } - @Override - public Writable serialize( - Object obj, ObjectInspector objInspector - ) throws SerDeException - { + @Override public Writable serialize(Object obj, ObjectInspector objInspector) throws SerDeException { throw new SerDeException("unimplemented"); } - @Override - public SerDeStats getSerDeStats() - { + @Override public SerDeStats getSerDeStats() { SerDeStats serDeStats = new SerDeStats(); serDeStats.setRawDataSize(rawDataSize); serDeStats.setRowCount(rowCount); return serDeStats; } - @Override - public Object deserialize(Writable blob) throws SerDeException - { + @Override public Object deserialize(Writable blob) throws SerDeException { BytesRefWritable record = (BytesRefWritable) blob; Map payload; try { payload = parseAsJson(record.getData()); rowCount += 1; rawDataSize += record.getData().length; - } - catch (IOException e) { + } catch (IOException e) { throw new SerDeException(e); } @@ -164,14 +154,15 @@ public Object deserialize(Writable blob) throws SerDeException output.add(null); } else { switch (columnTypes.get(i).getCategory()) { - case PRIMITIVE: - output.add(parseAsPrimitive(value, typeInfo)); - break; - case MAP: - case LIST: - case UNION: - case STRUCT: - throw new SerDeException("not supported yet"); + case PRIMITIVE: + output.add(parseAsPrimitive(value, typeInfo)); + break; + case MAP: + case LIST: + case UNION: + case STRUCT: + default: + throw new SerDeException("not supported yet"); } } @@ -179,91 +170,76 @@ public Object deserialize(Writable blob) throws SerDeException return output; } - private Object parseAsPrimitive(JsonNode value, TypeInfo typeInfo) throws SerDeException - { - switch (TypeInfoFactory.getPrimitiveTypeInfo(typeInfo.getTypeName()) - .getPrimitiveCategory()) { - case TIMESTAMP: - TimestampWritable timestampWritable = new TimestampWritable(); - timestampWritable.setTime(TS_PARSER.parseMillis(value.textValue())); - return timestampWritable; - - case TIMESTAMPLOCALTZ: - final long numberOfMillis = TS_PARSER.parseMillis(value.textValue()); - return new TimestampLocalTZWritable(new TimestampTZ(ZonedDateTime - .ofInstant( - Instant.ofEpochMilli(numberOfMillis), - ((TimestampLocalTZTypeInfo) typeInfo).timeZone() - ))); - - case BYTE: - return new ByteWritable((byte) value.intValue()); - case SHORT: - return (new ShortWritable(value.shortValue())); - case INT: - return new IntWritable(value.intValue()); - case LONG: - return (new LongWritable((value.longValue()))); - case FLOAT: - return (new FloatWritable(value.floatValue())); - case DOUBLE: - return (new DoubleWritable(value.doubleValue())); - case DECIMAL: - return (new HiveDecimalWritable(HiveDecimal.create(value.decimalValue()))); - case CHAR: - return (new HiveCharWritable( - new HiveChar(value.textValue(), ((CharTypeInfo) typeInfo).getLength()))); - case VARCHAR: - return (new HiveVarcharWritable( - new HiveVarchar(value.textValue(), ((CharTypeInfo) typeInfo).getLength()))); - case STRING: - return (new Text(value.textValue())); - case BOOLEAN: - return (new BooleanWritable(value.isBoolean() ? value.booleanValue() : Boolean.valueOf(value.textValue()))); - default: - throw new SerDeException("Unknown type: " + typeInfo.getTypeName()); + private Object parseAsPrimitive(JsonNode value, TypeInfo typeInfo) throws SerDeException { + switch (TypeInfoFactory.getPrimitiveTypeInfo(typeInfo.getTypeName()).getPrimitiveCategory()) { + case TIMESTAMP: + TimestampWritable timestampWritable = new TimestampWritable(); + timestampWritable.setTime(TS_PARSER.parseMillis(value.textValue())); + return timestampWritable; + + case TIMESTAMPLOCALTZ: + final long numberOfMillis = TS_PARSER.parseMillis(value.textValue()); + return new TimestampLocalTZWritable(new TimestampTZ(ZonedDateTime.ofInstant(Instant.ofEpochMilli(numberOfMillis), + ((TimestampLocalTZTypeInfo) typeInfo).timeZone()))); + + case BYTE: + return new ByteWritable((byte) value.intValue()); + case SHORT: + return (new ShortWritable(value.shortValue())); + case INT: + return new IntWritable(value.intValue()); + case LONG: + return (new LongWritable((value.longValue()))); + case FLOAT: + return (new FloatWritable(value.floatValue())); + case DOUBLE: + return (new DoubleWritable(value.doubleValue())); + case DECIMAL: + return (new HiveDecimalWritable(HiveDecimal.create(value.decimalValue()))); + case CHAR: + return (new HiveCharWritable(new HiveChar(value.textValue(), ((CharTypeInfo) typeInfo).getLength()))); + case VARCHAR: + return (new HiveVarcharWritable(new HiveVarchar(value.textValue(), ((CharTypeInfo) typeInfo).getLength()))); + case STRING: + return (new Text(value.textValue())); + case BOOLEAN: + return (new BooleanWritable(value.isBoolean() ? value.booleanValue() : Boolean.valueOf(value.textValue()))); + default: + throw new SerDeException("Unknown type: " + typeInfo.getTypeName()); } } - private Map parseAsJson(byte[] value) throws IOException - { + private Map parseAsJson(byte[] value) throws IOException { JsonNode document = mapper.readValue(value, JsonNode.class); //Hive Column names are case insensitive. Map documentMap = Maps.newTreeMap(String.CASE_INSENSITIVE_ORDER); - document.fields() - .forEachRemaining(field -> documentMap.put(field.getKey().toLowerCase(), field.getValue())); + document.fields().forEachRemaining(field -> documentMap.put(field.getKey().toLowerCase(), field.getValue())); return documentMap; } - @Override - public ObjectInspector getObjectInspector() throws SerDeException - { + @Override public ObjectInspector getObjectInspector() throws SerDeException { if (inspector == null) { throw new SerDeException("null inspector ??"); } return inspector; } - private static DateTimeFormatter createAutoParser() - { - final DateTimeFormatter offsetElement = new DateTimeFormatterBuilder() - .appendTimeZoneOffset("Z", true, 2, 4) - .toFormatter(); + private static DateTimeFormatter createAutoParser() { + final DateTimeFormatter + offsetElement = + new DateTimeFormatterBuilder().appendTimeZoneOffset("Z", true, 2, 4).toFormatter(); - DateTimeParser timeOrOffset = new DateTimeFormatterBuilder() - .append( - null, - new DateTimeParser[]{ + DateTimeParser + timeOrOffset = + new DateTimeFormatterBuilder().append(null, + new DateTimeParser[] { new DateTimeFormatterBuilder().appendLiteral('T').toParser(), - new DateTimeFormatterBuilder().appendLiteral(' ').toParser() - } - ) - .appendOptional(ISODateTimeFormat.timeElementParser().getParser()) - .appendOptional(offsetElement.getParser()) - .toParser(); + new DateTimeFormatterBuilder().appendLiteral(' ').toParser() }) + .appendOptional(ISODateTimeFormat.timeElementParser().getParser()) + .appendOptional(offsetElement.getParser()) + .toParser(); - return new DateTimeFormatterBuilder() - .append(ISODateTimeFormat.dateElementParser()) + return new DateTimeFormatterBuilder().append(ISODateTimeFormat.dateElementParser()) .appendOptional(timeOrOffset) .toFormatter(); } diff --git a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaPullerInputFormat.java b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaPullerInputFormat.java index 8d5cd5b20bfc..aeec9642dccc 100644 --- a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaPullerInputFormat.java +++ b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaPullerInputFormat.java @@ -7,7 +7,7 @@ * "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 + * 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, @@ -50,29 +50,21 @@ import java.util.concurrent.TimeoutException; import java.util.stream.Collectors; - /** - * Kafka puller input format is in charge of reading a exact set of records from a Kafka Queue - * The input split will contain the set of topic partition and start/end offsets - * Records will be returned as bytes + * Kafka puller input format to read records from a Kafka Queue. + * 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 -{ - - private static final Logger log = LoggerFactory.getLogger(KafkaPullerInputFormat.class); + implements org.apache.hadoop.mapred.InputFormat { + private static final Logger LOG = LoggerFactory.getLogger(KafkaPullerInputFormat.class); - @Override - public InputSplit[] getSplits( - JobConf jobConf, int i - ) throws IOException - { - List inputSplits = null; + @Override public InputSplit[] getSplits(JobConf jobConf, int i) throws IOException { + List inputSplits; try { inputSplits = computeSplits(jobConf); - } - catch (InterruptedException e) { + } catch (InterruptedException e) { Thread.currentThread().interrupt(); throw new IOException(e); } @@ -81,8 +73,8 @@ public InputSplit[] getSplits( } /** - * Build a full scan using Kafka list partition then beginning/end offsets - * This function might block duo to calls like + * Build a full scan using Kafka list partition then beginning/end offsets. + * This function might block duo to calls like: * org.apache.kafka.clients.consumer.KafkaConsumer#beginningOffsets(java.util.Collection) * * @param topic kafka topic @@ -91,12 +83,9 @@ public InputSplit[] getSplits( * * @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) { final Map starOffsetsMap; final Map endOffsetsMap; @@ -105,128 +94,107 @@ private static List buildFullScanFromKafka( starOffsetsMap = consumer.beginningOffsets(topicPartitions); endOffsetsMap = consumer.endOffsets(topicPartitions); - if (log.isDebugEnabled()) { - log.info( - "Found the following partitions [{}]", - topicPartitions.stream().map(topicPartition -> topicPartition.toString()) - .collect(Collectors.joining(",")) - ); - starOffsetsMap.forEach((tp, start) -> log.info("TPartition [{}],Start offsets [{}]", tp, start)); - endOffsetsMap.forEach((tp, end) -> log.info("TPartition [{}],End offsets [{}]", tp, end)); + if (LOG.isDebugEnabled()) { + LOG.info("Found the following partitions [{}]", + topicPartitions.stream().map(topicPartition -> topicPartition.toString()).collect(Collectors.joining(","))); + starOffsetsMap.forEach((tp, start) -> LOG.info("TPartition [{}],Start offsets [{}]", tp, start)); + endOffsetsMap.forEach((tp, end) -> LOG.info("TPartition [{}],End offsets [{}]", tp, end)); } - return topicPartitions.stream().map( - topicPartition -> new KafkaPullerInputSplit( - topicPartition.topic(), + return topicPartitions.stream() + .map(topicPartition -> new KafkaPullerInputSplit(topicPartition.topic(), topicPartition.partition(), starOffsetsMap.get(topicPartition), endOffsetsMap.get(topicPartition), - tablePaths[0] - )).collect(Collectors.toList()); + tablePaths[0])) + .collect(Collectors.toList()); } private List computeSplits(Configuration configuration) - throws IOException, InterruptedException - { + throws IOException, InterruptedException { // this will be used to harness some KAFKA blocking calls final ExecutorService execService = Executors.newSingleThreadExecutor(); try (KafkaConsumer consumer = new KafkaConsumer(KafkaStreamingUtils.consumerProperties(configuration))) { final String topic = configuration.get(KafkaStorageHandler.HIVE_KAFKA_TOPIC); - final long timeoutMs = configuration.getLong( - KafkaStorageHandler.HIVE_KAFKA_POLL_TIMEOUT, - KafkaStorageHandler.DEFAULT_CONSUMER_POLL_TIMEOUT_MS - ); + final long + timeoutMs = + configuration.getLong(KafkaStorageHandler.HIVE_KAFKA_POLL_TIMEOUT, + KafkaStorageHandler.DEFAULT_CONSUMER_POLL_TIMEOUT_MS); // hive depends on FileSplits JobConf jobConf = new JobConf(configuration); Path[] tablePaths = org.apache.hadoop.mapred.FileInputFormat.getInputPaths(jobConf); - Future> futureFullHouse = execService.submit(() -> buildFullScanFromKafka( - topic, - consumer, - tablePaths - )); + Future> + futureFullHouse = + execService.submit(() -> buildFullScanFromKafka(topic, consumer, tablePaths)); List fullHouse; try { fullHouse = futureFullHouse.get(timeoutMs, TimeUnit.MILLISECONDS); - } - catch (TimeoutException | ExecutionException e) { + } catch (TimeoutException | ExecutionException e) { futureFullHouse.cancel(true); - log.error("can not generate full scan split", e); + LOG.error("can not generate full scan split", e); // at this point we can not go further fail split generation throw new IOException(e); } - - final ImmutableMap.Builder fullHouseMapBuilder = new ImmutableMap.Builder(); - fullHouse.stream().forEach(input -> fullHouseMapBuilder.put(new TopicPartition( - input.getTopic(), - input.getPartition() - ), input)); + final ImmutableMap.Builder + fullHouseMapBuilder = + new ImmutableMap.Builder(); + fullHouse.stream() + .forEach(input -> fullHouseMapBuilder.put(new TopicPartition(input.getTopic(), input.getPartition()), input)); final KafkaScanTrimmer kafkaScanTrimmer = new KafkaScanTrimmer(fullHouseMapBuilder.build(), consumer); final String filterExprSerialized = configuration.get(TableScanDesc.FILTER_EXPR_CONF_STR); if (filterExprSerialized != null && !filterExprSerialized.isEmpty()) { ExprNodeGenericFuncDesc filterExpr = SerializationUtilities.deserializeExpression(filterExprSerialized); - log.info("Kafka trimmer working on Filter tree {}", filterExpr.getExprString()); - Callable> trimmerWorker = () -> kafkaScanTrimmer.computeOptimizedScan(filterExpr) - .entrySet() - .stream() - .map(entry -> entry.getValue()) - .collect(Collectors.toList()); + LOG.info("Kafka trimmer working on Filter tree {}", filterExpr.getExprString()); + Callable> + trimmerWorker = () -> kafkaScanTrimmer.computeOptimizedScan(filterExpr) + .entrySet() + .stream() + .map(entry -> entry.getValue()) + .collect(Collectors.toList()); Future> futureTinyHouse = execService.submit(trimmerWorker); try { return futureTinyHouse.get(timeoutMs, TimeUnit.MILLISECONDS); - } - catch (ExecutionException | TimeoutException e) { + } catch (ExecutionException | TimeoutException e) { futureTinyHouse.cancel(true); - log.error("Had issue with trimmer will return full scan ", e); + LOG.error("Had issue with trimmer will return full scan ", e); return fullHouse; } } //Case null: it can be filter evaluated to false or no filter at all thus return full scan return fullHouse; - } - finally { + } finally { execService.shutdown(); } } - - private static List fetchTopicPartitions(String topic, KafkaConsumer consumer) - { + private static List fetchTopicPartitions(String topic, KafkaConsumer consumer) { // this will block till REQUEST_TIMEOUT_MS_CONFIG = "request.timeout.ms" // then throws org.apache.kafka.common.errors.TimeoutException if can not fetch metadata // @TODO add retry logic maybe List partitions = consumer.partitionsFor(topic); - return partitions.stream().map(p -> new TopicPartition(topic, p.partition())) - .collect(Collectors.toList()); + return partitions.stream().map(p -> new TopicPartition(topic, p.partition())).collect(Collectors.toList()); } - @Override - public RecordReader getRecordReader( - InputSplit inputSplit, - JobConf jobConf, Reporter reporter - ) throws IOException - { + @Override public RecordReader getRecordReader(InputSplit inputSplit, + JobConf jobConf, + Reporter reporter) throws IOException { return new KafkaPullerRecordReader((KafkaPullerInputSplit) inputSplit, jobConf); } - @Override - public List getSplits( - JobContext jobContext - ) throws IOException, InterruptedException - { + @Override public List getSplits(JobContext jobContext) + throws IOException, InterruptedException { return computeSplits(jobContext.getConfiguration()).stream() - .map(kafkaPullerInputSplit -> (org.apache.hadoop.mapreduce.InputSplit) kafkaPullerInputSplit) - .collect(Collectors.toList()); + .map(kafkaPullerInputSplit -> (org.apache.hadoop.mapreduce.InputSplit) kafkaPullerInputSplit) + .collect(Collectors.toList()); } - @Override - public org.apache.hadoop.mapreduce.RecordReader createRecordReader( - org.apache.hadoop.mapreduce.InputSplit inputSplit, TaskAttemptContext taskAttemptContext - ) throws IOException, InterruptedException - { + @Override public org.apache.hadoop.mapreduce.RecordReader createRecordReader( + org.apache.hadoop.mapreduce.InputSplit inputSplit, + TaskAttemptContext taskAttemptContext) throws IOException, InterruptedException { return new KafkaPullerRecordReader(); } } diff --git a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaPullerInputSplit.java b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaPullerInputSplit.java index 9c6e866b917a..33813e829ceb 100644 --- a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaPullerInputSplit.java +++ b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaPullerInputSplit.java @@ -7,7 +7,7 @@ * "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 + * 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, @@ -31,53 +31,40 @@ import java.util.Collections; import java.util.List; -public class KafkaPullerInputSplit extends FileSplit - implements org.apache.hadoop.mapred.InputSplit -{ +/** + * Kafka Hadoop Input Split Class. + */ +public class KafkaPullerInputSplit extends FileSplit implements org.apache.hadoop.mapred.InputSplit { private String topic; private long startOffset; private int partition; private long endOffset; - public KafkaPullerInputSplit() - { + public KafkaPullerInputSplit() { super((Path) null, 0, 0, (String[]) null); } - public KafkaPullerInputSplit( - String topic, int partition, long startOffset, - long endOffset, - Path dummyPath - ) - { + public KafkaPullerInputSplit(String topic, int partition, long startOffset, long endOffset, Path dummyPath) { super(dummyPath, 0, 0, (String[]) null); this.topic = topic; this.startOffset = startOffset; this.partition = partition; this.endOffset = endOffset; - Preconditions.checkArgument( - startOffset >= 0 && startOffset <= endOffset, + Preconditions.checkArgument(startOffset >= 0 && startOffset <= endOffset, "start [%s] has to be positive and >= end [%]", startOffset, - endOffset - ); + endOffset); } - @Override - public long getLength() - { + @Override public long getLength() { return 0; } - @Override - public String[] getLocations() throws IOException - { + @Override public String[] getLocations() throws IOException { return new String[0]; } - @Override - public void write(DataOutput dataOutput) throws IOException - { + @Override public void write(DataOutput dataOutput) throws IOException { super.write(dataOutput); dataOutput.writeUTF(topic); dataOutput.writeInt(partition); @@ -85,45 +72,34 @@ public void write(DataOutput dataOutput) throws IOException dataOutput.writeLong(endOffset); } - @Override - public void readFields(DataInput dataInput) throws IOException - { + @Override public void readFields(DataInput dataInput) throws IOException { super.readFields(dataInput); topic = dataInput.readUTF(); partition = dataInput.readInt(); startOffset = dataInput.readLong(); endOffset = dataInput.readLong(); - Preconditions.checkArgument( - startOffset >= 0 && startOffset <= endOffset, + Preconditions.checkArgument(startOffset >= 0 && startOffset <= endOffset, "start [%s] has to be positive and >= end [%]", startOffset, - endOffset - ); + endOffset); } - public String getTopic() - { + public String getTopic() { return topic; } - public int getPartition() - { + public int getPartition() { return partition; } - public long getStartOffset() - { + public long getStartOffset() { return startOffset; } - public long getEndOffset() - { + public long getEndOffset() { return endOffset; } - - - /** * Compute the intersection of 2 splits. Splits must share the same topic and partition number. * @@ -132,8 +108,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 KafkaPullerInputSplit intersectRange(KafkaPullerInputSplit split1, + KafkaPullerInputSplit split2) { assert (split1.topic == split2.topic); assert (split1.partition == split2.partition); final long startOffset = Math.max(split1.getStartOffset(), split2.getStartOffset()); @@ -161,9 +137,7 @@ public static KafkaPullerInputSplit unionRange(KafkaPullerInputSplit split1, Kaf return new KafkaPullerInputSplit(split1.topic, split1.partition, startOffset, endOffset, split1.getPath()); } - @Override - public boolean equals(Object o) - { + @Override public boolean equals(Object o) { if (this == o) { return true; } @@ -171,43 +145,41 @@ public boolean equals(Object o) return false; } KafkaPullerInputSplit that = (KafkaPullerInputSplit) o; - return Objects.equal(getTopic(), that.getTopic()) && Objects - .equal(getStartOffset(), that.getStartOffset()) && Objects - .equal(getPartition(), that.getPartition()) && Objects - .equal(getEndOffset(), that.getEndOffset()); + return Objects.equal(getTopic(), that.getTopic()) + && Objects.equal(getStartOffset(), that.getStartOffset()) + && Objects.equal(getPartition(), that.getPartition()) + && Objects.equal(getEndOffset(), that.getEndOffset()); } - @Override - public int hashCode() - { + @Override public int hashCode() { return Objects.hashCode(getTopic(), getStartOffset(), getPartition(), getEndOffset()); } - @Override - public String toString() - { - return "KafkaPullerInputSplit{" + - "topic='" + topic + '\'' + - ", startOffset=" + startOffset + - ", partition=" + partition + - ", endOffset=" + endOffset + - ", path=" + super.getPath().toString() + - '}'; - } - - public static KafkaPullerInputSplit copyOf(KafkaPullerInputSplit other) - { - return new KafkaPullerInputSplit( - other.getTopic(), + @Override public String toString() { + return "KafkaPullerInputSplit{" + + "topic='" + + topic + + '\'' + + ", startOffset=" + + startOffset + + ", partition=" + + partition + + ", endOffset=" + + endOffset + + ", path=" + + super.getPath().toString() + + '}'; + } + + public static KafkaPullerInputSplit copyOf(KafkaPullerInputSplit other) { + return new KafkaPullerInputSplit(other.getTopic(), other.getPartition(), other.getStartOffset(), other.getEndOffset(), - other.getPath() - ); + other.getPath()); } - public KafkaPullerInputSplit clone() - { + public KafkaPullerInputSplit clone() { return copyOf(this); } @@ -216,12 +188,20 @@ public static List slice(long sliceSize, final KafkaPulle ImmutableList.Builder builder = ImmutableList.builder(); long start = split.getStartOffset(); while (start < split.getEndOffset() - sliceSize) { - builder.add(new KafkaPullerInputSplit(split.topic, split.partition, start, start + sliceSize + 1, split.getPath())); + builder.add(new KafkaPullerInputSplit(split.topic, + split.partition, + start, + start + sliceSize + 1, + split.getPath())); start += sliceSize + 1; } // last split if (start < split.getEndOffset()) { - builder.add(new KafkaPullerInputSplit(split.topic, split.partition, start, split.getEndOffset(), split.getPath())); + builder.add(new KafkaPullerInputSplit(split.topic, + split.partition, + start, + split.getEndOffset(), + split.getPath())); } return builder.build(); } diff --git a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaPullerRecordReader.java b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaPullerRecordReader.java index 24f5141592c8..81f1204f8409 100644 --- a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaPullerRecordReader.java +++ b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaPullerRecordReader.java @@ -7,7 +7,7 @@ * "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 + * 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, @@ -35,11 +35,13 @@ import java.util.Iterator; import java.util.Properties; +/** + * Kafka Records Reader implementation. + */ public class KafkaPullerRecordReader extends RecordReader - implements org.apache.hadoop.mapred.RecordReader -{ + implements org.apache.hadoop.mapred.RecordReader { - private static final Logger log = LoggerFactory.getLogger(KafkaPullerRecordReader.class); + private static final Logger LOG = LoggerFactory.getLogger(KafkaPullerRecordReader.class); private final Closer closer = Closer.create(); private KafkaConsumer consumer = null; @@ -51,68 +53,58 @@ public class KafkaPullerRecordReader extends RecordReader= 0 && startOffset <= endOffset, + Preconditions.checkState(startOffset >= 0 && startOffset <= endOffset, "Start [%s] has to be positive and less or equal than End [%s]", startOffset, - endOffset - ); + endOffset); totalNumberRecords += endOffset - startOffset; initConsumer(); - pollTimeout = config.getLong( - KafkaStorageHandler.HIVE_KAFKA_POLL_TIMEOUT, - KafkaStorageHandler.DEFAULT_CONSUMER_POLL_TIMEOUT_MS - ); - log.debug("Consumer poll timeout [{}] ms", pollTimeout); + pollTimeout = + config.getLong(KafkaStorageHandler.HIVE_KAFKA_POLL_TIMEOUT, + KafkaStorageHandler.DEFAULT_CONSUMER_POLL_TIMEOUT_MS); + LOG.debug("Consumer poll timeout [{}] ms", pollTimeout); this.recordsCursor = new KafkaRecordIterator(consumer, topicPartition, startOffset, endOffset, pollTimeout); started = true; } } - @Override - synchronized public void initialize(org.apache.hadoop.mapreduce.InputSplit inputSplit, TaskAttemptContext context) - { + @Override public void initialize(org.apache.hadoop.mapreduce.InputSplit inputSplit, + TaskAttemptContext context) { initialize((KafkaPullerInputSplit) inputSplit, context.getConfiguration()); } - @Override - public boolean next(NullWritable nullWritable, KafkaRecordWritable bytesWritable) - { + @Override public boolean next(NullWritable nullWritable, KafkaRecordWritable bytesWritable) { if (started && recordsCursor.hasNext()) { ConsumerRecord record = recordsCursor.next(); bytesWritable.set(record); @@ -123,27 +115,19 @@ public boolean next(NullWritable nullWritable, KafkaRecordWritable bytesWritable return false; } - @Override - public NullWritable createKey() - { + @Override public NullWritable createKey() { return NullWritable.get(); } - @Override - public KafkaRecordWritable createValue() - { + @Override public KafkaRecordWritable createValue() { return new KafkaRecordWritable(); } - @Override - public long getPos() throws IOException - { + @Override public long getPos() throws IOException { return consumedRecords; } - @Override - public boolean nextKeyValue() throws IOException - { + @Override public boolean nextKeyValue() throws IOException { currentWritableValue = new KafkaRecordWritable(); if (next(NullWritable.get(), currentWritableValue)) { return true; @@ -152,21 +136,15 @@ public boolean nextKeyValue() throws IOException return false; } - @Override - public NullWritable getCurrentKey() throws IOException, InterruptedException - { + @Override public NullWritable getCurrentKey() throws IOException, InterruptedException { return NullWritable.get(); } - @Override - public KafkaRecordWritable getCurrentValue() throws IOException, InterruptedException - { + @Override public KafkaRecordWritable getCurrentValue() throws IOException, InterruptedException { return Preconditions.checkNotNull(currentWritableValue); } - @Override - public float getProgress() throws IOException - { + @Override public float getProgress() throws IOException { if (consumedRecords == 0) { return 0f; } @@ -176,13 +154,11 @@ public float getProgress() throws IOException return consumedRecords * 1.0f / totalNumberRecords; } - @Override - public void close() throws IOException - { + @Override public void close() throws IOException { if (!started) { return; } - log.trace("total read bytes [{}]", readBytes); + LOG.trace("total read bytes [{}]", readBytes); if (consumer != null) { consumer.wakeup(); } diff --git a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaRecordIterator.java b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaRecordIterator.java index d120b663ec24..7d5754d81b4a 100644 --- a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaRecordIterator.java +++ b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaRecordIterator.java @@ -7,7 +7,7 @@ * "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 + * 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, @@ -25,11 +25,11 @@ 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.TimeoutException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import javax.annotation.Nullable; -import java.util.Collection; import java.util.Collections; import java.util.Iterator; import java.util.List; @@ -39,16 +39,17 @@ * Iterator over Kafka Records to read records from a single topic partition inclusive start exclusive end. *

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

* When provided with an end offset it will return records up to the record with offset == endOffset - 1, - * Else If end offsets is null it will read up to the current end see {@link org.apache.kafka.clients.consumer.Consumer#endOffsets(Collection)} + * Else If end offsets is null it will read up to the current end see + * {@link org.apache.kafka.clients.consumer.Consumer#endOffsets(java.util.Collection)} *

* Current implementation of this Iterator will throw and exception if can not poll up to the endOffset - 1 */ -public class KafkaRecordIterator implements Iterator> -{ - private static final Logger log = LoggerFactory.getLogger(KafkaRecordIterator.class); +public class KafkaRecordIterator implements Iterator> { + private static final Logger LOG = LoggerFactory.getLogger(KafkaRecordIterator.class); private final Consumer consumer; private final TopicPartition topicPartition; @@ -60,6 +61,7 @@ public class KafkaRecordIterator implements Iterator nextRecord; private boolean hasMore = true; + private final boolean started; //Kafka consumer poll method return an iterator of records. private Iterator> consumerRecordIterator = null; @@ -71,113 +73,115 @@ public class KafkaRecordIterator implements Iterator consumer, TopicPartition topicPartition, - @Nullable Long startOffset, @Nullable Long endOffset, long pollTimeoutMs - ) - { + public KafkaRecordIterator(Consumer consumer, + TopicPartition topicPartition, + @Nullable Long startOffset, + @Nullable Long endOffset, + long pollTimeoutMs) { this.consumer = Preconditions.checkNotNull(consumer, "Consumer can not be null"); this.topicPartition = Preconditions.checkNotNull(topicPartition, "Topic partition can not be null"); this.pollTimeoutMs = pollTimeoutMs; Preconditions.checkState(this.pollTimeoutMs > 0, "poll timeout has to be positive number"); - this.startOffset = startOffset == null ? -1l : startOffset; - this.endOffset = endOffset == null ? -1l : endOffset; + this.startOffset = startOffset == null ? -1L : startOffset; + this.endOffset = endOffset == null ? -1L : endOffset; + assignAndSeek(); + this.started = true; } - public KafkaRecordIterator( - Consumer consumer, TopicPartition tp, long pollTimeoutMs - ) - { + public KafkaRecordIterator(Consumer consumer, TopicPartition tp, long pollTimeoutMs) { this(consumer, tp, null, null, pollTimeoutMs); } - private void assignAndSeek() - { + private void assignAndSeek() { // assign topic partition to consumer final List topicPartitionList = ImmutableList.of(topicPartition); - if (log.isTraceEnabled()) { + if (LOG.isTraceEnabled()) { stopwatch.reset().start(); } consumer.assign(topicPartitionList); // compute offsets and seek to start if (startOffset > -1) { - log.info("Seeking to offset [{}] of topic partition [{}]", startOffset, topicPartition); + LOG.info("Seeking to offset [{}] of topic partition [{}]", startOffset, topicPartition); consumer.seek(topicPartition, startOffset); } else { - log.info("Seeking to beginning of topic partition [{}]", topicPartition); + LOG.info("Seeking to beginning of topic partition [{}]", topicPartition); // seekToBeginning is lazy thus need to call position() or poll(0) this.consumer.seekToBeginning(Collections.singleton(topicPartition)); startOffset = consumer.position(topicPartition); } if (endOffset == -1) { this.endOffset = consumer.endOffsets(topicPartitionList).get(topicPartition); - log.debug("EndOffset is {}", endOffset); + LOG.info("EndOffset set to {}", endOffset); } currentOffset = consumer.position(topicPartition); Preconditions.checkState(this.endOffset >= currentOffset, - "End offset [%s] need to be greater than start offset [%s]", this.endOffset, currentOffset - ); - log.info("Kafka Iterator ready, assigned TopicPartition [{}]; startOffset [{}]; endOffset [{}]", topicPartition, - currentOffset, this.endOffset - ); - if (log.isTraceEnabled()) { + "End offset [%s] need to be greater than start offset [%s]", + this.endOffset, + currentOffset); + LOG.info("Kafka Iterator ready, assigned TopicPartition [{}]; startOffset [{}]; endOffset [{}]", + topicPartition, + currentOffset, + this.endOffset); + if (LOG.isTraceEnabled()) { stopwatch.stop(); - log.trace("Time to assign and seek [{}] ms", stopwatch.elapsed(TimeUnit.MILLISECONDS)); + LOG.trace("Time to assign and seek [{}] ms", stopwatch.elapsed(TimeUnit.MILLISECONDS)); } } @Override - public boolean hasNext() - { - if (records == null) { - assignAndSeek(); - } - //Init poll OR Need to poll at least one more record since currentOffset + 1 < endOffset - if (records == null || (hasMore == false && currentOffset + 1 < endOffset)) { + public boolean hasNext() { + /* + Poll more records if + Initial poll case -> (records == null) + OR + Need to poll at least one more record (currentOffset + 1 < endOffset) AND consumerRecordIterator is empty (!hasMore) + */ + if (!hasMore && currentOffset + 1 < endOffset || records == null) { pollRecords(); findNext(); } return hasMore; } - private void pollRecords() - { - if (log.isTraceEnabled()) { + /** + * Poll more records or Fail with {@link TimeoutException} if no records returned before reaching target end offset. + */ + private void pollRecords() { + if (LOG.isTraceEnabled()) { stopwatch.reset().start(); } + Preconditions.checkArgument(started); records = consumer.poll(pollTimeoutMs); - if (log.isTraceEnabled()) { + if (LOG.isTraceEnabled()) { stopwatch.stop(); - log.trace("Pulled [{}] records in [{}] ms", records.count(), - stopwatch.elapsed(TimeUnit.MILLISECONDS) - ); + LOG.trace("Pulled [{}] records in [{}] ms", records.count(), stopwatch.elapsed(TimeUnit.MILLISECONDS)); } // Fail if we can not poll within one lap of pollTimeoutMs. - Preconditions.checkState( - !records.isEmpty() || currentOffset == endOffset, - "Current offset: [%s]-TopicPartition:[%s], target End offset:[%s]." - + "Consumer returned 0 record due to exhausted poll timeout [%s]ms, try increasing[%s] ", - currentOffset, - topicPartition.toString(), - endOffset, - pollTimeoutMs, - KafkaStorageHandler.HIVE_KAFKA_POLL_TIMEOUT - ); + if (records.isEmpty() && currentOffset < endOffset) { + throw new TimeoutException(String.format("Current offset: [%s]-TopicPartition:[%s], target End offset:[%s]." + + "Consumer returned 0 record due to exhausted poll timeout [%s]ms, try increasing[%s]", + currentOffset, + topicPartition.toString(), + endOffset, + pollTimeoutMs, + KafkaStorageHandler.HIVE_KAFKA_POLL_TIMEOUT)); + } consumerRecordIterator = records.iterator(); } - @Override - public ConsumerRecord next() - { + @Override public ConsumerRecord next() { ConsumerRecord value = nextRecord; Preconditions.checkState(value.offset() < endOffset); findNext(); return Preconditions.checkNotNull(value); } - private void findNext() - { + /** + * Find the next element in the batch of returned records by previous poll or set hasMore to false tp poll more next + * call to {@link KafkaRecordIterator#hasNext()}. + */ + private void findNext() { if (consumerRecordIterator.hasNext()) { nextRecord = consumerRecordIterator.next(); hasMore = true; diff --git a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaRecordWritable.java b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaRecordWritable.java index 08cf53be6430..b6b8d391313a 100644 --- a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaRecordWritable.java +++ b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaRecordWritable.java @@ -7,7 +7,7 @@ * "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 + * 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, @@ -37,12 +37,13 @@ public class KafkaRecordWritable implements Writable { private int partition; private long offset; private long timestamp; - private byte [] value; + private byte[] value; public static KafkaRecordWritable fromKafkaRecord(ConsumerRecord consumerRecord) { - return new KafkaRecordWritable(consumerRecord.partition(), consumerRecord.offset(), - consumerRecord.timestamp(), consumerRecord.value() - ); + return new KafkaRecordWritable(consumerRecord.partition(), + consumerRecord.offset(), + consumerRecord.timestamp(), + consumerRecord.value()); } public void set(ConsumerRecord consumerRecord) { @@ -74,7 +75,7 @@ public KafkaRecordWritable() { timestamp = dataInput.readLong(); partition = dataInput.readInt(); offset = dataInput.readLong(); - int size = dataInput.readInt(); + int size = dataInput.readInt(); if (size > 0) { value = new byte[size]; dataInput.readFully(value); @@ -107,8 +108,10 @@ public byte[] getValue() { return false; } KafkaRecordWritable that = (KafkaRecordWritable) o; - return getPartition() == that.getPartition() && getOffset() == that.getOffset() - && getTimestamp() == that.getTimestamp() && Arrays.equals(getValue(), that.getValue()); + return getPartition() == that.getPartition() + && getOffset() == that.getOffset() + && getTimestamp() == that.getTimestamp() + && Arrays.equals(getValue(), that.getValue()); } @Override public int hashCode() { diff --git a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaScanTrimmer.java b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaScanTrimmer.java index 4ce82b593933..e90c4867869b 100644 --- a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaScanTrimmer.java +++ b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaScanTrimmer.java @@ -58,57 +58,50 @@ /** * Kafka Range trimmer, takes a full kafka scan and prune the scan based on a filter expression - * it is a Best effort trimmer and it can not replace the filter it self, filtration still takes place in Hive executor + * it is a Best effort trimmer and it can not replace the filter it self, filtration still takes place in Hive executor. */ -public class KafkaScanTrimmer -{ - private static final Logger log = LoggerFactory.getLogger(KafkaScanTrimmer.class); +public class KafkaScanTrimmer { + private static final Logger LOG = LoggerFactory.getLogger(KafkaScanTrimmer.class); 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 */ - public KafkaScanTrimmer( - Map fullHouse, - KafkaConsumer kafkaConsumer - ) - { + public KafkaScanTrimmer(Map fullHouse, KafkaConsumer kafkaConsumer) { this.fullHouse = fullHouse; this.kafkaConsumer = kafkaConsumer; } /** - * This might block due to calls like: + * This might block due to calls like. * org.apache.kafka.clients.consumer.KafkaConsumer#offsetsForTimes(java.util.Map) * * @param filterExpression filter expression to be used for pruning scan * * @return tiny house of of the full house based on filter expression */ - public Map computeOptimizedScan(ExprNodeGenericFuncDesc filterExpression) - { + public Map computeOptimizedScan(ExprNodeGenericFuncDesc filterExpression) { Map optimizedScan = parse(filterExpression); - if (log.isDebugEnabled()) { + if (LOG.isDebugEnabled()) { if (optimizedScan != null) { - log.debug("Optimized scan:"); - optimizedScan.forEach((tp, input) -> log.info( + LOG.debug("Optimized scan:"); + optimizedScan.forEach((tp, input) -> LOG.info( "Topic-[{}] Partition-[{}] - Split startOffset [{}] :-> endOffset [{}]", tp.topic(), tp.partition(), - input.getStartOffset(), input.getEndOffset() - )); + input.getStartOffset(), + input.getEndOffset())); } else { - log.debug("No optimization thus using full scan "); - fullHouse.forEach((tp, input) -> log.info( + LOG.debug("No optimization thus using full scan "); + fullHouse.forEach((tp, input) -> LOG.info( "Topic-[{}] Partition-[{}] - Split startOffset [{}] :-> endOffset [{}]", tp.topic(), tp.partition(), - input.getStartOffset(), input.getEndOffset() - )); + input.getStartOffset(), + input.getEndOffset())); } } return optimizedScan == null ? fullHouse : optimizedScan; @@ -119,9 +112,7 @@ public Map computeOptimizedScan(ExprNodeG * * @return Map of optimized kafka range scans or null if it is impossible to optimize. */ - @Nullable - private Map parse(ExprNodeDesc expression) - { + @Nullable private Map parse(ExprNodeDesc expression) { if (expression.getClass() != ExprNodeGenericFuncDesc.class) { return null; } @@ -153,7 +144,6 @@ private Map parse(ExprNodeDesc expression } } - /** * @param expr leaf node to push * @param operator operator @@ -163,11 +153,9 @@ private Map parse(ExprNodeDesc expression * * @return leaf scan or null if can not figure out push down */ - @Nullable - private Map pushLeaf( - ExprNodeGenericFuncDesc expr, PredicateLeaf.Operator operator, boolean negation - ) - { + @Nullable private Map pushLeaf(ExprNodeGenericFuncDesc expr, + PredicateLeaf.Operator operator, + boolean negation) { if (expr.getChildren().size() != 2) { return null; } @@ -203,36 +191,31 @@ private Map pushLeaf( constantDesc = (ExprNodeConstantDesc) extracted[0]; } - if (columnDesc.getColumn().equals(KafkaStorageHandler.__PARTITION)) { - return buildScanFormPartitionPredicate( - fullHouse, + if (columnDesc.getColumn().equals(KafkaStorageHandler.PARTITION_COLUMN)) { + return buildScanFormPartitionPredicate(fullHouse, operator, ((Number) constantDesc.getValue()).intValue(), flip, - negation - ); + negation); } - if (columnDesc.getColumn().equals(KafkaStorageHandler.__OFFSET)) { - return buildScanFromOffsetPredicate( - fullHouse, + if (columnDesc.getColumn().equals(KafkaStorageHandler.OFFSET_COLUMN)) { + return buildScanFromOffsetPredicate(fullHouse, operator, ((Number) constantDesc.getValue()).longValue(), flip, - negation - ); + negation); } - if (columnDesc.getColumn().equals(KafkaStorageHandler.__TIMESTAMP)) { + if (columnDesc.getColumn().equals(KafkaStorageHandler.TIMESTAMP_COLUMN)) { long timestamp = ((Number) constantDesc.getValue()).longValue(); return buildScanForTimesPredicate(fullHouse, operator, timestamp, flip, negation, kafkaConsumer); } return null; } - /** - * Trim kafka scan using a leaf binary predicate on partition column + * Trim kafka scan using a leaf binary predicate on partition column. * * @param fullScan kafka full scan to be optimized * @param operator predicate operator, equal, lessThan or lessThanEqual @@ -242,47 +225,45 @@ private Map pushLeaf( * * @return filtered kafka scan */ - @VisibleForTesting - protected static Map buildScanFormPartitionPredicate( + @VisibleForTesting protected static Map buildScanFormPartitionPredicate( Map fullScan, - PredicateLeaf.Operator operator, int partitionConst, boolean flip, boolean negation - ) - { + PredicateLeaf.Operator operator, + int partitionConst, + boolean flip, + boolean negation) { final Predicate predicate; final Predicate intermediatePredicate; switch (operator) { - case EQUALS: - predicate = topicPartition -> topicPartition != null && topicPartition.partition() == partitionConst; - break; - case LESS_THAN: - intermediatePredicate = flip - ? topicPartition -> topicPartition != null - && partitionConst < topicPartition.partition() - : topicPartition -> topicPartition != null - && topicPartition.partition() < partitionConst; - - predicate = negation ? intermediatePredicate.negate() : intermediatePredicate; - break; - case LESS_THAN_EQUALS: - intermediatePredicate = flip ? topicPartition -> topicPartition != null - && partitionConst - <= topicPartition.partition() - : topicPartition -> topicPartition != null - && topicPartition.partition() <= partitionConst; - - predicate = negation ? intermediatePredicate.negate() : intermediatePredicate; - break; - default: - //Default to select * for unknown cases - predicate = topicPartition -> true; + case EQUALS: + predicate = topicPartition -> topicPartition != null && topicPartition.partition() == partitionConst; + break; + case LESS_THAN: + intermediatePredicate = + flip ? + topicPartition -> topicPartition != null && partitionConst < topicPartition.partition() : + topicPartition -> topicPartition != null && topicPartition.partition() < partitionConst; + + predicate = negation ? intermediatePredicate.negate() : intermediatePredicate; + break; + case LESS_THAN_EQUALS: + intermediatePredicate = + flip ? + topicPartition -> topicPartition != null && partitionConst <= topicPartition.partition() : + topicPartition -> topicPartition != null && topicPartition.partition() <= partitionConst; + + predicate = negation ? intermediatePredicate.negate() : intermediatePredicate; + break; + default: + //Default to select * for unknown cases + predicate = topicPartition -> true; } 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())); + .stream() + .filter(entry -> predicate.test(entry.getKey())) + .forEach(entry -> builder.put(entry.getKey(), entry.getValue().clone())); return builder.build(); } @@ -295,48 +276,48 @@ protected static Map buildScanFormPartiti * * @return optimized kafka scan */ - @VisibleForTesting - protected static Map buildScanFromOffsetPredicate( - final Map fullScan, - PredicateLeaf.Operator operator, long offsetConst, boolean flip, boolean negation - ) - { + @VisibleForTesting protected static Map buildScanFromOffsetPredicate( + Map fullScan, + PredicateLeaf.Operator operator, + long offsetConst, + boolean flip, + boolean negation) { final boolean isEndBound; final long startOffset; final long endOffset; - if (flip == false && negation == false || flip == true && negation == true) { + if (flip == negation) { isEndBound = true; } else { isEndBound = false; } switch (operator) { - case LESS_THAN_EQUALS: - if (isEndBound) { - startOffset = -1; - endOffset = negation ? offsetConst : offsetConst + 1; - } else { - endOffset = -1; - startOffset = negation ? offsetConst + 1 : offsetConst; - } - break; - case EQUALS: - startOffset = offsetConst; - endOffset = offsetConst + 1; - break; - case LESS_THAN: - if (isEndBound) { - endOffset = negation ? offsetConst + 1 : offsetConst; - startOffset = -1; - } else { - endOffset = -1; - startOffset = negation ? offsetConst : offsetConst + 1; - } - break; - default: - // default to select * + case LESS_THAN_EQUALS: + if (isEndBound) { + startOffset = -1; + endOffset = negation ? offsetConst : offsetConst + 1; + } else { + endOffset = -1; + startOffset = negation ? offsetConst + 1 : offsetConst; + } + break; + case EQUALS: + startOffset = offsetConst; + endOffset = offsetConst + 1; + break; + case LESS_THAN: + if (isEndBound) { + endOffset = negation ? offsetConst + 1 : offsetConst; startOffset = -1; + } else { endOffset = -1; + startOffset = negation ? offsetConst : offsetConst + 1; + } + break; + default: + // default to select * + startOffset = -1; + endOffset = -1; } final Map newScan = new HashMap<>(); @@ -344,53 +325,39 @@ protected static Map buildScanFromOffsetP fullScan.forEach((tp, existingInputSplit) -> { final KafkaPullerInputSplit newInputSplit; if (startOffset != -1 && endOffset == -1) { - newInputSplit = new KafkaPullerInputSplit( - tp.topic(), + newInputSplit = new KafkaPullerInputSplit(tp.topic(), tp.partition(), // @TODO make sure that this is okay //if the user as for start offset > max offset will replace with last offset Math.min(startOffset, existingInputSplit.getEndOffset()), existingInputSplit.getEndOffset(), - existingInputSplit.getPath() - ); + existingInputSplit.getPath()); } else if (endOffset != -1 && startOffset == -1) { - newInputSplit = new KafkaPullerInputSplit( - tp.topic(), - tp.partition(), - existingInputSplit.getStartOffset(), + newInputSplit = new KafkaPullerInputSplit(tp.topic(), tp.partition(), existingInputSplit.getStartOffset(), //@TODO check this, if user ask for non existing end offset ignore it and position head on start - Math.max(endOffset, existingInputSplit.getStartOffset()), - existingInputSplit.getPath() - ); + 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 KafkaPullerInputSplit(tp.topic(), tp.partition(), //@TODO check this with team if we have ask for offset out of range what to do ? // here am seeking to last offset - existingInputSplit.getEndOffset(), - existingInputSplit.getEndOffset(), - existingInputSplit.getPath() - ); + existingInputSplit.getEndOffset(), existingInputSplit.getEndOffset(), existingInputSplit.getPath()); } else { - newInputSplit = new KafkaPullerInputSplit( - tp.topic(), - tp.partition(), - startOffset, - endOffset, - existingInputSplit.getPath() - ); + newInputSplit = + new KafkaPullerInputSplit(tp.topic(), + tp.partition(), + startOffset, + endOffset, + existingInputSplit.getPath()); } } else { - newInputSplit = new KafkaPullerInputSplit( - tp.topic(), - tp.partition(), - existingInputSplit.getStartOffset(), - existingInputSplit.getEndOffset(), - existingInputSplit.getPath() - ); + newInputSplit = + new KafkaPullerInputSplit(tp.topic(), + tp.partition(), + existingInputSplit.getStartOffset(), + existingInputSplit.getEndOffset(), + existingInputSplit.getPath()); } newScan.put(tp, KafkaPullerInputSplit.intersectRange(newInputSplit, existingInputSplit)); @@ -399,14 +366,17 @@ protected static Map buildScanFromOffsetP return newScan; } - @Nullable - protected static Map buildScanForTimesPredicate( - final Map fullHouse, - PredicateLeaf.Operator operator, long timestamp, boolean flip, boolean negation, KafkaConsumer consumer - ) - { - long increment = (flip && operator == PredicateLeaf.Operator.LESS_THAN - || negation && operator == PredicateLeaf.Operator.LESS_THAN_EQUALS) ? 1L : 0L; + @Nullable protected static Map buildScanForTimesPredicate( + Map fullHouse, + PredicateLeaf.Operator operator, + long timestamp, + boolean flip, + boolean negation, + KafkaConsumer consumer) { + long + increment = + (flip && operator == PredicateLeaf.Operator.LESS_THAN + || negation && operator == PredicateLeaf.Operator.LESS_THAN_EQUALS) ? 1L : 0L; // only accepted cases are timestamp_column [ > ; >= ; = ]constant if (operator == PredicateLeaf.Operator.EQUALS || flip ^ negation) { final Map timePartitionsMap = Maps.toMap(fullHouse.keySet(), tp -> timestamp + increment); @@ -419,18 +389,15 @@ protected static Map buildScanForTimesPre 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( - tp.topic(), + return new KafkaPullerInputSplit(tp.topic(), tp.partition(), startOffset, existing.getEndOffset(), - existing.getPath() - ); + existing.getPath()); }); return newScan; - } - catch (Exception e) { - log.error("Error while looking up offsets for time", e); + } catch (Exception e) { + LOG.error("Error while looking up offsets for time", e); //Bailout when can not figure out offsets for times. return null; } @@ -444,17 +411,12 @@ protected static Map buildScanForTimesPre * * @return either full scan or an optimized sub scan. */ - private Map pushAndOp(ExprNodeGenericFuncDesc expr) - { + 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, KafkaPullerInputSplit.copyOf(input))); - for (ExprNodeDesc child : - expr.getChildren()) { + for (ExprNodeDesc child : expr.getChildren()) { Map scan = parse(child); if (scan != null) { Set currentKeys = ImmutableSet.copyOf(currentScan.keySet()); @@ -475,12 +437,9 @@ private Map pushAndOp(ExprNodeGenericFunc return currentScan; } - @Nullable - private Map pushOrOp(ExprNodeGenericFuncDesc expr) - { + @Nullable private Map pushOrOp(ExprNodeGenericFuncDesc expr) { final Map currentScan = new HashMap<>(); - for (ExprNodeDesc child : - expr.getChildren()) { + for (ExprNodeDesc child : expr.getChildren()) { Map scan = parse(child); if (scan == null) { // if any of the children is unknown bailout @@ -495,8 +454,7 @@ private Map pushOrOp(ExprNodeGenericFuncD return currentScan; } - private static ExprNodeDesc getColumnExpr(ExprNodeDesc expr) - { + private static ExprNodeDesc getColumnExpr(ExprNodeDesc expr) { if (expr instanceof ExprNodeColumnDesc) { return expr; } @@ -509,12 +467,15 @@ private static ExprNodeDesc getColumnExpr(ExprNodeDesc expr) } GenericUDF udf = funcDesc.getGenericUDF(); // check if its a simple cast expression. - if ((udf instanceof GenericUDFBridge || udf instanceof GenericUDFToBinary - || udf instanceof GenericUDFToChar || udf instanceof GenericUDFToVarchar - || udf instanceof GenericUDFToDecimal || udf instanceof GenericUDFToDate - || udf instanceof GenericUDFToUnixTimeStamp || udf instanceof GenericUDFToUtcTimestamp) - && funcDesc.getChildren().size() == 1 - && funcDesc.getChildren().get(0) instanceof ExprNodeColumnDesc) { + if ((udf instanceof GenericUDFBridge + || udf instanceof GenericUDFToBinary + || udf instanceof GenericUDFToChar + || udf instanceof GenericUDFToVarchar + || udf instanceof GenericUDFToDecimal + || udf instanceof GenericUDFToDate + || udf instanceof GenericUDFToUnixTimeStamp + || udf instanceof GenericUDFToUtcTimestamp) && funcDesc.getChildren().size() == 1 && funcDesc.getChildren() + .get(0) instanceof ExprNodeColumnDesc) { return expr.getChildren().get(0); } return expr; diff --git a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaStorageHandler.java b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaStorageHandler.java index 0639707e733f..ca26045e8a74 100644 --- a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaStorageHandler.java +++ b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaStorageHandler.java @@ -7,7 +7,7 @@ * "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 + * 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, @@ -38,152 +38,108 @@ import java.util.HashMap; import java.util.Map; -public class KafkaStorageHandler implements HiveStorageHandler -{ +/** + * Hive Kafka storage handler to allow user querying Stream of tuples from a Kafka queue. + */ +public class KafkaStorageHandler implements HiveStorageHandler { - public static final String __TIMESTAMP = "__timestamp"; - public static final String __PARTITION = "__partition"; - public static final String __OFFSET = "__offset"; + public static final String TIMESTAMP_COLUMN = "__timestamp"; + public static final String PARTITION_COLUMN = "__partition"; + public static final String OFFSET_COLUMN = "__offset"; public static final String SERDE_CLASS_NAME = "kafka.serde.class"; public static final String HIVE_KAFKA_TOPIC = "kafka.topic"; public static final String CONSUMER_CONFIGURATION_PREFIX = "kafka.consumer"; public static final String HIVE_KAFKA_BOOTSTRAP_SERVERS = "kafka.bootstrap.servers"; public static final String HIVE_KAFKA_POLL_TIMEOUT = "hive.kafka.poll.timeout.ms"; - public static final long DEFAULT_CONSUMER_POLL_TIMEOUT_MS = 5000l; // 5 seconds + public static final long DEFAULT_CONSUMER_POLL_TIMEOUT_MS = 5000L; // 5 seconds - private static final Logger log = LoggerFactory.getLogger(KafkaStorageHandler.class); + private static final Logger LOG = LoggerFactory.getLogger(KafkaStorageHandler.class); Configuration configuration; - @Override - public Class getInputFormatClass() - { + @Override public Class getInputFormatClass() { return KafkaPullerInputFormat.class; } - @Override - public Class getOutputFormatClass() - { + @Override public Class getOutputFormatClass() { return NullOutputFormat.class; } - @Override - public Class getSerDeClass() - { + @Override public Class getSerDeClass() { return GenericKafkaSerDe.class; } - @Override - public HiveMetaHook getMetaHook() - { + @Override public HiveMetaHook getMetaHook() { return null; } - @Override - public HiveAuthorizationProvider getAuthorizationProvider() throws HiveException - { + @Override public HiveAuthorizationProvider getAuthorizationProvider() throws HiveException { return new DefaultHiveAuthorizationProvider(); } - @Override - public void configureInputJobProperties( - TableDesc tableDesc, - Map jobProperties - ) - { - jobProperties.put(HIVE_KAFKA_TOPIC, Preconditions - .checkNotNull( - tableDesc.getProperties().getProperty(HIVE_KAFKA_TOPIC), - "kafka topic missing set table property->" + HIVE_KAFKA_TOPIC - )); - log.debug("Table properties: Kafka Topic {}", tableDesc.getProperties().getProperty(HIVE_KAFKA_TOPIC)); - jobProperties.put(HIVE_KAFKA_BOOTSTRAP_SERVERS, Preconditions - .checkNotNull( - tableDesc.getProperties().getProperty(HIVE_KAFKA_BOOTSTRAP_SERVERS), - "Broker address missing set table property->" + HIVE_KAFKA_BOOTSTRAP_SERVERS - )); - log.debug("Table properties: Kafka broker {}", tableDesc.getProperties().getProperty(HIVE_KAFKA_BOOTSTRAP_SERVERS)); - jobProperties.put( - SERDE_CLASS_NAME, - tableDesc.getProperties().getProperty(SERDE_CLASS_NAME, KafkaJsonSerDe.class.getName()) - ); - - log.info("Table properties: SerDe class name {}", jobProperties.get(SERDE_CLASS_NAME)); + @Override public void configureInputJobProperties(TableDesc tableDesc, Map jobProperties) { + jobProperties.put(HIVE_KAFKA_TOPIC, + Preconditions.checkNotNull(tableDesc.getProperties().getProperty(HIVE_KAFKA_TOPIC), + "kafka topic missing set table property->" + HIVE_KAFKA_TOPIC)); + LOG.debug("Table properties: Kafka Topic {}", tableDesc.getProperties().getProperty(HIVE_KAFKA_TOPIC)); + jobProperties.put(HIVE_KAFKA_BOOTSTRAP_SERVERS, + Preconditions.checkNotNull(tableDesc.getProperties().getProperty(HIVE_KAFKA_BOOTSTRAP_SERVERS), + "Broker address missing set table property->" + HIVE_KAFKA_BOOTSTRAP_SERVERS)); + LOG.debug("Table properties: Kafka broker {}", tableDesc.getProperties().getProperty(HIVE_KAFKA_BOOTSTRAP_SERVERS)); + jobProperties.put(SERDE_CLASS_NAME, + tableDesc.getProperties().getProperty(SERDE_CLASS_NAME, KafkaJsonSerDe.class.getName())); + + LOG.info("Table properties: SerDe class name {}", jobProperties.get(SERDE_CLASS_NAME)); //set extra properties tableDesc.getProperties() - .entrySet() - .stream() - .filter( - objectObjectEntry -> objectObjectEntry.getKey() - .toString() - .toLowerCase() - .startsWith(CONSUMER_CONFIGURATION_PREFIX)) - .forEach(entry -> { - String key = entry.getKey().toString().substring(CONSUMER_CONFIGURATION_PREFIX.length() + 1); - String value = entry.getValue().toString(); - jobProperties.put(key, value); - log.info("Setting extra job properties: key [{}] -> value [{}]", key, value); - - }); + .entrySet() + .stream() + .filter(objectObjectEntry -> objectObjectEntry.getKey() + .toString() + .toLowerCase() + .startsWith(CONSUMER_CONFIGURATION_PREFIX)) + .forEach(entry -> { + String key = entry.getKey().toString().substring(CONSUMER_CONFIGURATION_PREFIX.length() + 1); + 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 - ) - { + @Override public void configureInputJobCredentials(TableDesc tableDesc, Map secrets) { } - @Override - public void configureOutputJobProperties( - TableDesc tableDesc, - Map jobProperties - ) - { + @Override public void configureOutputJobProperties(TableDesc tableDesc, Map jobProperties) { } - @Override - public void configureTableJobProperties( - TableDesc tableDesc, - Map jobProperties - ) - { + @Override public void configureTableJobProperties(TableDesc tableDesc, Map jobProperties) { configureInputJobProperties(tableDesc, jobProperties); } - @Override - public void configureJobConf(TableDesc tableDesc, JobConf jobConf) - { + @Override public void configureJobConf(TableDesc tableDesc, JobConf jobConf) { Map properties = new HashMap<>(); configureInputJobProperties(tableDesc, properties); properties.forEach((key, value) -> jobConf.set(key, value)); try { KafkaStreamingUtils.copyDependencyJars(jobConf, KafkaStorageHandler.class); - } - catch (IOException e) { + } catch (IOException e) { throw new RuntimeException(e); } } - @Override - public void setConf(Configuration configuration) - { + @Override public void setConf(Configuration configuration) { this.configuration = configuration; } - @Override - public Configuration getConf() - { + @Override public Configuration getConf() { return configuration; } - @Override - public String toString() - { + @Override public String toString() { return "org.apache.hadoop.hive.kafka.KafkaStorageHandler"; } } diff --git a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaStreamingUtils.java b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaStreamingUtils.java index e5c83f935e57..132db9880dc8 100644 --- a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaStreamingUtils.java +++ b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaStreamingUtils.java @@ -7,7 +7,7 @@ * "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 + * 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, @@ -37,13 +37,13 @@ import static org.apache.kafka.clients.consumer.ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG; import static org.apache.kafka.clients.consumer.ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG; - /** - * Utilities class + * Utilities class. */ -public class KafkaStreamingUtils { +public final class KafkaStreamingUtils { - private KafkaStreamingUtils() {} + private KafkaStreamingUtils() { + } /** * @param configuration Job configs @@ -53,9 +53,9 @@ private KafkaStreamingUtils() {} public static Properties consumerProperties(Configuration configuration) { final Properties props = new Properties(); // those are very important to set to avoid long blocking - props.setProperty("request.timeout.ms", "10001" ); - props.setProperty("fetch.max.wait.ms", "10000" ); - props.setProperty("session.timeout.ms", "10000" ); + props.setProperty("request.timeout.ms", "10001"); + props.setProperty("fetch.max.wait.ms", "10000"); + props.setProperty("session.timeout.ms", "10000"); // we are managing the commit offset props.setProperty("enable.auto.commit", "false"); // we are seeking in the stream so no reset @@ -65,38 +65,37 @@ public static Properties consumerProperties(Configuration configuration) { props.setProperty(KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName()); props.setProperty(VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName()); // user can always override stuff - final Map kafkaProperties = + final Map + kafkaProperties = configuration.getValByRegex("^" + KafkaStorageHandler.CONSUMER_CONFIGURATION_PREFIX + "\\..*"); for (Map.Entry entry : kafkaProperties.entrySet()) { - props.setProperty(entry.getKey().substring( - KafkaStorageHandler.CONSUMER_CONFIGURATION_PREFIX.length() + 1), - entry.getValue() - ); + props.setProperty(entry.getKey().substring(KafkaStorageHandler.CONSUMER_CONFIGURATION_PREFIX.length() + 1), + entry.getValue()); } return props; } - public static void copyDependencyJars(Configuration conf, Class... classes) throws IOException - { + public 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.asList(classes).stream().filter(aClass -> aClass != null).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())); + jars.addAll(Arrays.asList(classes).stream().filter(aClass -> aClass != null) + .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; diff --git a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/package-info.java b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/package-info.java new file mode 100644 index 000000000000..8a0d8fd0b050 --- /dev/null +++ b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/package-info.java @@ -0,0 +1,23 @@ +/* + * 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 info file. + */ + +package org.apache.hadoop.hive.kafka; diff --git a/kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaPullerInputSplitTest.java b/kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaPullerInputSplitTest.java index aa3aba7e3ed4..be269868187f 100644 --- a/kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaPullerInputSplitTest.java +++ b/kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaPullerInputSplitTest.java @@ -29,21 +29,18 @@ import java.io.IOException; import java.util.List; -public class KafkaPullerInputSplitTest -{ +/** + * Kafka Hadoop InputSplit Test. + */ +public class KafkaPullerInputSplitTest { private String topic = "my_topic"; private KafkaPullerInputSplit expectedInputSplit; - public KafkaPullerInputSplitTest() - { - this.expectedInputSplit = new KafkaPullerInputSplit(this.topic, 1, 50L, 56L, - new Path("/tmp") - ); + public KafkaPullerInputSplitTest() { + this.expectedInputSplit = new KafkaPullerInputSplit(this.topic, 1, 50L, 56L, new Path("/tmp")); } - @Test - public void testWriteRead() throws IOException - { + @Test public void testWriteRead() throws IOException { DataOutput output = new DataOutputBuffer(); this.expectedInputSplit.write(output); KafkaPullerInputSplit kafkaPullerInputSplit = new KafkaPullerInputSplit(); @@ -53,132 +50,59 @@ public void testWriteRead() throws IOException Assert.assertEquals(this.expectedInputSplit, kafkaPullerInputSplit); } + @Test public void andRangeOverLapping() { + KafkaPullerInputSplit kafkaPullerInputSplit = new KafkaPullerInputSplit("test-topic", 2, 10, 400, new Path("/tmp")); - @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) - ); + 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")); - @Test - public void andRangeNonOverLapping() - { - KafkaPullerInputSplit kafkaPullerInputSplit = new KafkaPullerInputSplit( - "test-topic", - 2, - 10, - 400, - new Path("/tmp") - ); + Assert.assertEquals(null, KafkaPullerInputSplit.intersectRange(kafkaPullerInputSplit, kafkaPullerInputSplit2)); - KafkaPullerInputSplit kafkaPullerInputSplit2 = new KafkaPullerInputSplit( - "test-topic", - 2, - 550, - 700, - new Path("/tmp") - ); + } + @Test public void orRange() { + KafkaPullerInputSplit + kafkaPullerInputSplit = + new KafkaPullerInputSplit("test-topic", 2, 300, 400, new Path("/tmp")); - Assert.assertEquals(null, KafkaPullerInputSplit.intersectRange(kafkaPullerInputSplit, kafkaPullerInputSplit2)); + 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")); - @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)); + 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") - ); + @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") - ); + @Test public void testClone() { + KafkaPullerInputSplit + kafkaPullerInputSplit = + new KafkaPullerInputSplit("test-topic", 2, 300, 400, new Path("/tmp")); KafkaPullerInputSplit clone = kafkaPullerInputSplit.clone(); Assert.assertEquals(kafkaPullerInputSplit, clone); @@ -186,32 +110,22 @@ public void TestClone() } - @Test - public void testSlice() - { - KafkaPullerInputSplit kafkaPullerInputSplit = new KafkaPullerInputSplit( - "test-topic", - 2, - 300, - 400, - new Path("/tmp") - ); + @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.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); + .filter(kafkaPullerInputSplit1 -> kafkaPullerInputSplit.getStartOffset() + == kafkaPullerInputSplit1.getStartOffset()) + .count() == 1); Assert.assertTrue(kafkaPullerInputSplitList.stream() - .filter(kafkaPullerInputSplit1 -> kafkaPullerInputSplit.getEndOffset() - == kafkaPullerInputSplit1.getEndOffset()) - .count() == 1); + .filter(kafkaPullerInputSplit1 -> kafkaPullerInputSplit.getEndOffset() == kafkaPullerInputSplit1.getEndOffset()) + .count() == 1); } } diff --git a/kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaRecordIteratorTest.java b/kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaRecordIteratorTest.java index bc3141c718ba..41c31a47b4ae 100644 --- a/kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaRecordIteratorTest.java +++ b/kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaRecordIteratorTest.java @@ -40,7 +40,7 @@ import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.record.TimestampType; +import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.serialization.ByteArrayDeserializer; import org.apache.kafka.common.utils.Time; import org.junit.After; @@ -61,13 +61,15 @@ import java.util.Properties; import java.util.stream.Collectors; -public class KafkaRecordIteratorTest -{ - private static final Logger log = LoggerFactory.getLogger(KafkaRecordIteratorTest.class); - private static final String topic = "my_topic2"; +/** + * Kafka Iterator Tests. + */ +public class KafkaRecordIteratorTest { + private static final Logger LOG = LoggerFactory.getLogger(KafkaRecordIteratorTest.class); + private static final String TOPIC = "my_test_topic"; private static final List> RECORDS = new ArrayList(); private static final int RECORD_NUMBER = 100; - private static final TopicPartition TOPIC_PARTITION = new TopicPartition("my_topic2", 0); + private static final TopicPartition TOPIC_PARTITION = new TopicPartition(TOPIC, 0); public static final byte[] KEY_BYTES = "KEY".getBytes(Charset.forName("UTF-8")); private static ZkUtils zkUtils; private static ZkClient zkClient; @@ -77,122 +79,94 @@ public class KafkaRecordIteratorTest private KafkaConsumer consumer = null; private KafkaRecordIterator kafkaRecordIterator = null; private Configuration conf = new Configuration(); + private static EmbeddedZookeeper zkServer; - - public KafkaRecordIteratorTest() - { + public KafkaRecordIteratorTest() { } - @BeforeClass - public static void setupCluster() throws IOException, InterruptedException - { - log.info("init embedded Zookeeper"); - EmbeddedZookeeper zkServer = new EmbeddedZookeeper(); + @BeforeClass public static void setupCluster() throws IOException, InterruptedException { + LOG.info("init embedded Zookeeper"); + zkServer = new EmbeddedZookeeper(); zkConnect = "127.0.0.1:" + zkServer.port(); - zkClient = new ZkClient(zkConnect, 30000, 30000, ZKStringSerializer$.MODULE$); + zkClient = new ZkClient(zkConnect, 3000, 3000, ZKStringSerializer$.MODULE$); zkUtils = ZkUtils.apply(zkClient, false); - log.info("init kafka broker"); + LOG.info("init kafka broker"); Properties brokerProps = new Properties(); brokerProps.setProperty("zookeeper.connect", zkConnect); brokerProps.setProperty("broker.id", "0"); - brokerProps.setProperty("log.dirs", Files.createTempDirectory("kafka-").toAbsolutePath().toString()); + 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"); - brokerProps.setProperty("log.retention.ms", "1000000"); + brokerProps.setProperty("offsets.TOPIC.replication.factor", "1"); KafkaConfig config = new KafkaConfig(brokerProps); Time mock = new MockTime(); kafkaServer = TestUtils.createServer(config, mock); - log.info("Creating kafka topic [{}]", "my_topic2"); - AdminUtils.createTopic(zkUtils, topic, 1, 1, new Properties(), RackAwareMode.Disabled$.MODULE$); + kafkaServer.startup(); + LOG.info("Creating kafka TOPIC [{}]", TOPIC); + AdminUtils.createTopic(zkUtils, TOPIC, 1, 1, new Properties(), RackAwareMode.Disabled$.MODULE$); setupProducer(); sendData(); } - @Before - public void setUp() - { - log.info("setting up consumer"); + @Before public void setUp() { + LOG.info("setting up consumer"); this.setupConsumer(); this.kafkaRecordIterator = null; } - @Test - public void testHasNextAbsoluteStartEnd() - { + @Test public void testHasNextAbsoluteStartEnd() { this.kafkaRecordIterator = new KafkaRecordIterator(this.consumer, TOPIC_PARTITION, 0L, (long) RECORDS.size(), 100L); this.compareIterator(RECORDS, this.kafkaRecordIterator); } - @Test - public void testHasNextGivenStartEnd() - { - long startOffset = 2L; - long lastOffset = 4L; + @Test public void testHasNextGivenStartEnd() { this.kafkaRecordIterator = new KafkaRecordIterator(this.consumer, TOPIC_PARTITION, 2L, 4L, 100L); - this.compareIterator((List) RECORDS.stream().filter((consumerRecord) -> { - return consumerRecord.offset() >= 2L && consumerRecord.offset() < 4L; - }).collect(Collectors.toList()), this.kafkaRecordIterator); + this.compareIterator(RECORDS.stream() + .filter((consumerRecord) -> consumerRecord.offset() >= 2L && consumerRecord.offset() < 4L) + .collect(Collectors.toList()), this.kafkaRecordIterator); } - @Test - public void testHasNextNoOffsets() - { + @Test public void testHasNextNoOffsets() { this.kafkaRecordIterator = new KafkaRecordIterator(this.consumer, TOPIC_PARTITION, 100L); this.compareIterator(RECORDS, this.kafkaRecordIterator); } - @Test - public void testHasNextLastRecord() - { + @Test public void testHasNextLastRecord() { long startOffset = (long) (RECORDS.size() - 1); long lastOffset = (long) RECORDS.size(); this.kafkaRecordIterator = new KafkaRecordIterator(this.consumer, TOPIC_PARTITION, startOffset, lastOffset, 100L); - this.compareIterator((List) RECORDS.stream().filter((consumerRecord) -> { - return consumerRecord.offset() >= startOffset && consumerRecord.offset() < lastOffset; - }).collect(Collectors.toList()), this.kafkaRecordIterator); + this.compareIterator(RECORDS.stream() + .filter((consumerRecord) -> consumerRecord.offset() >= startOffset && consumerRecord.offset() < lastOffset) + .collect(Collectors.toList()), this.kafkaRecordIterator); } - @Test - public void testHasNextFirstRecord() - { - long startOffset = 0L; - long lastOffset = 1L; + @Test public void testHasNextFirstRecord() { this.kafkaRecordIterator = new KafkaRecordIterator(this.consumer, TOPIC_PARTITION, 0L, 1L, 100L); - this.compareIterator((List) RECORDS.stream().filter((consumerRecord) -> { - return consumerRecord.offset() >= 0L && consumerRecord.offset() < 1L; - }).collect(Collectors.toList()), this.kafkaRecordIterator); + this.compareIterator(RECORDS.stream() + .filter((consumerRecord) -> consumerRecord.offset() >= 0L && consumerRecord.offset() < 1L) + .collect(Collectors.toList()), this.kafkaRecordIterator); } - @Test - public void testHasNextNoStart() - { - long startOffset = 0L; - long lastOffset = 10L; + @Test public void testHasNextNoStart() { this.kafkaRecordIterator = new KafkaRecordIterator(this.consumer, TOPIC_PARTITION, (Long) null, 10L, 100L); - this.compareIterator((List) RECORDS.stream().filter((consumerRecord) -> { - return consumerRecord.offset() >= 0L && consumerRecord.offset() < 10L; - }).collect(Collectors.toList()), this.kafkaRecordIterator); + this.compareIterator(RECORDS.stream() + .filter((consumerRecord) -> consumerRecord.offset() >= 0L && consumerRecord.offset() < 10L) + .collect(Collectors.toList()), this.kafkaRecordIterator); } - @Test - public void testHasNextNoEnd() - { - long startOffset = 5L; + @Test public void testHasNextNoEnd() { long lastOffset = (long) RECORDS.size(); this.kafkaRecordIterator = new KafkaRecordIterator(this.consumer, TOPIC_PARTITION, 5L, (Long) null, 100L); - this.compareIterator((List) RECORDS.stream().filter((consumerRecord) -> { - return consumerRecord.offset() >= 5L && consumerRecord.offset() < lastOffset; - }).collect(Collectors.toList()), this.kafkaRecordIterator); + this.compareIterator(RECORDS.stream() + .filter((consumerRecord) -> consumerRecord.offset() >= 5L && consumerRecord.offset() < lastOffset) + .collect(Collectors.toList()), this.kafkaRecordIterator); } - @Test - public void testRecordReader() throws IOException, InterruptedException - { - InputSplit inputSplits = new KafkaPullerInputSplit("my_topic2", 0, 0L, 50L, null); + @Test public void testRecordReader() throws IOException { + InputSplit inputSplits = new KafkaPullerInputSplit(TOPIC, 0, 0L, 50L, null); KafkaPullerRecordReader recordReader = new KafkaPullerRecordReader((KafkaPullerInputSplit) inputSplits, this.conf); - List serRecords = (List) RECORDS.stream().map((recordx) -> { - return KafkaRecordWritable.fromKafkaRecord(recordx); - }).collect(Collectors.toList()); + List + serRecords = + RECORDS.stream().map((aRecord) -> KafkaRecordWritable.fromKafkaRecord(aRecord)).collect(Collectors.toList()); for (int i = 0; i < 50; ++i) { KafkaRecordWritable record = new KafkaRecordWritable(); @@ -203,95 +177,56 @@ public void testRecordReader() throws IOException, InterruptedException recordReader.close(); recordReader = new KafkaPullerRecordReader(); TaskAttemptContext context = new TaskAttemptContextImpl(this.conf, new TaskAttemptID()); - recordReader.initialize(new KafkaPullerInputSplit("my_topic2", 0, 50L, 100L, null), context); + recordReader.initialize(new KafkaPullerInputSplit(TOPIC, 0, 50L, 100L, null), context); for (int i = 50; i < 100; ++i) { KafkaRecordWritable record = new KafkaRecordWritable(); - Assert.assertTrue(recordReader.next((NullWritable) null, record)); + Assert.assertTrue(recordReader.next(null, record)); Assert.assertEquals(serRecords.get(i), record); } recordReader.close(); } - @Test( - expected = IllegalStateException.class - ) - public void testPullingBeyondLimit() - { - this.kafkaRecordIterator = new KafkaRecordIterator( - this.consumer, - TOPIC_PARTITION, - 0L, - (long) RECORDS.size() + 1L, - 100L - ); + @Test(expected = TimeoutException.class) public void testPullingBeyondLimit() { + this.kafkaRecordIterator = + new KafkaRecordIterator(this.consumer, TOPIC_PARTITION, 0L, 101L, 100L); this.compareIterator(RECORDS, this.kafkaRecordIterator); } - @Test( - expected = IllegalStateException.class - ) - public void testPullingStartGreaterThanEnd() - { + @Test(expected = IllegalStateException.class) public void testPullingStartGreaterThanEnd() { this.kafkaRecordIterator = new KafkaRecordIterator(this.consumer, TOPIC_PARTITION, 10L, 1L, 100L); this.compareIterator(RECORDS, this.kafkaRecordIterator); } - @Test( - expected = IllegalStateException.class - ) - public void testPullingFromEmptyTopic() - { + @Test(expected = TimeoutException.class) public void testPullingFromEmptyTopic() { this.kafkaRecordIterator = new KafkaRecordIterator(this.consumer, new TopicPartition("noHere", 0), 0L, 100L, 100L); this.compareIterator(RECORDS, this.kafkaRecordIterator); } - @Test( - expected = IllegalStateException.class - ) - public void testPullingFromEmptyPartition() - { - this.kafkaRecordIterator = new KafkaRecordIterator( - this.consumer, - new TopicPartition("my_topic2", 1), - 0L, - 100L, - 100L - ); + @Test(expected = TimeoutException.class) public void testPullingFromEmptyPartition() { + this.kafkaRecordIterator = + new KafkaRecordIterator(this.consumer, new TopicPartition(TOPIC, 1), 0L, 100L, 100L); this.compareIterator(RECORDS, this.kafkaRecordIterator); } - @Test - public void testStartIsEqualEnd() - { + @Test public void testStartIsEqualEnd() { this.kafkaRecordIterator = new KafkaRecordIterator(this.consumer, TOPIC_PARTITION, 10L, 10L, 100L); this.compareIterator(ImmutableList.of(), this.kafkaRecordIterator); } - - @Test - public void testStartIsTheLastOffset() - { - this.kafkaRecordIterator = new KafkaRecordIterator( - this.consumer, - TOPIC_PARTITION, - new Long(RECORD_NUMBER), - new Long(RECORD_NUMBER), - 100L - ); + @Test public void testStartIsTheLastOffset() { + this.kafkaRecordIterator = + new KafkaRecordIterator(this.consumer, TOPIC_PARTITION, new Long(RECORD_NUMBER), new Long(RECORD_NUMBER), 100L); this.compareIterator(ImmutableList.of(), this.kafkaRecordIterator); } - private void compareIterator( - List> expected, - Iterator> kafkaRecordIterator - ) - { + private void compareIterator(List> expected, + Iterator> kafkaRecordIterator) { expected.stream().forEachOrdered((expectedRecord) -> { Assert.assertTrue("record with offset " + expectedRecord.offset(), kafkaRecordIterator.hasNext()); ConsumerRecord record = kafkaRecordIterator.next(); - Assert.assertTrue(record.topic().equals(topic)); + Assert.assertTrue(record.topic().equals(TOPIC)); Assert.assertTrue(record.partition() == 0); Assert.assertEquals("Offsets not matching", expectedRecord.offset(), record.offset()); byte[] binaryExceptedValue = expectedRecord.value(); @@ -304,20 +239,18 @@ private void compareIterator( Assert.assertFalse(kafkaRecordIterator.hasNext()); } - private static void setupProducer() - { - log.info("Setting up kafka producer"); + 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"); + LOG.info("kafka producer started"); } - private void setupConsumer() - { + private void setupConsumer() { Properties consumerProps = new Properties(); consumerProps.setProperty("enable.auto.commit", "false"); consumerProps.setProperty("auto.offset.reset", "none"); @@ -332,57 +265,36 @@ private void setupConsumer() this.consumer = new KafkaConsumer(consumerProps); } - private static void sendData() throws InterruptedException - { - log.info("Sending {} records", RECORD_NUMBER); + private static void sendData() throws InterruptedException { + LOG.info("Sending {} records", RECORD_NUMBER); RECORDS.clear(); for (int i = 0; i < RECORD_NUMBER; ++i) { final byte[] value = ("VALUE-" + Integer.toString(i)).getBytes(Charset.forName("UTF-8")); //noinspection unchecked - producer.send(new ProducerRecord( - topic, - 0, - 0L, - KEY_BYTES, - value - )); + producer.send(new ProducerRecord(TOPIC, 0, 0L, KEY_BYTES, value)); //noinspection unchecked - RECORDS.add(new ConsumerRecord( - topic, - 0, - (long) i, - 0L, - (TimestampType) null, - 0L, - 0, - 0, - KEY_BYTES, - value - )); + RECORDS.add(new ConsumerRecord(TOPIC, 0, (long) i, 0L, null, 0L, 0, 0, KEY_BYTES, value)); } producer.close(); } - @After - public void tearDown() - { + @After public void tearDown() { this.kafkaRecordIterator = null; if (this.consumer != null) { this.consumer.close(); } } - @AfterClass - public static void tearDownCluster() - { + @AfterClass public static void tearDownCluster() { if (kafkaServer != null) { kafkaServer.shutdown(); + kafkaServer.zkUtils().close(); kafkaServer.awaitShutdown(); } - + zkServer.shutdown(); zkClient.close(); zkUtils.close(); } diff --git a/kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaRecordWritableTest.java b/kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaRecordWritableTest.java index 1327eb33117f..e28c924a9ad9 100644 --- a/kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaRecordWritableTest.java +++ b/kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaRecordWritableTest.java @@ -28,12 +28,14 @@ import java.io.DataOutputStream; import java.io.IOException; +/** + * Test class for kafka Writable. + */ public class KafkaRecordWritableTest { public KafkaRecordWritableTest() { } - @Test - public void testWriteReadFields() throws IOException { + @Test public void testWriteReadFields() throws IOException { ConsumerRecord record = new ConsumerRecord("topic", 0, 3L, "key".getBytes(), "value".getBytes()); KafkaRecordWritable kafkaRecordWritable = KafkaRecordWritable.fromKafkaRecord(record); ByteArrayOutputStream baos = new ByteArrayOutputStream(); diff --git a/kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaScanTrimmerTest.java b/kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaScanTrimmerTest.java index 232bfcbb96b1..c7e0e1f3eebe 100644 --- a/kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaScanTrimmerTest.java +++ b/kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaScanTrimmerTest.java @@ -46,8 +46,10 @@ import static org.junit.Assert.assertNotNull; -public class KafkaScanTrimmerTest -{ +/** + * Test Class for Kafka Trimmer Class. + */ +public class KafkaScanTrimmerTest { private static final Path PATH = new Path("/tmp"); private ExprNodeDesc zeroInt = ConstantExprBuilder.build(0); @@ -57,167 +59,131 @@ public class KafkaScanTrimmerTest private ExprNodeDesc seventyFiveLong = ConstantExprBuilder.build(75L); private ExprNodeDesc fortyLong = ConstantExprBuilder.build(40L); - private ExprNodeDesc partitionColumn = new ExprNodeColumnDesc( - TypeInfoFactory.intTypeInfo, - KafkaStorageHandler.__PARTITION, - null, - false - ); - private ExprNodeDesc offsetColumn = new ExprNodeColumnDesc( - TypeInfoFactory.longTypeInfo, - KafkaStorageHandler.__OFFSET, - null, - false - ); - /*private ExprNodeDesc timestampColumn = new ExprNodeColumnDesc( - TypeInfoFactory.longTypeInfo, - KafkaJsonSerDe.__TIMESTAMP, - null, - false - );*/ + private ExprNodeDesc + partitionColumn = + new ExprNodeColumnDesc(TypeInfoFactory.intTypeInfo, KafkaStorageHandler.PARTITION_COLUMN, null, false); + private ExprNodeDesc + offsetColumn = + new ExprNodeColumnDesc(TypeInfoFactory.longTypeInfo, KafkaStorageHandler.OFFSET_COLUMN, null, false); private String topic = "my_topic"; - private Map fullHouse = ImmutableMap.of( - new TopicPartition(topic, 0), - new KafkaPullerInputSplit( - topic, - 0, - 0, - 45, - PATH - ), - new TopicPartition(topic, 1), - new KafkaPullerInputSplit( - topic, - 1, - 5, - 1005, - PATH - ), - new TopicPartition(topic, 2), - new KafkaPullerInputSplit( - topic, - 2, - 9, - 100, - PATH - ), - new TopicPartition(topic, 3), - new KafkaPullerInputSplit( - topic, - 3, - 0, - 100, - PATH - ) - ); - - @Test - public void computeOptimizedScanPartitionBinaryOpFilter() - { + private Map + fullHouse = + ImmutableMap.of(new TopicPartition(topic, 0), + new KafkaPullerInputSplit(topic, 0, 0, 45, PATH), + new TopicPartition(topic, 1), + new KafkaPullerInputSplit(topic, 1, 5, 1005, PATH), + new TopicPartition(topic, 2), + new KafkaPullerInputSplit(topic, 2, 9, 100, PATH), + new TopicPartition(topic, 3), + new KafkaPullerInputSplit(topic, 3, 0, 100, PATH)); + + @Test public void computeOptimizedScanPartitionBinaryOpFilter() { KafkaScanTrimmer kafkaScanTrimmer = new KafkaScanTrimmer(fullHouse, null); int partitionId = 2; ExprNodeDesc constant = ConstantExprBuilder.build(partitionId); final List children = Lists.newArrayList(partitionColumn, constant); - ExprNodeGenericFuncDesc node = EQ(children); + ExprNodeGenericFuncDesc node = eq(children); assertNotNull(node); - Map actual = kafkaScanTrimmer.computeOptimizedScan(SerializationUtilities.deserializeExpression( - SerializationUtilities.serializeExpression(node))); + Map + actual = + kafkaScanTrimmer.computeOptimizedScan(SerializationUtilities + .deserializeExpression(SerializationUtilities.serializeExpression(node))); Map expected = Maps.filterValues(fullHouse, tp -> Objects.requireNonNull(tp).getPartition() == partitionId); Assert.assertEquals(expected, actual); - ExprNodeGenericFuncDesc lessNode = LESS_THAN(children); + ExprNodeGenericFuncDesc lessNode = lessThan(children); assertNotNull(lessNode); - actual = kafkaScanTrimmer.computeOptimizedScan(SerializationUtilities.deserializeExpression(SerializationUtilities.serializeExpression( - lessNode))); + actual = + kafkaScanTrimmer.computeOptimizedScan(SerializationUtilities + .deserializeExpression(SerializationUtilities.serializeExpression(lessNode))); expected = Maps.filterValues(fullHouse, tp -> Objects.requireNonNull(tp).getPartition() < partitionId); Assert.assertEquals(expected, actual); - - ExprNodeGenericFuncDesc lessEqNode = LESS_THAN_EQ(children); + ExprNodeGenericFuncDesc lessEqNode = lessThanEq(children); assertNotNull(lessEqNode); - actual = kafkaScanTrimmer.computeOptimizedScan(SerializationUtilities.deserializeExpression(SerializationUtilities.serializeExpression( - lessEqNode))); + actual = + kafkaScanTrimmer.computeOptimizedScan(SerializationUtilities + .deserializeExpression(SerializationUtilities.serializeExpression(lessEqNode))); expected = Maps.filterValues(fullHouse, tp -> Objects.requireNonNull(tp).getPartition() <= partitionId); Assert.assertEquals(expected, actual); } - - @Test - public void computeOptimizedScanFalseFilter() - { + @Test public void computeOptimizedScanFalseFilter() { KafkaScanTrimmer kafkaScanTrimmer = new KafkaScanTrimmer(fullHouse, null); - ExprNodeGenericFuncDesc falseFilter = AND(Lists.newArrayList( - EQ(Lists.newArrayList(partitionColumn, zeroInt)), - EQ(Lists.newArrayList(partitionColumn, threeInt)) - )); + ExprNodeGenericFuncDesc + falseFilter = + and(Lists.newArrayList(eq(Lists.newArrayList(partitionColumn, zeroInt)), + eq(Lists.newArrayList(partitionColumn, threeInt)))); assertNotNull(falseFilter); - Map actual = kafkaScanTrimmer.computeOptimizedScan(SerializationUtilities.deserializeExpression( - SerializationUtilities.serializeExpression(falseFilter))); + Map + actual = + kafkaScanTrimmer.computeOptimizedScan(SerializationUtilities + .deserializeExpression(SerializationUtilities.serializeExpression(falseFilter))); Assert.assertTrue(actual.isEmpty()); - ExprNodeGenericFuncDesc falseFilter2 = AND(Lists.newArrayList( - EQ(Lists.newArrayList(offsetColumn, thirtyFiveLong)), - EQ(Lists.newArrayList(offsetColumn, fortyLong)) - )); + ExprNodeGenericFuncDesc + falseFilter2 = + and(Lists.newArrayList(eq(Lists.newArrayList(offsetColumn, thirtyFiveLong)), + eq(Lists.newArrayList(offsetColumn, fortyLong)))); assertNotNull(falseFilter2); - actual = kafkaScanTrimmer.computeOptimizedScan(SerializationUtilities.deserializeExpression( - SerializationUtilities.serializeExpression(falseFilter2))); + actual = + kafkaScanTrimmer.computeOptimizedScan(SerializationUtilities + .deserializeExpression(SerializationUtilities.serializeExpression(falseFilter2))); Assert.assertTrue(actual.isEmpty()); - ExprNodeGenericFuncDesc filter3 = OR(Lists.newArrayList(falseFilter, falseFilter2)); + ExprNodeGenericFuncDesc filter3 = or(Lists.newArrayList(falseFilter, falseFilter2)); assertNotNull(filter3); - actual = kafkaScanTrimmer.computeOptimizedScan(SerializationUtilities.deserializeExpression( - SerializationUtilities.serializeExpression(filter3))); + actual = + kafkaScanTrimmer.computeOptimizedScan(SerializationUtilities + .deserializeExpression(SerializationUtilities.serializeExpression(filter3))); Assert.assertTrue(actual.isEmpty()); - ExprNodeGenericFuncDesc filter4 = AND(Lists.newArrayList( - filter3, - EQ(Lists.newArrayList(partitionColumn, zeroInt)) - )); + ExprNodeGenericFuncDesc + filter4 = + and(Lists.newArrayList(filter3, eq(Lists.newArrayList(partitionColumn, zeroInt)))); assertNotNull(filter4); - actual = kafkaScanTrimmer.computeOptimizedScan(SerializationUtilities.deserializeExpression( - SerializationUtilities.serializeExpression(filter4))); + actual = + kafkaScanTrimmer.computeOptimizedScan(SerializationUtilities + .deserializeExpression(SerializationUtilities.serializeExpression(filter4))); Assert.assertTrue(actual.isEmpty()); } - @Test - public void computeOptimizedScanOrAndCombinedFilter() - { + @Test public void computeOptimizedScanOrAndCombinedFilter() { KafkaScanTrimmer kafkaScanTrimmer = new KafkaScanTrimmer(fullHouse, null); // partition = 0 and 30 <= offset < 35 or partition = 3 and 35 <= offset < 75 or (partition = 0 and offset = 40) + ExprNodeGenericFuncDesc + part1 = + and(Lists.newArrayList(greaterThanEq(Lists.newArrayList(offsetColumn, thirtyLong)), + eq(Lists.newArrayList(partitionColumn, zeroInt)), + lessThan(Lists.newArrayList(offsetColumn, thirtyFiveLong)))); - ExprNodeGenericFuncDesc part1 = AND(Lists.newArrayList( - GREATER_THAN_EQ(Lists.newArrayList(offsetColumn, thirtyLong)), - EQ(Lists.newArrayList(partitionColumn, zeroInt)), - LESS_THAN(Lists.newArrayList(offsetColumn, thirtyFiveLong)) - )); + ExprNodeGenericFuncDesc + part2 = + and(Lists.newArrayList(greaterThanEq(Lists.newArrayList(offsetColumn, thirtyFiveLong)), + eq(Lists.newArrayList(partitionColumn, threeInt)), + lessThan(Lists.newArrayList(offsetColumn, seventyFiveLong)))); - ExprNodeGenericFuncDesc part2 = AND(Lists.newArrayList( - GREATER_THAN_EQ(Lists.newArrayList(offsetColumn, thirtyFiveLong)), - EQ(Lists.newArrayList(partitionColumn, threeInt)), - LESS_THAN(Lists.newArrayList(offsetColumn, seventyFiveLong)) - )); + ExprNodeGenericFuncDesc + part3 = + and(Lists.newArrayList(eq(Lists.newArrayList(offsetColumn, fortyLong)), + eq(Lists.newArrayList(partitionColumn, zeroInt)))); - ExprNodeGenericFuncDesc part3 = AND(Lists.newArrayList( - EQ(Lists.newArrayList(offsetColumn, fortyLong)), - EQ(Lists.newArrayList(partitionColumn, zeroInt)) - )); - - ExprNodeGenericFuncDesc orExpression = OR(Lists.newArrayList(part1, part2, part3)); + ExprNodeGenericFuncDesc orExpression = or(Lists.newArrayList(part1, part2, part3)); assertNotNull(orExpression); - Map actual = kafkaScanTrimmer.computeOptimizedScan(SerializationUtilities.deserializeExpression( - SerializationUtilities.serializeExpression( - orExpression))); + Map + actual = + kafkaScanTrimmer.computeOptimizedScan(SerializationUtilities + .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); @@ -226,505 +192,378 @@ public void computeOptimizedScanOrAndCombinedFilter() Map expected = ImmutableMap.of(tpZero, split1, toThree, split2); Assert.assertEquals(expected, actual); - } - @Test - public void computeOptimizedScanPartitionOrAndCombinedFilter() - { + @Test public void computeOptimizedScanPartitionOrAndCombinedFilter() { KafkaScanTrimmer kafkaScanTrimmer = new KafkaScanTrimmer(fullHouse, null); // partition = 1 or (partition >2 and <= 3) - ExprNodeGenericFuncDesc eq = EQ(Lists.newArrayList(partitionColumn, ConstantExprBuilder.build(1))); - ExprNodeGenericFuncDesc lessEq = LESS_THAN_EQ(Lists.newArrayList(partitionColumn, ConstantExprBuilder.build(3))); - ExprNodeGenericFuncDesc greater = GREATER_THAN(Lists.newArrayList(partitionColumn, ConstantExprBuilder.build(2))); - ExprNodeGenericFuncDesc orNode = OR(Lists.newArrayList(AND(Lists.newArrayList(lessEq, greater)), eq)); - - Map actual = kafkaScanTrimmer.computeOptimizedScan(SerializationUtilities.deserializeExpression( - SerializationUtilities.serializeExpression(orNode))); - Map expected = Maps.filterValues( - fullHouse, - tp -> Objects.requireNonNull(tp).getPartition() == 1 || tp.getPartition() == 3 - ); + ExprNodeGenericFuncDesc eq = eq(Lists.newArrayList(partitionColumn, ConstantExprBuilder.build(1))); + ExprNodeGenericFuncDesc lessEq = lessThanEq(Lists.newArrayList(partitionColumn, ConstantExprBuilder.build(3))); + ExprNodeGenericFuncDesc greater = greaterThan(Lists.newArrayList(partitionColumn, ConstantExprBuilder.build(2))); + ExprNodeGenericFuncDesc orNode = or(Lists.newArrayList(and(Lists.newArrayList(lessEq, greater)), eq)); + + Map + actual = + kafkaScanTrimmer.computeOptimizedScan(SerializationUtilities + .deserializeExpression(SerializationUtilities.serializeExpression(orNode))); + Map + expected = + Maps.filterValues(fullHouse, tp -> Objects.requireNonNull(tp).getPartition() == 1 || tp.getPartition() == 3); Assert.assertEquals(expected, actual); assertNotNull(orNode); } - - @Test - public void buildScanFormPartitionPredicateEq() - { - Map actual = KafkaScanTrimmer.buildScanFormPartitionPredicate( - fullHouse, - PredicateLeaf.Operator.EQUALS, - 3, - false, - false - ); + @Test public void buildScanFormPartitionPredicateEq() { + Map + actual = + KafkaScanTrimmer.buildScanFormPartitionPredicate(fullHouse, PredicateLeaf.Operator.EQUALS, 3, false, false); TopicPartition topicPartition = new TopicPartition(topic, 3); Assert.assertEquals(fullHouse.get(topicPartition), actual.get(topicPartition)); } - @Test - public void buildScanFormPartitionPredicateLess() - { + @Test public void buildScanFormPartitionPredicateLess() { // partitionConst < partitionColumn (flip true) int partitionConst = 2; - Map actual = KafkaScanTrimmer.buildScanFormPartitionPredicate( - fullHouse, - PredicateLeaf.Operator.LESS_THAN, - partitionConst, - true, - false - ); - - Map expected = Maps.filterEntries( - fullHouse, - entry -> Objects.requireNonNull(entry).getKey().partition() > partitionConst - ); + Map + actual = + KafkaScanTrimmer.buildScanFormPartitionPredicate(fullHouse, + PredicateLeaf.Operator.LESS_THAN, + partitionConst, + true, + false); + + Map + expected = + Maps.filterEntries(fullHouse, entry -> Objects.requireNonNull(entry).getKey().partition() > partitionConst); Assert.assertEquals(expected, actual); Assert.assertFalse(actual.isEmpty()); // partitionConst >= partitionColumn (flip true, negation true) - actual = KafkaScanTrimmer.buildScanFormPartitionPredicate( - fullHouse, - PredicateLeaf.Operator.LESS_THAN, - partitionConst, - true, - true - ); - - expected = Maps.filterEntries( - fullHouse, - entry -> partitionConst >= Objects.requireNonNull(entry).getKey().partition() - ); + actual = + KafkaScanTrimmer.buildScanFormPartitionPredicate(fullHouse, + PredicateLeaf.Operator.LESS_THAN, + partitionConst, + true, + true); + + expected = + Maps.filterEntries(fullHouse, entry -> partitionConst >= Objects.requireNonNull(entry).getKey().partition()); Assert.assertEquals(expected, actual); Assert.assertFalse(actual.isEmpty()); // partitionColumn >= partitionConst (negation true) - actual = KafkaScanTrimmer.buildScanFormPartitionPredicate( - fullHouse, - PredicateLeaf.Operator.LESS_THAN, - partitionConst, - false, - true - ); - - expected = Maps.filterEntries( - fullHouse, - entry -> Objects.requireNonNull(entry).getKey().partition() >= partitionConst - ); + actual = + KafkaScanTrimmer.buildScanFormPartitionPredicate(fullHouse, + PredicateLeaf.Operator.LESS_THAN, + partitionConst, + false, + true); + + expected = + Maps.filterEntries(fullHouse, entry -> Objects.requireNonNull(entry).getKey().partition() >= partitionConst); Assert.assertEquals(expected, actual); Assert.assertFalse(actual.isEmpty()); // partitionColumn < partitionConst (negation true) - actual = KafkaScanTrimmer.buildScanFormPartitionPredicate( - fullHouse, - PredicateLeaf.Operator.LESS_THAN, - partitionConst, - false, - false - ); - - expected = Maps.filterEntries( - fullHouse, - entry -> Objects.requireNonNull(entry).getKey().partition() < partitionConst - ); + actual = + KafkaScanTrimmer.buildScanFormPartitionPredicate(fullHouse, + PredicateLeaf.Operator.LESS_THAN, + partitionConst, + false, + false); + + expected = + Maps.filterEntries(fullHouse, entry -> Objects.requireNonNull(entry).getKey().partition() < partitionConst); Assert.assertEquals(expected, actual); Assert.assertFalse(actual.isEmpty()); } - @Test - public void buildScanFormPartitionPredicateLessEq() - { + @Test public void buildScanFormPartitionPredicateLessEq() { // partitionConst <= partitionColumn (flip true) int partitionConst = 2; - Map actual = KafkaScanTrimmer.buildScanFormPartitionPredicate( - fullHouse, - PredicateLeaf.Operator.LESS_THAN_EQUALS, - partitionConst, - true, - false - ); - - Map expected = Maps.filterEntries( - fullHouse, - entry -> Objects.requireNonNull(entry).getKey().partition() >= partitionConst - ); + Map + actual = + KafkaScanTrimmer.buildScanFormPartitionPredicate(fullHouse, + PredicateLeaf.Operator.LESS_THAN_EQUALS, + partitionConst, + true, + false); + + Map + expected = + Maps.filterEntries(fullHouse, entry -> Objects.requireNonNull(entry).getKey().partition() >= partitionConst); Assert.assertEquals(expected, actual); Assert.assertFalse(actual.isEmpty()); // partitionConst > partitionColumn (flip true, negation true) - actual = KafkaScanTrimmer.buildScanFormPartitionPredicate( - fullHouse, - PredicateLeaf.Operator.LESS_THAN_EQUALS, - partitionConst, - true, - true - ); - - expected = Maps.filterEntries( - fullHouse, - entry -> partitionConst > Objects.requireNonNull(entry).getKey().partition() - ); + actual = + KafkaScanTrimmer.buildScanFormPartitionPredicate(fullHouse, + PredicateLeaf.Operator.LESS_THAN_EQUALS, + partitionConst, + true, + true); + + expected = + Maps.filterEntries(fullHouse, entry -> partitionConst > Objects.requireNonNull(entry).getKey().partition()); Assert.assertEquals(expected, actual); Assert.assertFalse(actual.isEmpty()); - // partitionColumn > partitionConst (negation true) - actual = KafkaScanTrimmer.buildScanFormPartitionPredicate( - fullHouse, - PredicateLeaf.Operator.LESS_THAN_EQUALS, - partitionConst, - false, - true - ); - - expected = Maps.filterEntries(fullHouse, entry -> Objects.requireNonNull(entry).getKey().partition() > partitionConst); + actual = + KafkaScanTrimmer.buildScanFormPartitionPredicate(fullHouse, + PredicateLeaf.Operator.LESS_THAN_EQUALS, + partitionConst, + false, + true); + + expected = + Maps.filterEntries(fullHouse, entry -> Objects.requireNonNull(entry).getKey().partition() > partitionConst); Assert.assertEquals(expected, actual); Assert.assertFalse(actual.isEmpty()); // partitionColumn <= partitionConst (negation true) - actual = KafkaScanTrimmer.buildScanFormPartitionPredicate( - fullHouse, - PredicateLeaf.Operator.LESS_THAN_EQUALS, - partitionConst, - false, - false - ); - - expected = Maps.filterEntries( - fullHouse, - entry -> Objects.requireNonNull(entry).getKey().partition() <= partitionConst - ); + actual = + KafkaScanTrimmer.buildScanFormPartitionPredicate(fullHouse, + PredicateLeaf.Operator.LESS_THAN_EQUALS, + partitionConst, + false, + false); + + expected = + Maps.filterEntries(fullHouse, entry -> Objects.requireNonNull(entry).getKey().partition() <= partitionConst); Assert.assertEquals(expected, actual); Assert.assertFalse(actual.isEmpty()); } - - @Test - public void buildScanFromOffsetPredicateEq() - { + @Test public void buildScanFromOffsetPredicateEq() { long constantOffset = 30; - Map actual = KafkaScanTrimmer.buildScanFromOffsetPredicate( - fullHouse, - PredicateLeaf.Operator.EQUALS, - constantOffset, - false, - false - ); - Map expected = Maps.transformValues( - fullHouse, - entry -> new KafkaPullerInputSplit( - Objects.requireNonNull(entry).getTopic(), - entry.getPartition(), + Map + actual = + KafkaScanTrimmer.buildScanFromOffsetPredicate(fullHouse, + PredicateLeaf.Operator.EQUALS, constantOffset, - constantOffset + 1, - entry.getPath() - ) - ); + false, + false); + Map + expected = + Maps.transformValues(fullHouse, + entry -> new KafkaPullerInputSplit(Objects.requireNonNull(entry).getTopic(), + entry.getPartition(), + constantOffset, + constantOffset + 1, + entry.getPath())); Assert.assertEquals(expected, actual); // seek to end if offset is out of reach - actual = KafkaScanTrimmer.buildScanFromOffsetPredicate( - fullHouse, - PredicateLeaf.Operator.EQUALS, - 3000000L, - false, - false - ); - expected = Maps.transformValues( - fullHouse, - entry -> new KafkaPullerInputSplit( - Objects.requireNonNull(entry).getTopic(), - entry.getPartition(), - entry.getEndOffset(), - entry.getEndOffset(), - entry.getPath() - ) - ); + actual = + KafkaScanTrimmer.buildScanFromOffsetPredicate(fullHouse, PredicateLeaf.Operator.EQUALS, 3000000L, false, false); + expected = + Maps.transformValues(fullHouse, + entry -> new KafkaPullerInputSplit(Objects.requireNonNull(entry).getTopic(), + entry.getPartition(), + entry.getEndOffset(), + entry.getEndOffset(), + entry.getPath())); Assert.assertEquals(expected, actual); // seek to end if offset is out of reach - actual = KafkaScanTrimmer.buildScanFromOffsetPredicate( - fullHouse, - PredicateLeaf.Operator.EQUALS, - 0L, - false, - false - ); - - expected = Maps.transformValues( - fullHouse, - entry -> new KafkaPullerInputSplit( - Objects.requireNonNull(entry).getTopic(), - entry.getPartition(), - entry.getStartOffset() > 0 ? entry.getEndOffset() : 0, - entry.getStartOffset() > 0 ? entry.getEndOffset() : 1, - entry.getPath() - ) - ); + actual = KafkaScanTrimmer.buildScanFromOffsetPredicate(fullHouse, PredicateLeaf.Operator.EQUALS, 0L, false, false); + + expected = + Maps.transformValues(fullHouse, + entry -> new KafkaPullerInputSplit(Objects.requireNonNull(entry).getTopic(), + entry.getPartition(), + entry.getStartOffset() > 0 ? entry.getEndOffset() : 0, + entry.getStartOffset() > 0 ? entry.getEndOffset() : 1, + entry.getPath())); Assert.assertEquals(expected, actual); - } - @Test - public void buildScanFromOffsetPredicateLess() - { + @Test public void buildScanFromOffsetPredicateLess() { long constantOffset = 50; // columnOffset < constant - Map actual = KafkaScanTrimmer.buildScanFromOffsetPredicate( - fullHouse, - PredicateLeaf.Operator.LESS_THAN, - constantOffset, - false, - false - ); - - Map expected = Maps.transformValues( - fullHouse, - entry -> new KafkaPullerInputSplit( - Objects.requireNonNull(entry).getTopic(), - entry.getPartition(), - entry.getStartOffset(), - Math.min(constantOffset, entry.getEndOffset()), - entry.getPath() - ) - ); + Map + actual = + KafkaScanTrimmer.buildScanFromOffsetPredicate(fullHouse, + PredicateLeaf.Operator.LESS_THAN, + constantOffset, + false, + false); + + Map + expected = + Maps.transformValues(fullHouse, + entry -> new KafkaPullerInputSplit(Objects.requireNonNull(entry).getTopic(), + entry.getPartition(), + entry.getStartOffset(), + Math.min(constantOffset, entry.getEndOffset()), + entry.getPath())); Assert.assertEquals(expected, actual); - // columnOffset > constant - actual = KafkaScanTrimmer.buildScanFromOffsetPredicate( - fullHouse, - PredicateLeaf.Operator.LESS_THAN, - constantOffset, - true, - false - ); - - expected = Maps.transformValues( - fullHouse, - entry -> new KafkaPullerInputSplit( - Objects.requireNonNull(entry).getTopic(), - entry.getPartition(), - Math.min(entry.getEndOffset(), Math.max(entry.getStartOffset(), constantOffset + 1)), - entry.getEndOffset(), - entry.getPath() - ) - ); + actual = + KafkaScanTrimmer.buildScanFromOffsetPredicate(fullHouse, + PredicateLeaf.Operator.LESS_THAN, + constantOffset, + true, + false); + + expected = + Maps.transformValues(fullHouse, + entry -> new KafkaPullerInputSplit(Objects.requireNonNull(entry).getTopic(), + entry.getPartition(), + Math.min(entry.getEndOffset(), Math.max(entry.getStartOffset(), constantOffset + 1)), + entry.getEndOffset(), + entry.getPath())); Assert.assertEquals(expected, actual); // columnOffset >= constant - actual = KafkaScanTrimmer.buildScanFromOffsetPredicate( - fullHouse, - PredicateLeaf.Operator.LESS_THAN, - constantOffset, - false, - true - ); - - expected = Maps.transformValues( - fullHouse, - entry -> new KafkaPullerInputSplit( - Objects.requireNonNull(entry).getTopic(), - entry.getPartition(), - Math.min(entry.getEndOffset(), Math.max(entry.getStartOffset(), constantOffset)), - entry.getEndOffset(), - entry.getPath() - ) - ); + actual = + KafkaScanTrimmer.buildScanFromOffsetPredicate(fullHouse, + PredicateLeaf.Operator.LESS_THAN, + constantOffset, + false, + true); + + expected = + Maps.transformValues(fullHouse, + entry -> new KafkaPullerInputSplit(Objects.requireNonNull(entry).getTopic(), + entry.getPartition(), + Math.min(entry.getEndOffset(), Math.max(entry.getStartOffset(), constantOffset)), + entry.getEndOffset(), + entry.getPath())); Assert.assertEquals(expected, actual); - -// columnOffset <= constant - actual = KafkaScanTrimmer.buildScanFromOffsetPredicate( - fullHouse, - PredicateLeaf.Operator.LESS_THAN, - constantOffset, - true, - true - ); - - expected = Maps.transformValues( - fullHouse, - entry -> new KafkaPullerInputSplit( - Objects.requireNonNull(entry).getTopic(), - entry.getPartition(), - entry.getStartOffset(), - Math.min(constantOffset + 1, entry.getEndOffset()), - entry.getPath() - ) - ); + // columnOffset <= constant + actual = + KafkaScanTrimmer.buildScanFromOffsetPredicate(fullHouse, + PredicateLeaf.Operator.LESS_THAN, + constantOffset, + true, + true); + + expected = + Maps.transformValues(fullHouse, + entry -> new KafkaPullerInputSplit(Objects.requireNonNull(entry).getTopic(), + entry.getPartition(), + entry.getStartOffset(), + Math.min(constantOffset + 1, entry.getEndOffset()), + entry.getPath())); Assert.assertEquals(expected, actual); } - @Test - public void buildScanFromOffsetPredicateLessEq() - { + @Test public void buildScanFromOffsetPredicateLessEq() { long constantOffset = 50; // columnOffset < constant - Map actual = KafkaScanTrimmer.buildScanFromOffsetPredicate( - fullHouse, - PredicateLeaf.Operator.LESS_THAN_EQUALS, - constantOffset, - false, - false - ); - - Map expected = Maps.transformValues( - fullHouse, - entry -> new KafkaPullerInputSplit( - Objects.requireNonNull(entry).getTopic(), - entry.getPartition(), - entry.getStartOffset(), - Math.min(constantOffset + 1, entry.getEndOffset()), - entry.getPath() - ) - ); + Map + actual = + KafkaScanTrimmer.buildScanFromOffsetPredicate(fullHouse, + PredicateLeaf.Operator.LESS_THAN_EQUALS, + constantOffset, + false, + false); + + Map + expected = + Maps.transformValues(fullHouse, + entry -> new KafkaPullerInputSplit(Objects.requireNonNull(entry).getTopic(), + entry.getPartition(), + entry.getStartOffset(), + Math.min(constantOffset + 1, entry.getEndOffset()), + entry.getPath())); Assert.assertEquals(expected, actual); // columnOffset >= constant - actual = KafkaScanTrimmer.buildScanFromOffsetPredicate( - fullHouse, - PredicateLeaf.Operator.LESS_THAN_EQUALS, - constantOffset, - true, - false - ); - - expected = Maps.transformValues( - fullHouse, - entry -> new KafkaPullerInputSplit( - Objects.requireNonNull(entry).getTopic(), - entry.getPartition(), - Math.min(entry.getEndOffset(), Math.max(entry.getStartOffset(), constantOffset)), - entry.getEndOffset(), - entry.getPath() - ) - ); + actual = + KafkaScanTrimmer.buildScanFromOffsetPredicate(fullHouse, + PredicateLeaf.Operator.LESS_THAN_EQUALS, + constantOffset, + true, + false); + + expected = + Maps.transformValues(fullHouse, + entry -> new KafkaPullerInputSplit(Objects.requireNonNull(entry).getTopic(), + entry.getPartition(), + Math.min(entry.getEndOffset(), Math.max(entry.getStartOffset(), constantOffset)), + entry.getEndOffset(), + entry.getPath())); Assert.assertEquals(expected, actual); // columnOffset > constant - actual = KafkaScanTrimmer.buildScanFromOffsetPredicate( - fullHouse, - PredicateLeaf.Operator.LESS_THAN_EQUALS, - constantOffset, - false, - true - ); - - expected = Maps.transformValues( - fullHouse, - entry -> new KafkaPullerInputSplit( - Objects.requireNonNull(entry).getTopic(), - entry.getPartition(), - Math.min(entry.getEndOffset(), Math.max(entry.getStartOffset(), constantOffset + 1)), - entry.getEndOffset(), - entry.getPath() - ) - ); + actual = + KafkaScanTrimmer.buildScanFromOffsetPredicate(fullHouse, + PredicateLeaf.Operator.LESS_THAN_EQUALS, + constantOffset, + false, + true); + + expected = + Maps.transformValues(fullHouse, + entry -> new KafkaPullerInputSplit(Objects.requireNonNull(entry).getTopic(), + entry.getPartition(), + Math.min(entry.getEndOffset(), Math.max(entry.getStartOffset(), constantOffset + 1)), + entry.getEndOffset(), + entry.getPath())); Assert.assertEquals(expected, actual); // columnOffset < constant - actual = KafkaScanTrimmer.buildScanFromOffsetPredicate( - fullHouse, - PredicateLeaf.Operator.LESS_THAN_EQUALS, - constantOffset, - true, - true - ); - - expected = Maps.transformValues( - fullHouse, - entry -> new KafkaPullerInputSplit( - Objects.requireNonNull(entry).getTopic(), - entry.getPartition(), - entry.getStartOffset(), - Math.min(constantOffset, entry.getEndOffset()), - entry.getPath() - ) - ); + actual = + KafkaScanTrimmer.buildScanFromOffsetPredicate(fullHouse, + PredicateLeaf.Operator.LESS_THAN_EQUALS, + constantOffset, + true, + true); + + expected = + Maps.transformValues(fullHouse, + entry -> new KafkaPullerInputSplit(Objects.requireNonNull(entry).getTopic(), + entry.getPartition(), + entry.getStartOffset(), + Math.min(constantOffset, entry.getEndOffset()), + entry.getPath())); Assert.assertEquals(expected, actual); } - private static class ConstantExprBuilder - { - static ExprNodeDesc build(long constant) - { + private static class ConstantExprBuilder { + static ExprNodeDesc build(long constant) { return new ExprNodeConstantDesc(TypeInfoFactory.longTypeInfo, constant); } - static ExprNodeDesc build(int constant) - { + static ExprNodeDesc build(int constant) { return new ExprNodeConstantDesc(TypeInfoFactory.longTypeInfo, constant); } } - - private static ExprNodeGenericFuncDesc OR(List children) - { - return new ExprNodeGenericFuncDesc( - TypeInfoFactory.booleanTypeInfo, - new GenericUDFOPOr(), - children - ); + private static ExprNodeGenericFuncDesc or(List children) { + return new ExprNodeGenericFuncDesc(TypeInfoFactory.booleanTypeInfo, new GenericUDFOPOr(), children); } - private static ExprNodeGenericFuncDesc AND(List children) - { - return new ExprNodeGenericFuncDesc( - TypeInfoFactory.booleanTypeInfo, - new GenericUDFOPAnd(), - children - ); + private static ExprNodeGenericFuncDesc and(List children) { + return new ExprNodeGenericFuncDesc(TypeInfoFactory.booleanTypeInfo, new GenericUDFOPAnd(), children); } - private static ExprNodeGenericFuncDesc EQ(List children) - { - return new ExprNodeGenericFuncDesc( - children.get(0).getTypeInfo(), - new GenericUDFOPEqual(), - children - ); + private static ExprNodeGenericFuncDesc eq(List children) { + return new ExprNodeGenericFuncDesc(children.get(0).getTypeInfo(), new GenericUDFOPEqual(), children); } - private static ExprNodeGenericFuncDesc LESS_THAN(List children) - { - return new ExprNodeGenericFuncDesc( - children.get(0).getTypeInfo(), - new GenericUDFOPLessThan(), - children - ); + private static ExprNodeGenericFuncDesc lessThan(List children) { + return new ExprNodeGenericFuncDesc(children.get(0).getTypeInfo(), new GenericUDFOPLessThan(), children); } - private static ExprNodeGenericFuncDesc LESS_THAN_EQ(List children) - { - return new ExprNodeGenericFuncDesc( - children.get(0).getTypeInfo(), - new GenericUDFOPEqualOrLessThan(), - children - ); + private static ExprNodeGenericFuncDesc lessThanEq(List children) { + return new ExprNodeGenericFuncDesc(children.get(0).getTypeInfo(), new GenericUDFOPEqualOrLessThan(), children); } - private static ExprNodeGenericFuncDesc GREATER_THAN(List children) - { - return new ExprNodeGenericFuncDesc( - children.get(0).getTypeInfo(), - new GenericUDFOPGreaterThan(), - children - ); + private static ExprNodeGenericFuncDesc greaterThan(List children) { + return new ExprNodeGenericFuncDesc(children.get(0).getTypeInfo(), new GenericUDFOPGreaterThan(), children); } - private static ExprNodeGenericFuncDesc GREATER_THAN_EQ(List children) - { - return new ExprNodeGenericFuncDesc( - children.get(0).getTypeInfo(), - new GenericUDFOPEqualOrGreaterThan(), - children - ); + private static ExprNodeGenericFuncDesc greaterThanEq(List children) { + return new ExprNodeGenericFuncDesc(children.get(0).getTypeInfo(), new GenericUDFOPEqualOrGreaterThan(), children); } -} \ No newline at end of file +} diff --git a/kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaStreamingUtilsTest.java b/kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaStreamingUtilsTest.java index 9b107f137f1a..28c532f61113 100644 --- a/kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaStreamingUtilsTest.java +++ b/kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaStreamingUtilsTest.java @@ -24,12 +24,14 @@ import java.util.Properties; +/** + * Test for Utility class. + */ public class KafkaStreamingUtilsTest { public KafkaStreamingUtilsTest() { } - @Test - public void testConsumerProperties() { + @Test public void testConsumerProperties() { Configuration configuration = new Configuration(); configuration.set("kafka.bootstrap.servers", "localhost:9090"); configuration.set("kafka.consumer.fetch.max.wait.ms", "40"); diff --git a/kafka-handler/src/test/org/apache/hadoop/hive/kafka/package-info.java b/kafka-handler/src/test/org/apache/hadoop/hive/kafka/package-info.java new file mode 100644 index 000000000000..8a0d8fd0b050 --- /dev/null +++ b/kafka-handler/src/test/org/apache/hadoop/hive/kafka/package-info.java @@ -0,0 +1,23 @@ +/* + * 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 info file. + */ + +package org.apache.hadoop.hive.kafka; diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/cli/LlapServiceDriver.java b/llap-server/src/java/org/apache/hadoop/hive/llap/cli/LlapServiceDriver.java index ae1de26adc47..ffdd340fdef0 100644 --- a/llap-server/src/java/org/apache/hadoop/hive/llap/cli/LlapServiceDriver.java +++ b/llap-server/src/java/org/apache/hadoop/hive/llap/cli/LlapServiceDriver.java @@ -18,44 +18,9 @@ package org.apache.hadoop.hive.llap.cli; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStreamWriter; -import java.io.PrintWriter; -import java.net.URI; -import java.net.URISyntaxException; -import java.net.URL; -import java.nio.file.Paths; -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Map; -import java.util.Map.Entry; -import java.util.Collection; -import java.util.List; -import java.util.Properties; -import java.util.Set; -import java.util.concurrent.Callable; -import java.util.concurrent.CompletionService; -import java.util.concurrent.ExecutorCompletionService; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.Future; - +import com.google.common.base.Preconditions; +import com.google.common.util.concurrent.ThreadFactoryBuilder; import org.apache.commons.lang3.StringUtils; -import org.apache.hadoop.hive.llap.LlapUtil; -import org.apache.hadoop.hive.llap.configuration.LlapDaemonConfiguration; -import org.apache.hadoop.hive.llap.daemon.impl.LlapConstants; -import org.apache.hadoop.hive.llap.daemon.impl.StaticPermanentFunctionChecker; -import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos; -import org.apache.hadoop.hive.llap.tezplugins.LlapTezUtils; -import org.apache.hadoop.registry.client.binding.RegistryUtils; -import org.apache.tez.dag.api.TezConfiguration; -import org.codehaus.jettison.json.JSONException; -import org.codehaus.jettison.json.JSONObject; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; @@ -64,8 +29,14 @@ import org.apache.hadoop.hive.common.CompressionUtils; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.conf.HiveConf.ConfVars; +import org.apache.hadoop.hive.llap.LlapUtil; import org.apache.hadoop.hive.llap.cli.LlapOptionsProcessor.LlapOptions; +import org.apache.hadoop.hive.llap.configuration.LlapDaemonConfiguration; +import org.apache.hadoop.hive.llap.daemon.impl.LlapConstants; +import org.apache.hadoop.hive.llap.daemon.impl.StaticPermanentFunctionChecker; +import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos; import org.apache.hadoop.hive.llap.io.api.impl.LlapInputFormat; +import org.apache.hadoop.hive.llap.tezplugins.LlapTezUtils; import org.apache.hadoop.hive.metastore.api.Function; import org.apache.hadoop.hive.metastore.api.ResourceUri; import org.apache.hadoop.hive.ql.exec.Utilities; @@ -77,22 +48,50 @@ import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapreduce.Job; +import org.apache.hadoop.registry.client.binding.RegistryUtils; import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.tez.dag.api.TezConfiguration; +import org.codehaus.jettison.json.JSONException; +import org.codehaus.jettison.json.JSONObject; import org.eclipse.jetty.rewrite.handler.Rule; import org.eclipse.jetty.util.ssl.SslContextFactory; import org.joda.time.DateTime; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; -import com.google.common.base.Preconditions; -import com.google.common.util.concurrent.ThreadFactoryBuilder; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStreamWriter; +import java.io.PrintWriter; +import java.net.URI; +import java.net.URISyntaxException; +import java.net.URL; +import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Properties; +import java.util.Set; +import java.util.concurrent.Callable; +import java.util.concurrent.CompletionService; +import java.util.concurrent.ExecutorCompletionService; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; public class LlapServiceDriver { protected static final Logger LOG = LoggerFactory.getLogger(LlapServiceDriver.class.getName()); - private static final String[] DEFAULT_AUX_CLASSES = new String[] { - "org.apache.hive.hcatalog.data.JsonSerDe","org.apache.hadoop.hive.druid.DruidStorageHandler", - "org.apache.hive.storage.jdbc.JdbcStorageHandler", "org.apache.commons.dbcp.BasicDataSourceFactory", - "org.apache.commons.pool.impl.GenericObjectPool", "org.apache.hadoop.hive.kafka.KafkaStorageHandler" - }; + private static final String[] + DEFAULT_AUX_CLASSES = + new String[] { "org.apache.hive.hcatalog.data.JsonSerDe", "org.apache.hadoop.hive.druid.DruidStorageHandler", + "org.apache.hive.storage.jdbc.JdbcStorageHandler", "org.apache.commons.dbcp.BasicDataSourceFactory", + "org.apache.commons.pool.impl.GenericObjectPool", "org.apache.hadoop.hive.kafka.KafkaStorageHandler" }; private static final String HBASE_SERDE_CLASS = "org.apache.hadoop.hive.hbase.HBaseSerDe"; private static final String[] NEEDED_CONFIGS = LlapDaemonConfiguration.DAEMON_CONFIGS; private static final String[] OPTIONAL_CONFIGS = LlapDaemonConfiguration.SSL_DAEMON_CONFIGS; From e99cac11432dcd0471a784e7d7eb389fda2309d4 Mon Sep 17 00:00:00 2001 From: Slim Bouguerra Date: Tue, 14 Aug 2018 18:45:33 -0700 Subject: [PATCH 10/33] adding more tests Change-Id: I72a8a9083d4a0c387fede5ee572d831bc5db7ec8 --- .../apache/hadoop/hive/kafka/KafkaRecordIteratorTest.java | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaRecordIteratorTest.java b/kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaRecordIteratorTest.java index 41c31a47b4ae..1440c31e4110 100644 --- a/kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaRecordIteratorTest.java +++ b/kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaRecordIteratorTest.java @@ -221,6 +221,12 @@ public KafkaRecordIteratorTest() { this.compareIterator(ImmutableList.of(), this.kafkaRecordIterator); } + @Test public void testStartIsTheFirstOffset() { + this.kafkaRecordIterator = + new KafkaRecordIterator(this.consumer, TOPIC_PARTITION, 0L, 0L, 100L); + this.compareIterator(ImmutableList.of(), this.kafkaRecordIterator); + } + private void compareIterator(List> expected, Iterator> kafkaRecordIterator) { expected.stream().forEachOrdered((expectedRecord) -> { From df2b7774c086bd6359890c19886ff64bb42af028 Mon Sep 17 00:00:00 2001 From: Slim Bouguerra Date: Wed, 15 Aug 2018 09:14:04 -0700 Subject: [PATCH 11/33] adding empty iterator Change-Id: Ibe29a536a689250211385ef2297f254701572fd5 --- .../hive/kafka/KafkaPullerRecordReader.java | 5 ++++- .../hadoop/hive/kafka/KafkaRecordIterator.java | 16 ++++++++++++++-- .../hive/kafka/KafkaRecordIteratorTest.java | 2 +- 3 files changed, 19 insertions(+), 4 deletions(-) diff --git a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaPullerRecordReader.java b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaPullerRecordReader.java index 81f1204f8409..fbd3de3da50e 100644 --- a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaPullerRecordReader.java +++ b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaPullerRecordReader.java @@ -94,7 +94,10 @@ private synchronized void initialize(KafkaPullerInputSplit inputSplit, Configura config.getLong(KafkaStorageHandler.HIVE_KAFKA_POLL_TIMEOUT, KafkaStorageHandler.DEFAULT_CONSUMER_POLL_TIMEOUT_MS); LOG.debug("Consumer poll timeout [{}] ms", pollTimeout); - this.recordsCursor = new KafkaRecordIterator(consumer, topicPartition, startOffset, endOffset, pollTimeout); + this.recordsCursor = + startOffset == endOffset ? + new KafkaRecordIterator.EmptyIterator() : + new KafkaRecordIterator(consumer, topicPartition, startOffset, endOffset, pollTimeout); started = true; } } diff --git a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaRecordIterator.java b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaRecordIterator.java index 7d5754d81b4a..8cac8f6b83a9 100644 --- a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaRecordIterator.java +++ b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaRecordIterator.java @@ -88,7 +88,7 @@ public KafkaRecordIterator(Consumer consumer, this.started = true; } - public KafkaRecordIterator(Consumer consumer, TopicPartition tp, long pollTimeoutMs) { + KafkaRecordIterator(Consumer consumer, TopicPartition tp, long pollTimeoutMs) { this(consumer, tp, null, null, pollTimeoutMs); } @@ -132,7 +132,7 @@ private void assignAndSeek() { @Override public boolean hasNext() { /* - Poll more records if + Poll more records from Kafka queue IF: Initial poll case -> (records == null) OR Need to poll at least one more record (currentOffset + 1 < endOffset) AND consumerRecordIterator is empty (!hasMore) @@ -194,4 +194,16 @@ private void findNext() { nextRecord = null; } + /** + * Empty iterator for empty splits when startOffset == endOffset, this is added to avoid clumsy if condition. + */ + protected static final class EmptyIterator implements Iterator> { + @Override public boolean hasNext() { + return false; + } + + @Override public ConsumerRecord next() { + throw new IllegalStateException("this is an empty iterator"); + } + } } diff --git a/kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaRecordIteratorTest.java b/kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaRecordIteratorTest.java index 1440c31e4110..acb7ea66b450 100644 --- a/kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaRecordIteratorTest.java +++ b/kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaRecordIteratorTest.java @@ -114,7 +114,7 @@ public KafkaRecordIteratorTest() { } @Test public void testHasNextAbsoluteStartEnd() { - this.kafkaRecordIterator = new KafkaRecordIterator(this.consumer, TOPIC_PARTITION, 0L, (long) RECORDS.size(), 100L); + this.kafkaRecordIterator = new KafkaRecordIterator(this.consumer, TOPIC_PARTITION, 0L, (long) RECORDS.size(), 200L); this.compareIterator(RECORDS, this.kafkaRecordIterator); } From 49edead90167b7b84ff9bde474f185078e3ab2dd Mon Sep 17 00:00:00 2001 From: Slim Bouguerra Date: Wed, 15 Aug 2018 09:45:09 -0700 Subject: [PATCH 12/33] filter empty splits anyway Change-Id: I31a947946efb68718951e6cb0fe08bc4c38e8f5e --- .../java/org/apache/hadoop/hive/kafka/KafkaJsonSerDe.java | 3 ++- .../apache/hadoop/hive/kafka/KafkaPullerInputFormat.java | 6 +++++- 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaJsonSerDe.java b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaJsonSerDe.java index b44aceab44a5..cdc2a9ec975c 100644 --- a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaJsonSerDe.java +++ b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaJsonSerDe.java @@ -74,7 +74,8 @@ import java.util.stream.Collectors; /** - * Basic JsonSerde to make use of such storage handler smooth and easy. For production please use Hive native JsonSerde + * 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 { private static final Logger LOG = LoggerFactory.getLogger(KafkaJsonSerDe.class); diff --git a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaPullerInputFormat.java b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaPullerInputFormat.java index aeec9642dccc..3066dcf7ba29 100644 --- a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaPullerInputFormat.java +++ b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaPullerInputFormat.java @@ -157,7 +157,11 @@ private List computeSplits(Configuration configuration) Future> futureTinyHouse = execService.submit(trimmerWorker); try { - return futureTinyHouse.get(timeoutMs, TimeUnit.MILLISECONDS); + return futureTinyHouse.get(timeoutMs, TimeUnit.MILLISECONDS) + .stream() + // filter out empty splits + .filter(split -> split.getStartOffset() < split.getEndOffset()) + .collect(Collectors.toList()); } catch (ExecutionException | TimeoutException e) { futureTinyHouse.cancel(true); LOG.error("Had issue with trimmer will return full scan ", e); From f103ec8f148a3529b26435dfa5f54c5d77e42c67 Mon Sep 17 00:00:00 2001 From: Slim Bouguerra Date: Wed, 15 Aug 2018 09:53:43 -0700 Subject: [PATCH 13/33] fix typo in functions names Change-Id: I712f09cfb8eeb16a1246df01190483eaa9a86004 --- .../hadoop/hive/kafka/KafkaScanTrimmer.java | 6 +++--- .../hive/kafka/KafkaScanTrimmerTest.java | 18 +++++++++--------- 2 files changed, 12 insertions(+), 12 deletions(-) diff --git a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaScanTrimmer.java b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaScanTrimmer.java index e90c4867869b..a85dc97658bf 100644 --- a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaScanTrimmer.java +++ b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaScanTrimmer.java @@ -192,7 +192,7 @@ public Map computeOptimizedScan(ExprNodeG } if (columnDesc.getColumn().equals(KafkaStorageHandler.PARTITION_COLUMN)) { - return buildScanFormPartitionPredicate(fullHouse, + return buildScanFromPartitionPredicate(fullHouse, operator, ((Number) constantDesc.getValue()).intValue(), flip, @@ -225,7 +225,7 @@ public Map computeOptimizedScan(ExprNodeG * * @return filtered kafka scan */ - @VisibleForTesting protected static Map buildScanFormPartitionPredicate( + @VisibleForTesting protected static Map buildScanFromPartitionPredicate( Map fullScan, PredicateLeaf.Operator operator, int partitionConst, @@ -328,7 +328,7 @@ public Map computeOptimizedScan(ExprNodeG newInputSplit = new KafkaPullerInputSplit(tp.topic(), tp.partition(), // @TODO make sure that this is okay - //if the user as for start offset > max offset will replace with last offset + //if the user ask for start offset > max offset will replace with last offset Math.min(startOffset, existingInputSplit.getEndOffset()), existingInputSplit.getEndOffset(), existingInputSplit.getPath()); diff --git a/kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaScanTrimmerTest.java b/kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaScanTrimmerTest.java index c7e0e1f3eebe..6444b47c4564 100644 --- a/kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaScanTrimmerTest.java +++ b/kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaScanTrimmerTest.java @@ -217,7 +217,7 @@ public class KafkaScanTrimmerTest { @Test public void buildScanFormPartitionPredicateEq() { Map actual = - KafkaScanTrimmer.buildScanFormPartitionPredicate(fullHouse, PredicateLeaf.Operator.EQUALS, 3, false, false); + KafkaScanTrimmer.buildScanFromPartitionPredicate(fullHouse, PredicateLeaf.Operator.EQUALS, 3, false, false); TopicPartition topicPartition = new TopicPartition(topic, 3); Assert.assertEquals(fullHouse.get(topicPartition), actual.get(topicPartition)); } @@ -227,7 +227,7 @@ public class KafkaScanTrimmerTest { int partitionConst = 2; Map actual = - KafkaScanTrimmer.buildScanFormPartitionPredicate(fullHouse, + KafkaScanTrimmer.buildScanFromPartitionPredicate(fullHouse, PredicateLeaf.Operator.LESS_THAN, partitionConst, true, @@ -241,7 +241,7 @@ public class KafkaScanTrimmerTest { // partitionConst >= partitionColumn (flip true, negation true) actual = - KafkaScanTrimmer.buildScanFormPartitionPredicate(fullHouse, + KafkaScanTrimmer.buildScanFromPartitionPredicate(fullHouse, PredicateLeaf.Operator.LESS_THAN, partitionConst, true, @@ -254,7 +254,7 @@ public class KafkaScanTrimmerTest { // partitionColumn >= partitionConst (negation true) actual = - KafkaScanTrimmer.buildScanFormPartitionPredicate(fullHouse, + KafkaScanTrimmer.buildScanFromPartitionPredicate(fullHouse, PredicateLeaf.Operator.LESS_THAN, partitionConst, false, @@ -267,7 +267,7 @@ public class KafkaScanTrimmerTest { // partitionColumn < partitionConst (negation true) actual = - KafkaScanTrimmer.buildScanFormPartitionPredicate(fullHouse, + KafkaScanTrimmer.buildScanFromPartitionPredicate(fullHouse, PredicateLeaf.Operator.LESS_THAN, partitionConst, false, @@ -284,7 +284,7 @@ public class KafkaScanTrimmerTest { int partitionConst = 2; Map actual = - KafkaScanTrimmer.buildScanFormPartitionPredicate(fullHouse, + KafkaScanTrimmer.buildScanFromPartitionPredicate(fullHouse, PredicateLeaf.Operator.LESS_THAN_EQUALS, partitionConst, true, @@ -298,7 +298,7 @@ public class KafkaScanTrimmerTest { // partitionConst > partitionColumn (flip true, negation true) actual = - KafkaScanTrimmer.buildScanFormPartitionPredicate(fullHouse, + KafkaScanTrimmer.buildScanFromPartitionPredicate(fullHouse, PredicateLeaf.Operator.LESS_THAN_EQUALS, partitionConst, true, @@ -311,7 +311,7 @@ public class KafkaScanTrimmerTest { // partitionColumn > partitionConst (negation true) actual = - KafkaScanTrimmer.buildScanFormPartitionPredicate(fullHouse, + KafkaScanTrimmer.buildScanFromPartitionPredicate(fullHouse, PredicateLeaf.Operator.LESS_THAN_EQUALS, partitionConst, false, @@ -324,7 +324,7 @@ public class KafkaScanTrimmerTest { // partitionColumn <= partitionConst (negation true) actual = - KafkaScanTrimmer.buildScanFormPartitionPredicate(fullHouse, + KafkaScanTrimmer.buildScanFromPartitionPredicate(fullHouse, PredicateLeaf.Operator.LESS_THAN_EQUALS, partitionConst, false, From 9591091a4ae07b1556c125c16f9b5002f81141b8 Mon Sep 17 00:00:00 2001 From: Slim Bouguerra Date: Wed, 15 Aug 2018 16:18:57 -0700 Subject: [PATCH 14/33] adding some tests with filter on timestamp Change-Id: I133bc72117571dfb1c1f69bcecbda3c1c8be050b --- .../clientpositive/kafka_storage_handler.q | 4 ++++ .../druid/kafka_storage_handler.q.out | 20 +++++++++++++++++++ 2 files changed, 24 insertions(+) diff --git a/ql/src/test/queries/clientpositive/kafka_storage_handler.q b/ql/src/test/queries/clientpositive/kafka_storage_handler.q index 83a5015f0725..9dc5680e375c 100644 --- a/ql/src/test/queries/clientpositive/kafka_storage_handler.q +++ b/ql/src/test/queries/clientpositive/kafka_storage_handler.q @@ -31,6 +31,10 @@ Select `__partition`, `__offset`,`__time`, `page`, `user`, `language`, `country` from kafka_table where `__offset` > 7 and `__partition` = 0 OR `__offset` = 4 and `__partition` = 0 OR `__offset` <= 1 and `__partition` = 0; +-- Timestamp filter + +Select `__partition`, `__offset`, `user` from kafka_table where +`__timestamp` > 1000 * to_unix_timestamp(CURRENT_TIMESTAMP - interval '1' HOURS) ; -- non existing partition Select count(*) from kafka_table where `__partition` = 1; diff --git a/ql/src/test/results/clientpositive/druid/kafka_storage_handler.q.out b/ql/src/test/results/clientpositive/druid/kafka_storage_handler.q.out index e5a15c2634e0..ec7ca062e614 100644 --- a/ql/src/test/results/clientpositive/druid/kafka_storage_handler.q.out +++ b/ql/src/test/results/clientpositive/druid/kafka_storage_handler.q.out @@ -141,6 +141,26 @@ POSTHOOK: Output: hdfs://### HDFS PATH ### 0 4 NULL Coyote Tango stringer ja Japan Asia wikipedia false true false true 1 10 -9 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 `__partition`, `__offset`, `user` from kafka_table where +`__timestamp` > 1000 * to_unix_timestamp(CURRENT_TIMESTAMP - interval '1' HOURS) +PREHOOK: type: QUERY +PREHOOK: Input: default@kafka_table +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: Select `__partition`, `__offset`, `user` from kafka_table where +`__timestamp` > 1000 * to_unix_timestamp(CURRENT_TIMESTAMP - interval '1' HOURS) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@kafka_table +POSTHOOK: Output: hdfs://### HDFS PATH ### +0 0 nuclear +0 1 speed +0 2 masterYi +0 3 triplets +0 4 stringer +0 5 nuclear +0 6 speed +0 7 masterYi +0 8 triplets +0 9 stringer PREHOOK: query: Select count(*) from kafka_table where `__partition` = 1 PREHOOK: type: QUERY PREHOOK: Input: default@kafka_table From 9b7f1ba8d69c1c49a9918c200310575e4700e688 Mon Sep 17 00:00:00 2001 From: Slim Bouguerra Date: Fri, 17 Aug 2018 11:11:46 -0700 Subject: [PATCH 15/33] return -1 from getPos to skip Header footer check Change-Id: I051b3559998849a23112f6597e30a2006704c615 --- .../org/apache/hadoop/hive/kafka/KafkaPullerInputSplit.java | 1 + .../org/apache/hadoop/hive/kafka/KafkaPullerRecordReader.java | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaPullerInputSplit.java b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaPullerInputSplit.java index 33813e829ceb..a4f1de415398 100644 --- a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaPullerInputSplit.java +++ b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaPullerInputSplit.java @@ -208,4 +208,5 @@ public static List slice(long sliceSize, final KafkaPulle return Collections.singletonList(copyOf(split)); } + } diff --git a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaPullerRecordReader.java b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaPullerRecordReader.java index fbd3de3da50e..5b79d6ebfbfb 100644 --- a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaPullerRecordReader.java +++ b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaPullerRecordReader.java @@ -127,7 +127,7 @@ private synchronized void initialize(KafkaPullerInputSplit inputSplit, Configura } @Override public long getPos() throws IOException { - return consumedRecords; + return -1; } @Override public boolean nextKeyValue() throws IOException { From 41deffeedf8a0b616973e64933504ded7a2b29da Mon Sep 17 00:00:00 2001 From: Slim Bouguerra Date: Fri, 17 Aug 2018 13:15:45 -0700 Subject: [PATCH 16/33] use byteswritabe instead of bytesref to use lazybinary sede Change-Id: Ib20537c52ea1154b7d2bf2e92e57ddc79414d0c4 --- .../java/org/apache/hadoop/hive/kafka/GenericKafkaSerDe.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/GenericKafkaSerDe.java b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/GenericKafkaSerDe.java index a8ccbc1733a9..3996eaa2ac95 100644 --- a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/GenericKafkaSerDe.java +++ b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/GenericKafkaSerDe.java @@ -35,6 +35,7 @@ import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo; 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.Text; @@ -160,7 +161,7 @@ private AbstractSerDe createDelegate(String className) { row = delegateSerDe.deserialize(avroGenericRecordWritable); } else { // default assuming delegate Serde know how to deal with - row = delegateSerDe.deserialize(new BytesRefWritable(record.getValue())); + row = delegateSerDe.deserialize(new BytesWritable(record.getValue())); } return columnNames.stream().map(name -> { From 9d603a8c4e46c5e2ef1cf3d66e6b0f9a548505fb Mon Sep 17 00:00:00 2001 From: Slim Bouguerra Date: Fri, 17 Aug 2018 15:41:56 -0700 Subject: [PATCH 17/33] refactor names and add docs Change-Id: Ia769a3e8af5f07a4f3625c2d4ba1c5b03f864a63 --- .../hadoop/hive/kafka/GenericKafkaSerDe.java | 14 +++--- .../hive/kafka/KafkaPullerInputFormat.java | 6 +-- .../hive/kafka/KafkaPullerRecordReader.java | 4 +- .../hive/kafka/KafkaRecordIterator.java | 2 +- .../hadoop/hive/kafka/KafkaScanTrimmer.java | 7 +-- .../hive/kafka/KafkaStorageHandler.java | 38 ++++++--------- .../hive/kafka/KafkaStreamingUtils.java | 47 +++++++++++++++++-- .../hive/kafka/KafkaScanTrimmerTest.java | 4 +- 8 files changed, 76 insertions(+), 46 deletions(-) diff --git a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/GenericKafkaSerDe.java b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/GenericKafkaSerDe.java index 3996eaa2ac95..a918ca5ef17c 100644 --- a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/GenericKafkaSerDe.java +++ b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/GenericKafkaSerDe.java @@ -63,9 +63,9 @@ public class GenericKafkaSerDe extends AbstractSerDe { // ORDER of fields and types matters here public static final ImmutableList METADATA_COLUMN_NAMES = - ImmutableList.of(KafkaStorageHandler.PARTITION_COLUMN, - KafkaStorageHandler.OFFSET_COLUMN, - KafkaStorageHandler.TIMESTAMP_COLUMN); + ImmutableList.of(KafkaStreamingUtils.PARTITION_COLUMN, + KafkaStreamingUtils.OFFSET_COLUMN, + KafkaStreamingUtils.TIMESTAMP_COLUMN); public static final ImmutableList METADATA_PRIMITIVE_TYPE_INFO = ImmutableList.of(TypeInfoFactory.intTypeInfo, TypeInfoFactory.longTypeInfo, TypeInfoFactory.longTypeInfo); @@ -76,7 +76,7 @@ public class GenericKafkaSerDe extends AbstractSerDe { StructObjectInspector delegateObjectInspector; @Override public void initialize(@Nullable Configuration conf, Properties tbl) throws SerDeException { - final String className = tbl.getProperty(KafkaStorageHandler.SERDE_CLASS_NAME, KafkaJsonSerDe.class.getName()); + final String className = tbl.getProperty(KafkaStreamingUtils.SERDE_CLASS_NAME, KafkaJsonSerDe.class.getName()); delegateSerDe = createDelegate(className); delegateSerDe.initialize(conf, tbl); LOG.info("Using SerDe instance {}", delegateSerDe.getClass().getCanonicalName()); @@ -166,11 +166,11 @@ private AbstractSerDe createDelegate(String className) { return columnNames.stream().map(name -> { switch (name) { - case KafkaStorageHandler.PARTITION_COLUMN: + case KafkaStreamingUtils.PARTITION_COLUMN: return new IntWritable(record.getPartition()); - case KafkaStorageHandler.OFFSET_COLUMN: + case KafkaStreamingUtils.OFFSET_COLUMN: return new LongWritable(record.getOffset()); - case KafkaStorageHandler.TIMESTAMP_COLUMN: + case KafkaStreamingUtils.TIMESTAMP_COLUMN: return new LongWritable(record.getTimestamp()); default: return delegateObjectInspector.getStructFieldData(row, delegateObjectInspector.getStructFieldRef(name)); diff --git a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaPullerInputFormat.java b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaPullerInputFormat.java index 3066dcf7ba29..ed421f8ccd12 100644 --- a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaPullerInputFormat.java +++ b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaPullerInputFormat.java @@ -114,11 +114,11 @@ private List computeSplits(Configuration configuration) // this will be used to harness some KAFKA blocking calls final ExecutorService execService = Executors.newSingleThreadExecutor(); try (KafkaConsumer consumer = new KafkaConsumer(KafkaStreamingUtils.consumerProperties(configuration))) { - final String topic = configuration.get(KafkaStorageHandler.HIVE_KAFKA_TOPIC); + final String topic = configuration.get(KafkaStreamingUtils.HIVE_KAFKA_TOPIC); final long timeoutMs = - configuration.getLong(KafkaStorageHandler.HIVE_KAFKA_POLL_TIMEOUT, - KafkaStorageHandler.DEFAULT_CONSUMER_POLL_TIMEOUT_MS); + configuration.getLong(KafkaStreamingUtils.HIVE_KAFKA_POLL_TIMEOUT, + KafkaStreamingUtils.DEFAULT_CONSUMER_POLL_TIMEOUT_MS); // hive depends on FileSplits JobConf jobConf = new JobConf(configuration); Path[] tablePaths = org.apache.hadoop.mapred.FileInputFormat.getInputPaths(jobConf); diff --git a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaPullerRecordReader.java b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaPullerRecordReader.java index 5b79d6ebfbfb..e023687bf261 100644 --- a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaPullerRecordReader.java +++ b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaPullerRecordReader.java @@ -91,8 +91,8 @@ private synchronized void initialize(KafkaPullerInputSplit inputSplit, Configura totalNumberRecords += endOffset - startOffset; initConsumer(); pollTimeout = - config.getLong(KafkaStorageHandler.HIVE_KAFKA_POLL_TIMEOUT, - KafkaStorageHandler.DEFAULT_CONSUMER_POLL_TIMEOUT_MS); + config.getLong(KafkaStreamingUtils.HIVE_KAFKA_POLL_TIMEOUT, + KafkaStreamingUtils.DEFAULT_CONSUMER_POLL_TIMEOUT_MS); LOG.debug("Consumer poll timeout [{}] ms", pollTimeout); this.recordsCursor = startOffset == endOffset ? diff --git a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaRecordIterator.java b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaRecordIterator.java index 8cac8f6b83a9..edcd9e6dd52c 100644 --- a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaRecordIterator.java +++ b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaRecordIterator.java @@ -165,7 +165,7 @@ private void pollRecords() { topicPartition.toString(), endOffset, pollTimeoutMs, - KafkaStorageHandler.HIVE_KAFKA_POLL_TIMEOUT)); + KafkaStreamingUtils.HIVE_KAFKA_POLL_TIMEOUT)); } consumerRecordIterator = records.iterator(); } diff --git a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaScanTrimmer.java b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaScanTrimmer.java index a85dc97658bf..8904548e1867 100644 --- a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaScanTrimmer.java +++ b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaScanTrimmer.java @@ -191,7 +191,8 @@ public Map computeOptimizedScan(ExprNodeG constantDesc = (ExprNodeConstantDesc) extracted[0]; } - if (columnDesc.getColumn().equals(KafkaStorageHandler.PARTITION_COLUMN)) { + + if (columnDesc.getColumn().equals(KafkaStreamingUtils.PARTITION_COLUMN)) { return buildScanFromPartitionPredicate(fullHouse, operator, ((Number) constantDesc.getValue()).intValue(), @@ -199,7 +200,7 @@ public Map computeOptimizedScan(ExprNodeG negation); } - if (columnDesc.getColumn().equals(KafkaStorageHandler.OFFSET_COLUMN)) { + if (columnDesc.getColumn().equals(KafkaStreamingUtils.OFFSET_COLUMN)) { return buildScanFromOffsetPredicate(fullHouse, operator, ((Number) constantDesc.getValue()).longValue(), @@ -207,7 +208,7 @@ public Map computeOptimizedScan(ExprNodeG negation); } - if (columnDesc.getColumn().equals(KafkaStorageHandler.TIMESTAMP_COLUMN)) { + if (columnDesc.getColumn().equals(KafkaStreamingUtils.TIMESTAMP_COLUMN)) { long timestamp = ((Number) constantDesc.getValue()).longValue(); return buildScanForTimesPredicate(fullHouse, operator, timestamp, flip, negation, kafkaConsumer); } diff --git a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaStorageHandler.java b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaStorageHandler.java index ca26045e8a74..81f365e6344c 100644 --- a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaStorageHandler.java +++ b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaStorageHandler.java @@ -43,16 +43,6 @@ */ public class KafkaStorageHandler implements HiveStorageHandler { - public static final String TIMESTAMP_COLUMN = "__timestamp"; - public static final String PARTITION_COLUMN = "__partition"; - public static final String OFFSET_COLUMN = "__offset"; - public static final String SERDE_CLASS_NAME = "kafka.serde.class"; - public static final String HIVE_KAFKA_TOPIC = "kafka.topic"; - public static final String CONSUMER_CONFIGURATION_PREFIX = "kafka.consumer"; - public static final String HIVE_KAFKA_BOOTSTRAP_SERVERS = "kafka.bootstrap.servers"; - public static final String HIVE_KAFKA_POLL_TIMEOUT = "hive.kafka.poll.timeout.ms"; - public static final long DEFAULT_CONSUMER_POLL_TIMEOUT_MS = 5000L; // 5 seconds - private static final Logger LOG = LoggerFactory.getLogger(KafkaStorageHandler.class); Configuration configuration; @@ -78,18 +68,18 @@ public class KafkaStorageHandler implements HiveStorageHandler { } @Override public void configureInputJobProperties(TableDesc tableDesc, Map jobProperties) { - jobProperties.put(HIVE_KAFKA_TOPIC, - Preconditions.checkNotNull(tableDesc.getProperties().getProperty(HIVE_KAFKA_TOPIC), - "kafka topic missing set table property->" + HIVE_KAFKA_TOPIC)); - LOG.debug("Table properties: Kafka Topic {}", tableDesc.getProperties().getProperty(HIVE_KAFKA_TOPIC)); - jobProperties.put(HIVE_KAFKA_BOOTSTRAP_SERVERS, - Preconditions.checkNotNull(tableDesc.getProperties().getProperty(HIVE_KAFKA_BOOTSTRAP_SERVERS), - "Broker address missing set table property->" + HIVE_KAFKA_BOOTSTRAP_SERVERS)); - LOG.debug("Table properties: Kafka broker {}", tableDesc.getProperties().getProperty(HIVE_KAFKA_BOOTSTRAP_SERVERS)); - jobProperties.put(SERDE_CLASS_NAME, - tableDesc.getProperties().getProperty(SERDE_CLASS_NAME, KafkaJsonSerDe.class.getName())); - - LOG.info("Table properties: SerDe class name {}", jobProperties.get(SERDE_CLASS_NAME)); + jobProperties.put(KafkaStreamingUtils.HIVE_KAFKA_TOPIC, + Preconditions.checkNotNull(tableDesc.getProperties().getProperty(KafkaStreamingUtils.HIVE_KAFKA_TOPIC), + "kafka topic missing set table property->" + KafkaStreamingUtils.HIVE_KAFKA_TOPIC)); + LOG.debug("Table properties: Kafka Topic {}", tableDesc.getProperties().getProperty(KafkaStreamingUtils.HIVE_KAFKA_TOPIC)); + jobProperties.put(KafkaStreamingUtils.HIVE_KAFKA_BOOTSTRAP_SERVERS, + Preconditions.checkNotNull(tableDesc.getProperties().getProperty(KafkaStreamingUtils.HIVE_KAFKA_BOOTSTRAP_SERVERS), + "Broker address missing set table property->" + KafkaStreamingUtils.HIVE_KAFKA_BOOTSTRAP_SERVERS)); + LOG.debug("Table properties: Kafka broker {}", tableDesc.getProperties().getProperty(KafkaStreamingUtils.HIVE_KAFKA_BOOTSTRAP_SERVERS)); + 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() @@ -98,9 +88,9 @@ public class KafkaStorageHandler implements HiveStorageHandler { .filter(objectObjectEntry -> objectObjectEntry.getKey() .toString() .toLowerCase() - .startsWith(CONSUMER_CONFIGURATION_PREFIX)) + .startsWith(KafkaStreamingUtils.CONSUMER_CONFIGURATION_PREFIX)) .forEach(entry -> { - String key = entry.getKey().toString().substring(CONSUMER_CONFIGURATION_PREFIX.length() + 1); + String key = entry.getKey().toString().substring(KafkaStreamingUtils.CONSUMER_CONFIGURATION_PREFIX.length() + 1); String value = entry.getValue().toString(); jobProperties.put(key, value); LOG.info("Setting extra job properties: key [{}] -> value [{}]", key, value); diff --git a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaStreamingUtils.java b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaStreamingUtils.java index 132db9880dc8..fa434d9c788c 100644 --- a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaStreamingUtils.java +++ b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaStreamingUtils.java @@ -38,10 +38,49 @@ import static org.apache.kafka.clients.consumer.ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG; /** - * Utilities class. + * Constant, Table properties, Utilities class. */ public final class KafkaStreamingUtils { + /** + * MANDATORY Table property indicating kafka topic backing the table + */ + public static final String HIVE_KAFKA_TOPIC = "kafka.topic"; + /** + * MANDATORY Table property indicating kafka broker(s) connection string. + */ + public 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} + */ + public 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} + */ + public static final String HIVE_KAFKA_POLL_TIMEOUT = "hive.kafka.poll.timeout.ms"; + /** + * default poll timeout for fetching metadata and record batch + */ + public static final long DEFAULT_CONSUMER_POLL_TIMEOUT_MS = 5000L; // 5 seconds + /** + * Record Timestamp column name, added as extra meta column of type long + */ + public static final String TIMESTAMP_COLUMN = "__timestamp"; + /** + * Record Kafka Partition column name added as extra meta column of type int + */ + public static final String PARTITION_COLUMN = "__partition"; + /** + * Record offset column name added as extra metadata column to row as long + */ + public static final String OFFSET_COLUMN = "__offset"; + /** + * 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 + */ + protected static final String CONSUMER_CONFIGURATION_PREFIX = "kafka.consumer"; + private KafkaStreamingUtils() { } @@ -60,16 +99,16 @@ public static Properties consumerProperties(Configuration configuration) { props.setProperty("enable.auto.commit", "false"); // we are seeking in the stream so no reset props.setProperty("auto.offset.reset", "none"); - String brokerEndPoint = configuration.get(KafkaStorageHandler.HIVE_KAFKA_BOOTSTRAP_SERVERS); + String brokerEndPoint = configuration.get(HIVE_KAFKA_BOOTSTRAP_SERVERS); props.setProperty(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, brokerEndPoint); props.setProperty(KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName()); props.setProperty(VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName()); // user can always override stuff final Map kafkaProperties = - configuration.getValByRegex("^" + KafkaStorageHandler.CONSUMER_CONFIGURATION_PREFIX + "\\..*"); + configuration.getValByRegex("^" + CONSUMER_CONFIGURATION_PREFIX + "\\..*"); for (Map.Entry entry : kafkaProperties.entrySet()) { - props.setProperty(entry.getKey().substring(KafkaStorageHandler.CONSUMER_CONFIGURATION_PREFIX.length() + 1), + props.setProperty(entry.getKey().substring(CONSUMER_CONFIGURATION_PREFIX.length() + 1), entry.getValue()); } return props; diff --git a/kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaScanTrimmerTest.java b/kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaScanTrimmerTest.java index 6444b47c4564..289dafde367d 100644 --- a/kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaScanTrimmerTest.java +++ b/kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaScanTrimmerTest.java @@ -61,10 +61,10 @@ public class KafkaScanTrimmerTest { private ExprNodeDesc partitionColumn = - new ExprNodeColumnDesc(TypeInfoFactory.intTypeInfo, KafkaStorageHandler.PARTITION_COLUMN, null, false); + new ExprNodeColumnDesc(TypeInfoFactory.intTypeInfo, KafkaStreamingUtils.PARTITION_COLUMN, null, false); private ExprNodeDesc offsetColumn = - new ExprNodeColumnDesc(TypeInfoFactory.longTypeInfo, KafkaStorageHandler.OFFSET_COLUMN, null, false); + new ExprNodeColumnDesc(TypeInfoFactory.longTypeInfo, KafkaStreamingUtils.OFFSET_COLUMN, null, false); private String topic = "my_topic"; private Map From f764f8adff37704bf81c284d7b48914264de9b94 Mon Sep 17 00:00:00 2001 From: Slim Bouguerra Date: Fri, 17 Aug 2018 15:56:11 -0700 Subject: [PATCH 18/33] remove unsed kafka propreties Change-Id: Idc1bd85d8fc88df6ab0238bb8dfe50344264d78c --- .../org/apache/hadoop/hive/kafka/KafkaStreamingUtils.java | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaStreamingUtils.java b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaStreamingUtils.java index fa434d9c788c..1c75a47c9db0 100644 --- a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaStreamingUtils.java +++ b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaStreamingUtils.java @@ -91,10 +91,8 @@ private KafkaStreamingUtils() { */ public static Properties consumerProperties(Configuration configuration) { final Properties props = new Properties(); - // those are very important to set to avoid long blocking - props.setProperty("request.timeout.ms", "10001"); - props.setProperty("fetch.max.wait.ms", "10000"); - props.setProperty("session.timeout.ms", "10000"); + // important to set to avoid long blocking + props.setProperty(CommonClientConfigs.REQUEST_TIMEOUT_MS_CONFIG, "10000"); // we are managing the commit offset props.setProperty("enable.auto.commit", "false"); // we are seeking in the stream so no reset From 5ddac6e0c026294785d26f4f92bfa608caf6f35c Mon Sep 17 00:00:00 2001 From: Slim Bouguerra Date: Fri, 17 Aug 2018 15:59:23 -0700 Subject: [PATCH 19/33] more cleanup Change-Id: I2b493ef799ebb9fcf4d5c3847fb5c375337f6075 --- .../java/org/apache/hadoop/hive/kafka/KafkaStreamingUtils.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaStreamingUtils.java b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaStreamingUtils.java index 1c75a47c9db0..d0cc58274b6a 100644 --- a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaStreamingUtils.java +++ b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaStreamingUtils.java @@ -91,8 +91,6 @@ private KafkaStreamingUtils() { */ public static Properties consumerProperties(Configuration configuration) { final Properties props = new Properties(); - // important to set to avoid long blocking - props.setProperty(CommonClientConfigs.REQUEST_TIMEOUT_MS_CONFIG, "10000"); // we are managing the commit offset props.setProperty("enable.auto.commit", "false"); // we are seeking in the stream so no reset From 3cdeb41ce04b75631b8832440bd945ef5ef6c852 Mon Sep 17 00:00:00 2001 From: Slim Bouguerra Date: Fri, 17 Aug 2018 16:14:02 -0700 Subject: [PATCH 20/33] refactor iterator test Change-Id: Iff2d04a62cdb0053af272679ea0530ceedea435e --- .../hive/kafka/KafkaRecordIteratorTest.java | 40 +++++++++---------- 1 file changed, 19 insertions(+), 21 deletions(-) diff --git a/kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaRecordIteratorTest.java b/kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaRecordIteratorTest.java index acb7ea66b450..2fd7c78204b1 100644 --- a/kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaRecordIteratorTest.java +++ b/kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaRecordIteratorTest.java @@ -55,22 +55,27 @@ import java.io.IOException; import java.nio.charset.Charset; import java.nio.file.Files; -import java.util.ArrayList; import java.util.Iterator; import java.util.List; import java.util.Properties; import java.util.stream.Collectors; +import java.util.stream.IntStream; /** * Kafka Iterator Tests. */ public class KafkaRecordIteratorTest { private static final Logger LOG = LoggerFactory.getLogger(KafkaRecordIteratorTest.class); - private static final String TOPIC = "my_test_topic"; - private static final List> RECORDS = new ArrayList(); private static final int RECORD_NUMBER = 100; + private static final String TOPIC = "my_test_topic"; private static final TopicPartition TOPIC_PARTITION = new TopicPartition(TOPIC, 0); public 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 ZkUtils zkUtils; private static ZkClient zkClient; private static KafkaProducer producer; @@ -189,8 +194,7 @@ public KafkaRecordIteratorTest() { } @Test(expected = TimeoutException.class) public void testPullingBeyondLimit() { - this.kafkaRecordIterator = - new KafkaRecordIterator(this.consumer, TOPIC_PARTITION, 0L, 101L, 100L); + this.kafkaRecordIterator = new KafkaRecordIterator(this.consumer, TOPIC_PARTITION, 0L, 101L, 100L); this.compareIterator(RECORDS, this.kafkaRecordIterator); } @@ -205,8 +209,7 @@ public KafkaRecordIteratorTest() { } @Test(expected = TimeoutException.class) public void testPullingFromEmptyPartition() { - this.kafkaRecordIterator = - new KafkaRecordIterator(this.consumer, new TopicPartition(TOPIC, 1), 0L, 100L, 100L); + this.kafkaRecordIterator = new KafkaRecordIterator(this.consumer, new TopicPartition(TOPIC, 1), 0L, 100L, 100L); this.compareIterator(RECORDS, this.kafkaRecordIterator); } @@ -222,8 +225,7 @@ public KafkaRecordIteratorTest() { } @Test public void testStartIsTheFirstOffset() { - this.kafkaRecordIterator = - new KafkaRecordIterator(this.consumer, TOPIC_PARTITION, 0L, 0L, 100L); + this.kafkaRecordIterator = new KafkaRecordIterator(this.consumer, TOPIC_PARTITION, 0L, 0L, 100L); this.compareIterator(ImmutableList.of(), this.kafkaRecordIterator); } @@ -271,19 +273,15 @@ private void setupConsumer() { this.consumer = new KafkaConsumer(consumerProps); } - private static void sendData() throws InterruptedException { + private static void sendData() { LOG.info("Sending {} records", RECORD_NUMBER); - RECORDS.clear(); - for (int i = 0; i < RECORD_NUMBER; ++i) { - - final byte[] value = ("VALUE-" + Integer.toString(i)).getBytes(Charset.forName("UTF-8")); - //noinspection unchecked - producer.send(new ProducerRecord(TOPIC, 0, 0L, KEY_BYTES, value)); - - //noinspection unchecked - RECORDS.add(new ConsumerRecord(TOPIC, 0, (long) i, 0L, null, 0L, 0, 0, KEY_BYTES, value)); - } - + RECORDS.stream() + .map(consumerRecord -> new ProducerRecord(consumerRecord.topic(), + consumerRecord.partition(), + consumerRecord.timestamp(), + consumerRecord.key(), + consumerRecord.value())) + .forEach(producerRecord -> producer.send(producerRecord)); producer.close(); } From c023e51b5ab28f8fd432e1f93a2cf1cb84656854 Mon Sep 17 00:00:00 2001 From: Slim Bouguerra Date: Fri, 17 Aug 2018 18:40:41 -0700 Subject: [PATCH 21/33] adding supplier for reader Change-Id: I3bb58d13be3e32bb7e9f5df6574004a3645bbb75 --- kafka-handler/pom.xml | 4 ++ .../hadoop/hive/kafka/GenericKafkaSerDe.java | 37 +++++++++++++++---- 2 files changed, 34 insertions(+), 7 deletions(-) diff --git a/kafka-handler/pom.xml b/kafka-handler/pom.xml index cbcb2fb170d2..6c58bf1df19c 100644 --- a/kafka-handler/pom.xml +++ b/kafka-handler/pom.xml @@ -45,6 +45,10 @@ provided ${project.version} + + com.google.guava + guava + org.apache.hadoop hadoop-common diff --git a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/GenericKafkaSerDe.java b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/GenericKafkaSerDe.java index a918ca5ef17c..a784d121165d 100644 --- a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/GenericKafkaSerDe.java +++ b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/GenericKafkaSerDe.java @@ -18,8 +18,16 @@ 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.ImmutableList; 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.serde.serdeConstants; import org.apache.hadoop.hive.serde2.AbstractSerDe; @@ -29,7 +37,7 @@ import org.apache.hadoop.hive.serde2.SerDeUtils; import org.apache.hadoop.hive.serde2.avro.AvroGenericRecordWritable; import org.apache.hadoop.hive.serde2.avro.AvroSerDe; -import org.apache.hadoop.hive.serde2.columnar.BytesRefWritable; +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.StructObjectInspector; @@ -45,9 +53,8 @@ import org.slf4j.LoggerFactory; import javax.annotation.Nullable; -import java.io.ByteArrayInputStream; -import java.io.DataInputStream; import java.io.IOException; +import java.rmi.server.UID; import java.util.ArrayList; import java.util.Arrays; import java.util.List; @@ -55,8 +62,8 @@ 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 BytesRefWritable}. + * 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); @@ -75,6 +82,11 @@ public class GenericKafkaSerDe extends AbstractSerDe { private final List columnNames = Lists.newArrayList(); StructObjectInspector delegateObjectInspector; + //GenericDatumReader gdr; + //DatumReader reader = new SpecificDatumReader<>(schema); + private final UID uid = new UID(); + 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 = createDelegate(className); @@ -116,6 +128,13 @@ public class GenericKafkaSerDe extends AbstractSerDe { .map(KafkaJsonSerDe.typeInfoToObjectInspector) .collect(Collectors.toList())); + gdrSupplier = 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); + return new SpecificDatumReader<>(schema); + }); + objectInspector = ObjectInspectorFactory.getStandardStructObjectInspector(columnNames, inspectors); } @@ -153,8 +172,12 @@ private AbstractSerDe createDelegate(String className) { } else if (delegateSerDe instanceof AvroSerDe) { AvroGenericRecordWritable avroGenericRecordWritable = new AvroGenericRecordWritable(); try { - ByteArrayInputStream byteArrayInputStream = new ByteArrayInputStream(record.getValue()); - avroGenericRecordWritable.readFields(new DataInputStream(byteArrayInputStream)); + GenericRecord + 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); } From 59f7872b95f5668f629df52104e7807a52c22b97 Mon Sep 17 00:00:00 2001 From: Slim Bouguerra Date: Fri, 17 Aug 2018 19:59:52 -0700 Subject: [PATCH 22/33] fix the order of columns for Avro Serd Change-Id: I5dfc21059e530976af04771b6ea1d5d467774a0a --- .../hadoop/hive/kafka/GenericKafkaSerDe.java | 43 ++++++------------- 1 file changed, 12 insertions(+), 31 deletions(-) diff --git a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/GenericKafkaSerDe.java b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/GenericKafkaSerDe.java index a784d121165d..d3c9bcbcf2d6 100644 --- a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/GenericKafkaSerDe.java +++ b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/GenericKafkaSerDe.java @@ -29,12 +29,10 @@ import org.apache.avro.io.DecoderFactory; 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.JsonSerDe; import org.apache.hadoop.hive.serde2.SerDeException; import org.apache.hadoop.hive.serde2.SerDeStats; -import org.apache.hadoop.hive.serde2.SerDeUtils; import org.apache.hadoop.hive.serde2.avro.AvroGenericRecordWritable; import org.apache.hadoop.hive.serde2.avro.AvroSerDe; import org.apache.hadoop.hive.serde2.avro.AvroSerdeUtils; @@ -56,7 +54,6 @@ import java.io.IOException; import java.rmi.server.UID; import java.util.ArrayList; -import java.util.Arrays; import java.util.List; import java.util.Properties; import java.util.stream.Collectors; @@ -80,10 +77,7 @@ public class GenericKafkaSerDe extends AbstractSerDe { private AbstractSerDe delegateSerDe; private ObjectInspector objectInspector; private final List columnNames = Lists.newArrayList(); - StructObjectInspector delegateObjectInspector; - - //GenericDatumReader gdr; - //DatumReader reader = new SpecificDatumReader<>(schema); + private StructObjectInspector delegateObjectInspector; private final UID uid = new UID(); private Supplier> gdrSupplier; @@ -97,29 +91,16 @@ public class GenericKafkaSerDe extends AbstractSerDe { .getClass() .getName()); } - delegateObjectInspector = (StructObjectInspector) delegateSerDe.getObjectInspector(); - final List inspectors; - // Get column names and types - String columnNameProperty = tbl.getProperty(serdeConstants.LIST_COLUMNS); - final String - columnNameDelimiter = - tbl.containsKey(serdeConstants.COLUMN_NAME_DELIMITER) ? - tbl.getProperty(serdeConstants.COLUMN_NAME_DELIMITER) : - String.valueOf(SerDeUtils.COMMA); - // all table column names - if (!columnNameProperty.isEmpty()) { - columnNames.addAll(Arrays.asList(columnNameProperty.split(columnNameDelimiter))); - } - + // Build column names Order matters here + columnNames.addAll(delegateObjectInspector.getAllStructFieldRefs() + .stream() + .map(element -> element.getFieldName()) + .collect(Collectors.toList())); columnNames.addAll(METADATA_COLUMN_NAMES); - if (LOG.isDebugEnabled()) { - LOG.debug("columns: {}, {}", columnNameProperty, columnNames); - } - - inspectors = new ArrayList<>(columnNames.size()); + final List inspectors = new ArrayList<>(columnNames.size()); inspectors.addAll(delegateObjectInspector.getAllStructFieldRefs() .stream() .map(structField -> structField.getFieldObjectInspector()) @@ -127,15 +108,16 @@ public class GenericKafkaSerDe extends AbstractSerDe { inspectors.addAll(METADATA_PRIMITIVE_TYPE_INFO.stream() .map(KafkaJsonSerDe.typeInfoToObjectInspector) .collect(Collectors.toList())); + objectInspector = ObjectInspectorFactory.getStandardStructObjectInspector(columnNames, inspectors); + // lazy supplier to read Avro Records if needed gdrSupplier = 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.info("Building Avro Reader with schema {}", schemaFromProperty); return new SpecificDatumReader<>(schema); }); - - objectInspector = ObjectInspectorFactory.getStandardStructObjectInspector(columnNames, inspectors); } private AbstractSerDe createDelegate(String className) { @@ -171,10 +153,9 @@ private AbstractSerDe createDelegate(String className) { row = delegateSerDe.deserialize(new Text(record.getValue())); } else if (delegateSerDe instanceof AvroSerDe) { AvroGenericRecordWritable avroGenericRecordWritable = new AvroGenericRecordWritable(); + GenericRecord avroRecord; try { - GenericRecord - avroRecord = - gdrSupplier.get().read(null, DecoderFactory.get().binaryDecoder(record.getValue(), null)); + avroRecord = gdrSupplier.get().read(null, DecoderFactory.get().binaryDecoder(record.getValue(), null)); avroGenericRecordWritable.setRecord(avroRecord); avroGenericRecordWritable.setRecordReaderID(uid); avroGenericRecordWritable.setFileSchema(avroRecord.getSchema()); From 00615766903dbf10fa8a3c8c94f55cbeacbf626f Mon Sep 17 00:00:00 2001 From: Slim Bouguerra Date: Fri, 17 Aug 2018 20:34:05 -0700 Subject: [PATCH 23/33] fix the json parser to use bytewritables Change-Id: I2e1f63b3a10fe45a4eaad04c46bbba62e6a58b3b --- .../java/org/apache/hadoop/hive/kafka/KafkaJsonSerDe.java | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaJsonSerDe.java b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaJsonSerDe.java index cdc2a9ec975c..3de64dca44d3 100644 --- a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaJsonSerDe.java +++ b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaJsonSerDe.java @@ -49,6 +49,7 @@ import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils; import org.apache.hadoop.io.BooleanWritable; +import org.apache.hadoop.io.BytesWritable; import org.apache.hadoop.io.FloatWritable; import org.apache.hadoop.io.IntWritable; import org.apache.hadoop.io.LongWritable; @@ -135,12 +136,12 @@ public class KafkaJsonSerDe extends AbstractSerDe { } @Override public Object deserialize(Writable blob) throws SerDeException { - BytesRefWritable record = (BytesRefWritable) blob; + BytesWritable record = (BytesWritable) blob; Map payload; try { - payload = parseAsJson(record.getData()); + payload = parseAsJson(record.getBytes()); rowCount += 1; - rawDataSize += record.getData().length; + rawDataSize += record.getLength(); } catch (IOException e) { throw new SerDeException(e); } From 711e84164c1f4b959d46347e8def94cfd9b79573 Mon Sep 17 00:00:00 2001 From: Slim Bouguerra Date: Sat, 18 Aug 2018 07:40:59 -0700 Subject: [PATCH 24/33] small refactor for generic serde pulling inline code to fns Change-Id: I90381220de28f8ba0da87ad2b7ddf2372c7288ed --- .../hadoop/hive/kafka/GenericKafkaSerDe.java | 48 ++++++++----------- .../hive/kafka/KafkaStreamingUtils.java | 17 ++++++- 2 files changed, 36 insertions(+), 29 deletions(-) diff --git a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/GenericKafkaSerDe.java b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/GenericKafkaSerDe.java index d3c9bcbcf2d6..8b42cb7558de 100644 --- a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/GenericKafkaSerDe.java +++ b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/GenericKafkaSerDe.java @@ -38,6 +38,7 @@ 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.typeinfo.PrimitiveTypeInfo; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; @@ -46,7 +47,6 @@ import org.apache.hadoop.io.LongWritable; import org.apache.hadoop.io.Text; import org.apache.hadoop.io.Writable; -import org.apache.hive.common.util.ReflectionUtil; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -65,12 +65,12 @@ public class GenericKafkaSerDe extends AbstractSerDe { private static final Logger LOG = LoggerFactory.getLogger(GenericKafkaSerDe.class); // ORDER of fields and types matters here - public static final ImmutableList + private static final ImmutableList METADATA_COLUMN_NAMES = ImmutableList.of(KafkaStreamingUtils.PARTITION_COLUMN, KafkaStreamingUtils.OFFSET_COLUMN, KafkaStreamingUtils.TIMESTAMP_COLUMN); - public static final ImmutableList + private static final ImmutableList METADATA_PRIMITIVE_TYPE_INFO = ImmutableList.of(TypeInfoFactory.intTypeInfo, TypeInfoFactory.longTypeInfo, TypeInfoFactory.longTypeInfo); @@ -79,13 +79,15 @@ public class GenericKafkaSerDe extends AbstractSerDe { private final List columnNames = Lists.newArrayList(); private StructObjectInspector delegateObjectInspector; private final UID uid = new UID(); - private Supplier> gdrSupplier; + @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 = createDelegate(className); + delegateSerDe = KafkaStreamingUtils.createDelegate(className); + //noinspection deprecation delegateSerDe.initialize(conf, tbl); - LOG.info("Using SerDe instance {}", delegateSerDe.getClass().getCanonicalName()); + 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() @@ -96,14 +98,14 @@ public class GenericKafkaSerDe extends AbstractSerDe { // Build column names Order matters here columnNames.addAll(delegateObjectInspector.getAllStructFieldRefs() .stream() - .map(element -> element.getFieldName()) + .map(StructField::getFieldName) .collect(Collectors.toList())); columnNames.addAll(METADATA_COLUMN_NAMES); final List inspectors = new ArrayList<>(columnNames.size()); inspectors.addAll(delegateObjectInspector.getAllStructFieldRefs() .stream() - .map(structField -> structField.getFieldObjectInspector()) + .map(StructField::getFieldObjectInspector) .collect(Collectors.toList())); inspectors.addAll(METADATA_PRIMITIVE_TYPE_INFO.stream() .map(KafkaJsonSerDe.typeInfoToObjectInspector) @@ -111,25 +113,7 @@ public class GenericKafkaSerDe extends AbstractSerDe { objectInspector = ObjectInspectorFactory.getStandardStructObjectInspector(columnNames, inspectors); // lazy supplier to read Avro Records if needed - gdrSupplier = 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.info("Building Avro Reader with schema {}", schemaFromProperty); - return new SpecificDatumReader<>(schema); - }); - } - - private AbstractSerDe createDelegate(String className) { - final Class clazz; - try { - clazz = (Class) Class.forName(className); - } catch (ClassNotFoundException e) { - LOG.error("Failed a loading delegate SerDe {}", className); - throw new RuntimeException(e); - } - // we are not setting conf thus null is okay - return ReflectionUtil.newInstance(clazz, null); + gdrSupplier = getReaderSupplier(tbl); } @Override public Class getSerializedClass() { @@ -185,4 +169,14 @@ private AbstractSerDe createDelegate(String className) { @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 a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaStreamingUtils.java b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaStreamingUtils.java index d0cc58274b6a..3df5d49a0891 100644 --- a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaStreamingUtils.java +++ b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaStreamingUtils.java @@ -22,7 +22,9 @@ 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.common.serialization.ByteArrayDeserializer; @@ -89,7 +91,7 @@ private KafkaStreamingUtils() { * * @return default consumer properties */ - public static Properties consumerProperties(Configuration configuration) { + static Properties consumerProperties(Configuration configuration) { final Properties props = new Properties(); // we are managing the commit offset props.setProperty("enable.auto.commit", "false"); @@ -110,7 +112,7 @@ public static Properties consumerProperties(Configuration configuration) { return props; } - public static void copyDependencyJars(Configuration conf, Class... classes) throws IOException { + static void copyDependencyJars(Configuration conf, Class... classes) throws IOException { Set jars = new HashSet<>(); FileSystem localFs = FileSystem.getLocal(conf); jars.addAll(conf.getStringCollection("tmpjars")); @@ -137,4 +139,15 @@ public static void copyDependencyJars(Configuration conf, Class... classes) t } conf.set("tmpjars", StringUtils.arrayToString(jars.toArray(new String[jars.size()]))); } + + static AbstractSerDe createDelegate(String className) { + final Class clazz; + try { + 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); + } } From 71a292b3fefc05fdff16ea0a5d35482fd858fdcb Mon Sep 17 00:00:00 2001 From: Slim Bouguerra Date: Sun, 19 Aug 2018 21:40:26 -0700 Subject: [PATCH 25/33] added avro to kafka tests Change-Id: I17e1641061c5a5b816aa8be4d761ead2064b946e --- .../hive/kafka/SingleNodeKafkaCluster.java | 38 +- .../java/org/apache/hive/kafka/Wikipedia.java | 1418 +++++++++++++++++ .../org/apache/hadoop/hive/ql/QTestUtil.java | 56 +- .../clientpositive/kafka_storage_handler.q | 85 +- .../druid/kafka_storage_handler.q.out | 231 +++ .../serde2/avro/AvroLazyObjectInspector.java | 6 +- 6 files changed, 1825 insertions(+), 9 deletions(-) create mode 100644 itests/qtest-druid/src/main/java/org/apache/hive/kafka/Wikipedia.java diff --git a/itests/qtest-druid/src/main/java/org/apache/hive/kafka/SingleNodeKafkaCluster.java b/itests/qtest-druid/src/main/java/org/apache/hive/kafka/SingleNodeKafkaCluster.java index d839fd2db422..8dc26c6cf853 100644 --- a/itests/qtest-druid/src/main/java/org/apache/hive/kafka/SingleNodeKafkaCluster.java +++ b/itests/qtest-druid/src/main/java/org/apache/hive/kafka/SingleNodeKafkaCluster.java @@ -10,6 +10,7 @@ import org.apache.hadoop.service.AbstractService; import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.serialization.ByteArraySerializer; import org.apache.kafka.common.serialization.StringSerializer; import com.google.common.base.Throwables; @@ -25,6 +26,7 @@ import java.nio.charset.Charset; import java.util.List; import java.util.Properties; +import java.util.stream.IntStream; /** * This class has the hooks to start and stop single node kafka cluster. @@ -32,6 +34,10 @@ */ public class SingleNodeKafkaCluster extends AbstractService { private static final Logger log = LoggerFactory.getLogger(SingleNodeKafkaCluster.class); + private static final int BROKER_PORT = 9092; + private static final String LOCALHOST = "localhost"; + private static final String LOCALHOST_9092 = String.format("%s:%s", LOCALHOST, BROKER_PORT); + private final KafkaServerStartable serverStartable; private final String zkString; @@ -42,8 +48,8 @@ public SingleNodeKafkaCluster(String name, String logDir, Integer zkPort){ this.zkString = String.format("localhost:%d", zkPort); properties.setProperty("zookeeper.connect", zkString); properties.setProperty("broker.id", String.valueOf(1)); - properties.setProperty("host.name", "localhost"); - properties.setProperty("port", Integer.toString(9092)); + properties.setProperty("host.name", LOCALHOST); + properties.setProperty("port", Integer.toString(BROKER_PORT)); properties.setProperty("log.dir", logDir); properties.setProperty("log.flush.interval.messages", String.valueOf(1)); properties.setProperty("offsets.topic.replication.factor", String.valueOf(1)); @@ -80,7 +86,7 @@ public void createTopicWithData(String topicName, File datafile){ createTopic(topicName); // set up kafka producer Properties properties = new Properties(); - properties.put("bootstrap.servers", "localhost:9092"); + properties.put("bootstrap.servers", LOCALHOST_9092); properties.put("acks", "1"); properties.put("retries", "3"); @@ -91,12 +97,36 @@ public void createTopicWithData(String topicName, File datafile){ )){ List events = Files.readLines(datafile, Charset.forName("UTF-8")); for(String event : events){ - producer.send(new ProducerRecord(topicName, event)); + producer.send(new ProducerRecord<>(topicName, event)); } } catch (IOException e) { Throwables.propagate(e); } + } + + public void createTopicWithData(String topic, List events) { + createTopic(topic); + // set up kafka producer + Properties properties = new Properties(); + properties.put("bootstrap.servers", LOCALHOST_9092); + properties.put("acks", "1"); + properties.put("retries", "3"); + try(KafkaProducer producer = new KafkaProducer<>( + properties, + new ByteArraySerializer(), + new ByteArraySerializer() + )){ + // 1534736225090 -> 08/19/2018 20:37:05 + IntStream.range(0, events.size()) + .mapToObj(i -> new ProducerRecord<>(topic, + null, + // 1534736225090 -> Mon Aug 20 2018 03:37:05 + 1534736225090L + 1000 * 3600 * i, + "key".getBytes(), + events.get(i))) + .forEach(r -> producer.send(r)); + } } public void createTopic(String topic) { diff --git a/itests/qtest-druid/src/main/java/org/apache/hive/kafka/Wikipedia.java b/itests/qtest-druid/src/main/java/org/apache/hive/kafka/Wikipedia.java new file mode 100644 index 000000000000..2fb180b33a87 --- /dev/null +++ b/itests/qtest-druid/src/main/java/org/apache/hive/kafka/Wikipedia.java @@ -0,0 +1,1418 @@ +/** + * Autogenerated by Avro + * + * DO NOT EDIT DIRECTLY + */ +package org.apache.hive.kafka; + +import org.apache.avro.specific.SpecificData; +import org.apache.avro.message.BinaryMessageEncoder; +import org.apache.avro.message.BinaryMessageDecoder; +import org.apache.avro.message.SchemaStore; + +@SuppressWarnings("all") +@org.apache.avro.specific.AvroGenerated +public class Wikipedia extends org.apache.avro.specific.SpecificRecordBase implements org.apache.avro.specific.SpecificRecord { + private static final long serialVersionUID = 960374719287820723L; + public static final org.apache.avro.Schema SCHEMA$ = new org.apache.avro.Schema.Parser().parse("{\"type\":\"record\",\"name\":\"Wikipedia\",\"namespace\":\"org.apache.hive.kafka\",\"fields\":[{\"name\":\"isrobot\",\"type\":\"boolean\"},{\"name\":\"channel\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"}},{\"name\":\"timestamp\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"}},{\"name\":\"flags\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"}},{\"name\":\"isunpatrolled\",\"type\":\"boolean\"},{\"name\":\"page\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"}},{\"name\":\"diffurl\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"}},{\"name\":\"added\",\"type\":\"long\"},{\"name\":\"comment\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"}},{\"name\":\"commentlength\",\"type\":\"long\"},{\"name\":\"isnew\",\"type\":\"boolean\"},{\"name\":\"isminor\",\"type\":\"boolean\"},{\"name\":\"delta\",\"type\":\"long\"},{\"name\":\"isanonymous\",\"type\":\"boolean\"},{\"name\":\"user\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"}},{\"name\":\"deltabucket\",\"type\":\"double\"},{\"name\":\"deleted\",\"type\":\"long\"},{\"name\":\"namespace\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"}}],\"version\":\"1\"}"); + public static org.apache.avro.Schema getClassSchema() { return SCHEMA$; } + + private static SpecificData MODEL$ = new SpecificData(); + + private static final BinaryMessageEncoder ENCODER = + new BinaryMessageEncoder(MODEL$, SCHEMA$); + + private static final BinaryMessageDecoder DECODER = + new BinaryMessageDecoder(MODEL$, SCHEMA$); + + /** + * Return the BinaryMessageDecoder instance used by this class. + */ + public static BinaryMessageDecoder getDecoder() { + return DECODER; + } + + /** + * Create a new BinaryMessageDecoder instance for this class that uses the specified {@link SchemaStore}. + * @param resolver a {@link SchemaStore} used to find schemas by fingerprint + */ + public static BinaryMessageDecoder createDecoder(SchemaStore resolver) { + return new BinaryMessageDecoder(MODEL$, SCHEMA$, resolver); + } + + /** Serializes this Wikipedia to a ByteBuffer. */ + public java.nio.ByteBuffer toByteBuffer() throws java.io.IOException { + return ENCODER.encode(this); + } + + /** Deserializes a Wikipedia from a ByteBuffer. */ + public static Wikipedia fromByteBuffer( + java.nio.ByteBuffer b) throws java.io.IOException { + return DECODER.decode(b); + } + + @Deprecated public boolean isrobot; + @Deprecated public java.lang.String channel; + @Deprecated public java.lang.String timestamp; + @Deprecated public java.lang.String flags; + @Deprecated public boolean isunpatrolled; + @Deprecated public java.lang.String page; + @Deprecated public java.lang.String diffurl; + @Deprecated public long added; + @Deprecated public java.lang.String comment; + @Deprecated public long commentlength; + @Deprecated public boolean isnew; + @Deprecated public boolean isminor; + @Deprecated public long delta; + @Deprecated public boolean isanonymous; + @Deprecated public java.lang.String user; + @Deprecated public double deltabucket; + @Deprecated public long deleted; + @Deprecated public java.lang.String namespace; + + /** + * Default constructor. Note that this does not initialize fields + * to their default values from the schema. If that is desired then + * one should use newBuilder(). + */ + public Wikipedia() {} + + /** + * All-args constructor. + * @param isrobot The new value for isrobot + * @param channel The new value for channel + * @param timestamp The new value for timestamp + * @param flags The new value for flags + * @param isunpatrolled The new value for isunpatrolled + * @param page The new value for page + * @param diffurl The new value for diffurl + * @param added The new value for added + * @param comment The new value for comment + * @param commentlength The new value for commentlength + * @param isnew The new value for isnew + * @param isminor The new value for isminor + * @param delta The new value for delta + * @param isanonymous The new value for isanonymous + * @param user The new value for user + * @param deltabucket The new value for deltabucket + * @param deleted The new value for deleted + * @param namespace The new value for namespace + */ + public Wikipedia(java.lang.Boolean isrobot, java.lang.String channel, java.lang.String timestamp, java.lang.String flags, java.lang.Boolean isunpatrolled, java.lang.String page, java.lang.String diffurl, java.lang.Long added, java.lang.String comment, java.lang.Long commentlength, java.lang.Boolean isnew, java.lang.Boolean isminor, java.lang.Long delta, java.lang.Boolean isanonymous, java.lang.String user, java.lang.Double deltabucket, java.lang.Long deleted, java.lang.String namespace) { + this.isrobot = isrobot; + this.channel = channel; + this.timestamp = timestamp; + this.flags = flags; + this.isunpatrolled = isunpatrolled; + this.page = page; + this.diffurl = diffurl; + this.added = added; + this.comment = comment; + this.commentlength = commentlength; + this.isnew = isnew; + this.isminor = isminor; + this.delta = delta; + this.isanonymous = isanonymous; + this.user = user; + this.deltabucket = deltabucket; + this.deleted = deleted; + this.namespace = namespace; + } + + public org.apache.avro.Schema getSchema() { return SCHEMA$; } + // Used by DatumWriter. Applications should not call. + public java.lang.Object get(int field$) { + switch (field$) { + case 0: return isrobot; + case 1: return channel; + case 2: return timestamp; + case 3: return flags; + case 4: return isunpatrolled; + case 5: return page; + case 6: return diffurl; + case 7: return added; + case 8: return comment; + case 9: return commentlength; + case 10: return isnew; + case 11: return isminor; + case 12: return delta; + case 13: return isanonymous; + case 14: return user; + case 15: return deltabucket; + case 16: return deleted; + case 17: return namespace; + default: throw new org.apache.avro.AvroRuntimeException("Bad index"); + } + } + + // Used by DatumReader. Applications should not call. + @SuppressWarnings(value="unchecked") + public void put(int field$, java.lang.Object value$) { + switch (field$) { + case 0: isrobot = (java.lang.Boolean)value$; break; + case 1: channel = (java.lang.String)value$.toString(); break; + case 2: timestamp = (java.lang.String)value$.toString(); break; + case 3: flags = (java.lang.String)value$.toString(); break; + case 4: isunpatrolled = (java.lang.Boolean)value$; break; + case 5: page = (java.lang.String)value$.toString(); break; + case 6: diffurl = (java.lang.String)value$.toString(); break; + case 7: added = (java.lang.Long)value$; break; + case 8: comment = (java.lang.String)value$.toString(); break; + case 9: commentlength = (java.lang.Long)value$; break; + case 10: isnew = (java.lang.Boolean)value$; break; + case 11: isminor = (java.lang.Boolean)value$; break; + case 12: delta = (java.lang.Long)value$; break; + case 13: isanonymous = (java.lang.Boolean)value$; break; + case 14: user = (java.lang.String)value$.toString(); break; + case 15: deltabucket = (java.lang.Double)value$; break; + case 16: deleted = (java.lang.Long)value$; break; + case 17: namespace = (java.lang.String)value$.toString(); break; + default: throw new org.apache.avro.AvroRuntimeException("Bad index"); + } + } + + /** + * Gets the value of the 'isrobot' field. + * @return The value of the 'isrobot' field. + */ + public java.lang.Boolean getIsrobot() { + return isrobot; + } + + /** + * Sets the value of the 'isrobot' field. + * @param value the value to set. + */ + public void setIsrobot(java.lang.Boolean value) { + this.isrobot = value; + } + + /** + * Gets the value of the 'channel' field. + * @return The value of the 'channel' field. + */ + public java.lang.String getChannel() { + return channel; + } + + /** + * Sets the value of the 'channel' field. + * @param value the value to set. + */ + public void setChannel(java.lang.String value) { + this.channel = value; + } + + /** + * Gets the value of the 'timestamp' field. + * @return The value of the 'timestamp' field. + */ + public java.lang.String getTimestamp() { + return timestamp; + } + + /** + * Sets the value of the 'timestamp' field. + * @param value the value to set. + */ + public void setTimestamp(java.lang.String value) { + this.timestamp = value; + } + + /** + * Gets the value of the 'flags' field. + * @return The value of the 'flags' field. + */ + public java.lang.String getFlags() { + return flags; + } + + /** + * Sets the value of the 'flags' field. + * @param value the value to set. + */ + public void setFlags(java.lang.String value) { + this.flags = value; + } + + /** + * Gets the value of the 'isunpatrolled' field. + * @return The value of the 'isunpatrolled' field. + */ + public java.lang.Boolean getIsunpatrolled() { + return isunpatrolled; + } + + /** + * Sets the value of the 'isunpatrolled' field. + * @param value the value to set. + */ + public void setIsunpatrolled(java.lang.Boolean value) { + this.isunpatrolled = value; + } + + /** + * Gets the value of the 'page' field. + * @return The value of the 'page' field. + */ + public java.lang.String getPage() { + return page; + } + + /** + * Sets the value of the 'page' field. + * @param value the value to set. + */ + public void setPage(java.lang.String value) { + this.page = value; + } + + /** + * Gets the value of the 'diffurl' field. + * @return The value of the 'diffurl' field. + */ + public java.lang.String getDiffurl() { + return diffurl; + } + + /** + * Sets the value of the 'diffurl' field. + * @param value the value to set. + */ + public void setDiffurl(java.lang.String value) { + this.diffurl = value; + } + + /** + * Gets the value of the 'added' field. + * @return The value of the 'added' field. + */ + public java.lang.Long getAdded() { + return added; + } + + /** + * Sets the value of the 'added' field. + * @param value the value to set. + */ + public void setAdded(java.lang.Long value) { + this.added = value; + } + + /** + * Gets the value of the 'comment' field. + * @return The value of the 'comment' field. + */ + public java.lang.String getComment() { + return comment; + } + + /** + * Sets the value of the 'comment' field. + * @param value the value to set. + */ + public void setComment(java.lang.String value) { + this.comment = value; + } + + /** + * Gets the value of the 'commentlength' field. + * @return The value of the 'commentlength' field. + */ + public java.lang.Long getCommentlength() { + return commentlength; + } + + /** + * Sets the value of the 'commentlength' field. + * @param value the value to set. + */ + public void setCommentlength(java.lang.Long value) { + this.commentlength = value; + } + + /** + * Gets the value of the 'isnew' field. + * @return The value of the 'isnew' field. + */ + public java.lang.Boolean getIsnew() { + return isnew; + } + + /** + * Sets the value of the 'isnew' field. + * @param value the value to set. + */ + public void setIsnew(java.lang.Boolean value) { + this.isnew = value; + } + + /** + * Gets the value of the 'isminor' field. + * @return The value of the 'isminor' field. + */ + public java.lang.Boolean getIsminor() { + return isminor; + } + + /** + * Sets the value of the 'isminor' field. + * @param value the value to set. + */ + public void setIsminor(java.lang.Boolean value) { + this.isminor = value; + } + + /** + * Gets the value of the 'delta' field. + * @return The value of the 'delta' field. + */ + public java.lang.Long getDelta() { + return delta; + } + + /** + * Sets the value of the 'delta' field. + * @param value the value to set. + */ + public void setDelta(java.lang.Long value) { + this.delta = value; + } + + /** + * Gets the value of the 'isanonymous' field. + * @return The value of the 'isanonymous' field. + */ + public java.lang.Boolean getIsanonymous() { + return isanonymous; + } + + /** + * Sets the value of the 'isanonymous' field. + * @param value the value to set. + */ + public void setIsanonymous(java.lang.Boolean value) { + this.isanonymous = value; + } + + /** + * Gets the value of the 'user' field. + * @return The value of the 'user' field. + */ + public java.lang.String getUser() { + return user; + } + + /** + * Sets the value of the 'user' field. + * @param value the value to set. + */ + public void setUser(java.lang.String value) { + this.user = value; + } + + /** + * Gets the value of the 'deltabucket' field. + * @return The value of the 'deltabucket' field. + */ + public java.lang.Double getDeltabucket() { + return deltabucket; + } + + /** + * Sets the value of the 'deltabucket' field. + * @param value the value to set. + */ + public void setDeltabucket(java.lang.Double value) { + this.deltabucket = value; + } + + /** + * Gets the value of the 'deleted' field. + * @return The value of the 'deleted' field. + */ + public java.lang.Long getDeleted() { + return deleted; + } + + /** + * Sets the value of the 'deleted' field. + * @param value the value to set. + */ + public void setDeleted(java.lang.Long value) { + this.deleted = value; + } + + /** + * Gets the value of the 'namespace' field. + * @return The value of the 'namespace' field. + */ + public java.lang.String getNamespace() { + return namespace; + } + + /** + * Sets the value of the 'namespace' field. + * @param value the value to set. + */ + public void setNamespace(java.lang.String value) { + this.namespace = value; + } + + /** + * Creates a new Wikipedia RecordBuilder. + * @return A new Wikipedia RecordBuilder + */ + public static org.apache.hive.kafka.Wikipedia.Builder newBuilder() { + return new org.apache.hive.kafka.Wikipedia.Builder(); + } + + /** + * Creates a new Wikipedia RecordBuilder by copying an existing Builder. + * @param other The existing builder to copy. + * @return A new Wikipedia RecordBuilder + */ + public static org.apache.hive.kafka.Wikipedia.Builder newBuilder(org.apache.hive.kafka.Wikipedia.Builder other) { + return new org.apache.hive.kafka.Wikipedia.Builder(other); + } + + /** + * Creates a new Wikipedia RecordBuilder by copying an existing Wikipedia instance. + * @param other The existing instance to copy. + * @return A new Wikipedia RecordBuilder + */ + public static org.apache.hive.kafka.Wikipedia.Builder newBuilder(org.apache.hive.kafka.Wikipedia other) { + return new org.apache.hive.kafka.Wikipedia.Builder(other); + } + + /** + * RecordBuilder for Wikipedia instances. + */ + public static class Builder extends org.apache.avro.specific.SpecificRecordBuilderBase + implements org.apache.avro.data.RecordBuilder { + + private boolean isrobot; + private java.lang.String channel; + private java.lang.String timestamp; + private java.lang.String flags; + private boolean isunpatrolled; + private java.lang.String page; + private java.lang.String diffurl; + private long added; + private java.lang.String comment; + private long commentlength; + private boolean isnew; + private boolean isminor; + private long delta; + private boolean isanonymous; + private java.lang.String user; + private double deltabucket; + private long deleted; + private java.lang.String namespace; + + /** Creates a new Builder */ + private Builder() { + super(SCHEMA$); + } + + /** + * Creates a Builder by copying an existing Builder. + * @param other The existing Builder to copy. + */ + private Builder(org.apache.hive.kafka.Wikipedia.Builder other) { + super(other); + if (isValidValue(fields()[0], other.isrobot)) { + this.isrobot = data().deepCopy(fields()[0].schema(), other.isrobot); + fieldSetFlags()[0] = true; + } + if (isValidValue(fields()[1], other.channel)) { + this.channel = data().deepCopy(fields()[1].schema(), other.channel); + fieldSetFlags()[1] = true; + } + if (isValidValue(fields()[2], other.timestamp)) { + this.timestamp = data().deepCopy(fields()[2].schema(), other.timestamp); + fieldSetFlags()[2] = true; + } + if (isValidValue(fields()[3], other.flags)) { + this.flags = data().deepCopy(fields()[3].schema(), other.flags); + fieldSetFlags()[3] = true; + } + if (isValidValue(fields()[4], other.isunpatrolled)) { + this.isunpatrolled = data().deepCopy(fields()[4].schema(), other.isunpatrolled); + fieldSetFlags()[4] = true; + } + if (isValidValue(fields()[5], other.page)) { + this.page = data().deepCopy(fields()[5].schema(), other.page); + fieldSetFlags()[5] = true; + } + if (isValidValue(fields()[6], other.diffurl)) { + this.diffurl = data().deepCopy(fields()[6].schema(), other.diffurl); + fieldSetFlags()[6] = true; + } + if (isValidValue(fields()[7], other.added)) { + this.added = data().deepCopy(fields()[7].schema(), other.added); + fieldSetFlags()[7] = true; + } + if (isValidValue(fields()[8], other.comment)) { + this.comment = data().deepCopy(fields()[8].schema(), other.comment); + fieldSetFlags()[8] = true; + } + if (isValidValue(fields()[9], other.commentlength)) { + this.commentlength = data().deepCopy(fields()[9].schema(), other.commentlength); + fieldSetFlags()[9] = true; + } + if (isValidValue(fields()[10], other.isnew)) { + this.isnew = data().deepCopy(fields()[10].schema(), other.isnew); + fieldSetFlags()[10] = true; + } + if (isValidValue(fields()[11], other.isminor)) { + this.isminor = data().deepCopy(fields()[11].schema(), other.isminor); + fieldSetFlags()[11] = true; + } + if (isValidValue(fields()[12], other.delta)) { + this.delta = data().deepCopy(fields()[12].schema(), other.delta); + fieldSetFlags()[12] = true; + } + if (isValidValue(fields()[13], other.isanonymous)) { + this.isanonymous = data().deepCopy(fields()[13].schema(), other.isanonymous); + fieldSetFlags()[13] = true; + } + if (isValidValue(fields()[14], other.user)) { + this.user = data().deepCopy(fields()[14].schema(), other.user); + fieldSetFlags()[14] = true; + } + if (isValidValue(fields()[15], other.deltabucket)) { + this.deltabucket = data().deepCopy(fields()[15].schema(), other.deltabucket); + fieldSetFlags()[15] = true; + } + if (isValidValue(fields()[16], other.deleted)) { + this.deleted = data().deepCopy(fields()[16].schema(), other.deleted); + fieldSetFlags()[16] = true; + } + if (isValidValue(fields()[17], other.namespace)) { + this.namespace = data().deepCopy(fields()[17].schema(), other.namespace); + fieldSetFlags()[17] = true; + } + } + + /** + * Creates a Builder by copying an existing Wikipedia instance + * @param other The existing instance to copy. + */ + private Builder(org.apache.hive.kafka.Wikipedia other) { + super(SCHEMA$); + if (isValidValue(fields()[0], other.isrobot)) { + this.isrobot = data().deepCopy(fields()[0].schema(), other.isrobot); + fieldSetFlags()[0] = true; + } + if (isValidValue(fields()[1], other.channel)) { + this.channel = data().deepCopy(fields()[1].schema(), other.channel); + fieldSetFlags()[1] = true; + } + if (isValidValue(fields()[2], other.timestamp)) { + this.timestamp = data().deepCopy(fields()[2].schema(), other.timestamp); + fieldSetFlags()[2] = true; + } + if (isValidValue(fields()[3], other.flags)) { + this.flags = data().deepCopy(fields()[3].schema(), other.flags); + fieldSetFlags()[3] = true; + } + if (isValidValue(fields()[4], other.isunpatrolled)) { + this.isunpatrolled = data().deepCopy(fields()[4].schema(), other.isunpatrolled); + fieldSetFlags()[4] = true; + } + if (isValidValue(fields()[5], other.page)) { + this.page = data().deepCopy(fields()[5].schema(), other.page); + fieldSetFlags()[5] = true; + } + if (isValidValue(fields()[6], other.diffurl)) { + this.diffurl = data().deepCopy(fields()[6].schema(), other.diffurl); + fieldSetFlags()[6] = true; + } + if (isValidValue(fields()[7], other.added)) { + this.added = data().deepCopy(fields()[7].schema(), other.added); + fieldSetFlags()[7] = true; + } + if (isValidValue(fields()[8], other.comment)) { + this.comment = data().deepCopy(fields()[8].schema(), other.comment); + fieldSetFlags()[8] = true; + } + if (isValidValue(fields()[9], other.commentlength)) { + this.commentlength = data().deepCopy(fields()[9].schema(), other.commentlength); + fieldSetFlags()[9] = true; + } + if (isValidValue(fields()[10], other.isnew)) { + this.isnew = data().deepCopy(fields()[10].schema(), other.isnew); + fieldSetFlags()[10] = true; + } + if (isValidValue(fields()[11], other.isminor)) { + this.isminor = data().deepCopy(fields()[11].schema(), other.isminor); + fieldSetFlags()[11] = true; + } + if (isValidValue(fields()[12], other.delta)) { + this.delta = data().deepCopy(fields()[12].schema(), other.delta); + fieldSetFlags()[12] = true; + } + if (isValidValue(fields()[13], other.isanonymous)) { + this.isanonymous = data().deepCopy(fields()[13].schema(), other.isanonymous); + fieldSetFlags()[13] = true; + } + if (isValidValue(fields()[14], other.user)) { + this.user = data().deepCopy(fields()[14].schema(), other.user); + fieldSetFlags()[14] = true; + } + if (isValidValue(fields()[15], other.deltabucket)) { + this.deltabucket = data().deepCopy(fields()[15].schema(), other.deltabucket); + fieldSetFlags()[15] = true; + } + if (isValidValue(fields()[16], other.deleted)) { + this.deleted = data().deepCopy(fields()[16].schema(), other.deleted); + fieldSetFlags()[16] = true; + } + if (isValidValue(fields()[17], other.namespace)) { + this.namespace = data().deepCopy(fields()[17].schema(), other.namespace); + fieldSetFlags()[17] = true; + } + } + + /** + * Gets the value of the 'isrobot' field. + * @return The value. + */ + public java.lang.Boolean getIsrobot() { + return isrobot; + } + + /** + * Sets the value of the 'isrobot' field. + * @param value The value of 'isrobot'. + * @return This builder. + */ + public org.apache.hive.kafka.Wikipedia.Builder setIsrobot(boolean value) { + validate(fields()[0], value); + this.isrobot = value; + fieldSetFlags()[0] = true; + return this; + } + + /** + * Checks whether the 'isrobot' field has been set. + * @return True if the 'isrobot' field has been set, false otherwise. + */ + public boolean hasIsrobot() { + return fieldSetFlags()[0]; + } + + + /** + * Clears the value of the 'isrobot' field. + * @return This builder. + */ + public org.apache.hive.kafka.Wikipedia.Builder clearIsrobot() { + fieldSetFlags()[0] = false; + return this; + } + + /** + * Gets the value of the 'channel' field. + * @return The value. + */ + public java.lang.String getChannel() { + return channel; + } + + /** + * Sets the value of the 'channel' field. + * @param value The value of 'channel'. + * @return This builder. + */ + public org.apache.hive.kafka.Wikipedia.Builder setChannel(java.lang.String value) { + validate(fields()[1], value); + this.channel = value; + fieldSetFlags()[1] = true; + return this; + } + + /** + * Checks whether the 'channel' field has been set. + * @return True if the 'channel' field has been set, false otherwise. + */ + public boolean hasChannel() { + return fieldSetFlags()[1]; + } + + + /** + * Clears the value of the 'channel' field. + * @return This builder. + */ + public org.apache.hive.kafka.Wikipedia.Builder clearChannel() { + channel = null; + fieldSetFlags()[1] = false; + return this; + } + + /** + * Gets the value of the 'timestamp' field. + * @return The value. + */ + public java.lang.String getTimestamp() { + return timestamp; + } + + /** + * Sets the value of the 'timestamp' field. + * @param value The value of 'timestamp'. + * @return This builder. + */ + public org.apache.hive.kafka.Wikipedia.Builder setTimestamp(java.lang.String value) { + validate(fields()[2], value); + this.timestamp = value; + fieldSetFlags()[2] = true; + return this; + } + + /** + * Checks whether the 'timestamp' field has been set. + * @return True if the 'timestamp' field has been set, false otherwise. + */ + public boolean hasTimestamp() { + return fieldSetFlags()[2]; + } + + + /** + * Clears the value of the 'timestamp' field. + * @return This builder. + */ + public org.apache.hive.kafka.Wikipedia.Builder clearTimestamp() { + timestamp = null; + fieldSetFlags()[2] = false; + return this; + } + + /** + * Gets the value of the 'flags' field. + * @return The value. + */ + public java.lang.String getFlags() { + return flags; + } + + /** + * Sets the value of the 'flags' field. + * @param value The value of 'flags'. + * @return This builder. + */ + public org.apache.hive.kafka.Wikipedia.Builder setFlags(java.lang.String value) { + validate(fields()[3], value); + this.flags = value; + fieldSetFlags()[3] = true; + return this; + } + + /** + * Checks whether the 'flags' field has been set. + * @return True if the 'flags' field has been set, false otherwise. + */ + public boolean hasFlags() { + return fieldSetFlags()[3]; + } + + + /** + * Clears the value of the 'flags' field. + * @return This builder. + */ + public org.apache.hive.kafka.Wikipedia.Builder clearFlags() { + flags = null; + fieldSetFlags()[3] = false; + return this; + } + + /** + * Gets the value of the 'isunpatrolled' field. + * @return The value. + */ + public java.lang.Boolean getIsunpatrolled() { + return isunpatrolled; + } + + /** + * Sets the value of the 'isunpatrolled' field. + * @param value The value of 'isunpatrolled'. + * @return This builder. + */ + public org.apache.hive.kafka.Wikipedia.Builder setIsunpatrolled(boolean value) { + validate(fields()[4], value); + this.isunpatrolled = value; + fieldSetFlags()[4] = true; + return this; + } + + /** + * Checks whether the 'isunpatrolled' field has been set. + * @return True if the 'isunpatrolled' field has been set, false otherwise. + */ + public boolean hasIsunpatrolled() { + return fieldSetFlags()[4]; + } + + + /** + * Clears the value of the 'isunpatrolled' field. + * @return This builder. + */ + public org.apache.hive.kafka.Wikipedia.Builder clearIsunpatrolled() { + fieldSetFlags()[4] = false; + return this; + } + + /** + * Gets the value of the 'page' field. + * @return The value. + */ + public java.lang.String getPage() { + return page; + } + + /** + * Sets the value of the 'page' field. + * @param value The value of 'page'. + * @return This builder. + */ + public org.apache.hive.kafka.Wikipedia.Builder setPage(java.lang.String value) { + validate(fields()[5], value); + this.page = value; + fieldSetFlags()[5] = true; + return this; + } + + /** + * Checks whether the 'page' field has been set. + * @return True if the 'page' field has been set, false otherwise. + */ + public boolean hasPage() { + return fieldSetFlags()[5]; + } + + + /** + * Clears the value of the 'page' field. + * @return This builder. + */ + public org.apache.hive.kafka.Wikipedia.Builder clearPage() { + page = null; + fieldSetFlags()[5] = false; + return this; + } + + /** + * Gets the value of the 'diffurl' field. + * @return The value. + */ + public java.lang.String getDiffurl() { + return diffurl; + } + + /** + * Sets the value of the 'diffurl' field. + * @param value The value of 'diffurl'. + * @return This builder. + */ + public org.apache.hive.kafka.Wikipedia.Builder setDiffurl(java.lang.String value) { + validate(fields()[6], value); + this.diffurl = value; + fieldSetFlags()[6] = true; + return this; + } + + /** + * Checks whether the 'diffurl' field has been set. + * @return True if the 'diffurl' field has been set, false otherwise. + */ + public boolean hasDiffurl() { + return fieldSetFlags()[6]; + } + + + /** + * Clears the value of the 'diffurl' field. + * @return This builder. + */ + public org.apache.hive.kafka.Wikipedia.Builder clearDiffurl() { + diffurl = null; + fieldSetFlags()[6] = false; + return this; + } + + /** + * Gets the value of the 'added' field. + * @return The value. + */ + public java.lang.Long getAdded() { + return added; + } + + /** + * Sets the value of the 'added' field. + * @param value The value of 'added'. + * @return This builder. + */ + public org.apache.hive.kafka.Wikipedia.Builder setAdded(long value) { + validate(fields()[7], value); + this.added = value; + fieldSetFlags()[7] = true; + return this; + } + + /** + * Checks whether the 'added' field has been set. + * @return True if the 'added' field has been set, false otherwise. + */ + public boolean hasAdded() { + return fieldSetFlags()[7]; + } + + + /** + * Clears the value of the 'added' field. + * @return This builder. + */ + public org.apache.hive.kafka.Wikipedia.Builder clearAdded() { + fieldSetFlags()[7] = false; + return this; + } + + /** + * Gets the value of the 'comment' field. + * @return The value. + */ + public java.lang.String getComment() { + return comment; + } + + /** + * Sets the value of the 'comment' field. + * @param value The value of 'comment'. + * @return This builder. + */ + public org.apache.hive.kafka.Wikipedia.Builder setComment(java.lang.String value) { + validate(fields()[8], value); + this.comment = value; + fieldSetFlags()[8] = true; + return this; + } + + /** + * Checks whether the 'comment' field has been set. + * @return True if the 'comment' field has been set, false otherwise. + */ + public boolean hasComment() { + return fieldSetFlags()[8]; + } + + + /** + * Clears the value of the 'comment' field. + * @return This builder. + */ + public org.apache.hive.kafka.Wikipedia.Builder clearComment() { + comment = null; + fieldSetFlags()[8] = false; + return this; + } + + /** + * Gets the value of the 'commentlength' field. + * @return The value. + */ + public java.lang.Long getCommentlength() { + return commentlength; + } + + /** + * Sets the value of the 'commentlength' field. + * @param value The value of 'commentlength'. + * @return This builder. + */ + public org.apache.hive.kafka.Wikipedia.Builder setCommentlength(long value) { + validate(fields()[9], value); + this.commentlength = value; + fieldSetFlags()[9] = true; + return this; + } + + /** + * Checks whether the 'commentlength' field has been set. + * @return True if the 'commentlength' field has been set, false otherwise. + */ + public boolean hasCommentlength() { + return fieldSetFlags()[9]; + } + + + /** + * Clears the value of the 'commentlength' field. + * @return This builder. + */ + public org.apache.hive.kafka.Wikipedia.Builder clearCommentlength() { + fieldSetFlags()[9] = false; + return this; + } + + /** + * Gets the value of the 'isnew' field. + * @return The value. + */ + public java.lang.Boolean getIsnew() { + return isnew; + } + + /** + * Sets the value of the 'isnew' field. + * @param value The value of 'isnew'. + * @return This builder. + */ + public org.apache.hive.kafka.Wikipedia.Builder setIsnew(boolean value) { + validate(fields()[10], value); + this.isnew = value; + fieldSetFlags()[10] = true; + return this; + } + + /** + * Checks whether the 'isnew' field has been set. + * @return True if the 'isnew' field has been set, false otherwise. + */ + public boolean hasIsnew() { + return fieldSetFlags()[10]; + } + + + /** + * Clears the value of the 'isnew' field. + * @return This builder. + */ + public org.apache.hive.kafka.Wikipedia.Builder clearIsnew() { + fieldSetFlags()[10] = false; + return this; + } + + /** + * Gets the value of the 'isminor' field. + * @return The value. + */ + public java.lang.Boolean getIsminor() { + return isminor; + } + + /** + * Sets the value of the 'isminor' field. + * @param value The value of 'isminor'. + * @return This builder. + */ + public org.apache.hive.kafka.Wikipedia.Builder setIsminor(boolean value) { + validate(fields()[11], value); + this.isminor = value; + fieldSetFlags()[11] = true; + return this; + } + + /** + * Checks whether the 'isminor' field has been set. + * @return True if the 'isminor' field has been set, false otherwise. + */ + public boolean hasIsminor() { + return fieldSetFlags()[11]; + } + + + /** + * Clears the value of the 'isminor' field. + * @return This builder. + */ + public org.apache.hive.kafka.Wikipedia.Builder clearIsminor() { + fieldSetFlags()[11] = false; + return this; + } + + /** + * Gets the value of the 'delta' field. + * @return The value. + */ + public java.lang.Long getDelta() { + return delta; + } + + /** + * Sets the value of the 'delta' field. + * @param value The value of 'delta'. + * @return This builder. + */ + public org.apache.hive.kafka.Wikipedia.Builder setDelta(long value) { + validate(fields()[12], value); + this.delta = value; + fieldSetFlags()[12] = true; + return this; + } + + /** + * Checks whether the 'delta' field has been set. + * @return True if the 'delta' field has been set, false otherwise. + */ + public boolean hasDelta() { + return fieldSetFlags()[12]; + } + + + /** + * Clears the value of the 'delta' field. + * @return This builder. + */ + public org.apache.hive.kafka.Wikipedia.Builder clearDelta() { + fieldSetFlags()[12] = false; + return this; + } + + /** + * Gets the value of the 'isanonymous' field. + * @return The value. + */ + public java.lang.Boolean getIsanonymous() { + return isanonymous; + } + + /** + * Sets the value of the 'isanonymous' field. + * @param value The value of 'isanonymous'. + * @return This builder. + */ + public org.apache.hive.kafka.Wikipedia.Builder setIsanonymous(boolean value) { + validate(fields()[13], value); + this.isanonymous = value; + fieldSetFlags()[13] = true; + return this; + } + + /** + * Checks whether the 'isanonymous' field has been set. + * @return True if the 'isanonymous' field has been set, false otherwise. + */ + public boolean hasIsanonymous() { + return fieldSetFlags()[13]; + } + + + /** + * Clears the value of the 'isanonymous' field. + * @return This builder. + */ + public org.apache.hive.kafka.Wikipedia.Builder clearIsanonymous() { + fieldSetFlags()[13] = false; + return this; + } + + /** + * Gets the value of the 'user' field. + * @return The value. + */ + public java.lang.String getUser() { + return user; + } + + /** + * Sets the value of the 'user' field. + * @param value The value of 'user'. + * @return This builder. + */ + public org.apache.hive.kafka.Wikipedia.Builder setUser(java.lang.String value) { + validate(fields()[14], value); + this.user = value; + fieldSetFlags()[14] = true; + return this; + } + + /** + * Checks whether the 'user' field has been set. + * @return True if the 'user' field has been set, false otherwise. + */ + public boolean hasUser() { + return fieldSetFlags()[14]; + } + + + /** + * Clears the value of the 'user' field. + * @return This builder. + */ + public org.apache.hive.kafka.Wikipedia.Builder clearUser() { + user = null; + fieldSetFlags()[14] = false; + return this; + } + + /** + * Gets the value of the 'deltabucket' field. + * @return The value. + */ + public java.lang.Double getDeltabucket() { + return deltabucket; + } + + /** + * Sets the value of the 'deltabucket' field. + * @param value The value of 'deltabucket'. + * @return This builder. + */ + public org.apache.hive.kafka.Wikipedia.Builder setDeltabucket(double value) { + validate(fields()[15], value); + this.deltabucket = value; + fieldSetFlags()[15] = true; + return this; + } + + /** + * Checks whether the 'deltabucket' field has been set. + * @return True if the 'deltabucket' field has been set, false otherwise. + */ + public boolean hasDeltabucket() { + return fieldSetFlags()[15]; + } + + + /** + * Clears the value of the 'deltabucket' field. + * @return This builder. + */ + public org.apache.hive.kafka.Wikipedia.Builder clearDeltabucket() { + fieldSetFlags()[15] = false; + return this; + } + + /** + * Gets the value of the 'deleted' field. + * @return The value. + */ + public java.lang.Long getDeleted() { + return deleted; + } + + /** + * Sets the value of the 'deleted' field. + * @param value The value of 'deleted'. + * @return This builder. + */ + public org.apache.hive.kafka.Wikipedia.Builder setDeleted(long value) { + validate(fields()[16], value); + this.deleted = value; + fieldSetFlags()[16] = true; + return this; + } + + /** + * Checks whether the 'deleted' field has been set. + * @return True if the 'deleted' field has been set, false otherwise. + */ + public boolean hasDeleted() { + return fieldSetFlags()[16]; + } + + + /** + * Clears the value of the 'deleted' field. + * @return This builder. + */ + public org.apache.hive.kafka.Wikipedia.Builder clearDeleted() { + fieldSetFlags()[16] = false; + return this; + } + + /** + * Gets the value of the 'namespace' field. + * @return The value. + */ + public java.lang.String getNamespace() { + return namespace; + } + + /** + * Sets the value of the 'namespace' field. + * @param value The value of 'namespace'. + * @return This builder. + */ + public org.apache.hive.kafka.Wikipedia.Builder setNamespace(java.lang.String value) { + validate(fields()[17], value); + this.namespace = value; + fieldSetFlags()[17] = true; + return this; + } + + /** + * Checks whether the 'namespace' field has been set. + * @return True if the 'namespace' field has been set, false otherwise. + */ + public boolean hasNamespace() { + return fieldSetFlags()[17]; + } + + + /** + * Clears the value of the 'namespace' field. + * @return This builder. + */ + public org.apache.hive.kafka.Wikipedia.Builder clearNamespace() { + namespace = null; + fieldSetFlags()[17] = false; + return this; + } + + @Override + @SuppressWarnings("unchecked") + public Wikipedia build() { + try { + Wikipedia record = new Wikipedia(); + record.isrobot = fieldSetFlags()[0] ? this.isrobot : (java.lang.Boolean) defaultValue(fields()[0]); + record.channel = fieldSetFlags()[1] ? this.channel : (java.lang.String) defaultValue(fields()[1]); + record.timestamp = fieldSetFlags()[2] ? this.timestamp : (java.lang.String) defaultValue(fields()[2]); + record.flags = fieldSetFlags()[3] ? this.flags : (java.lang.String) defaultValue(fields()[3]); + record.isunpatrolled = fieldSetFlags()[4] ? this.isunpatrolled : (java.lang.Boolean) defaultValue(fields()[4]); + record.page = fieldSetFlags()[5] ? this.page : (java.lang.String) defaultValue(fields()[5]); + record.diffurl = fieldSetFlags()[6] ? this.diffurl : (java.lang.String) defaultValue(fields()[6]); + record.added = fieldSetFlags()[7] ? this.added : (java.lang.Long) defaultValue(fields()[7]); + record.comment = fieldSetFlags()[8] ? this.comment : (java.lang.String) defaultValue(fields()[8]); + record.commentlength = fieldSetFlags()[9] ? this.commentlength : (java.lang.Long) defaultValue(fields()[9]); + record.isnew = fieldSetFlags()[10] ? this.isnew : (java.lang.Boolean) defaultValue(fields()[10]); + record.isminor = fieldSetFlags()[11] ? this.isminor : (java.lang.Boolean) defaultValue(fields()[11]); + record.delta = fieldSetFlags()[12] ? this.delta : (java.lang.Long) defaultValue(fields()[12]); + record.isanonymous = fieldSetFlags()[13] ? this.isanonymous : (java.lang.Boolean) defaultValue(fields()[13]); + record.user = fieldSetFlags()[14] ? this.user : (java.lang.String) defaultValue(fields()[14]); + record.deltabucket = fieldSetFlags()[15] ? this.deltabucket : (java.lang.Double) defaultValue(fields()[15]); + record.deleted = fieldSetFlags()[16] ? this.deleted : (java.lang.Long) defaultValue(fields()[16]); + record.namespace = fieldSetFlags()[17] ? this.namespace : (java.lang.String) defaultValue(fields()[17]); + return record; + } catch (java.lang.Exception e) { + throw new org.apache.avro.AvroRuntimeException(e); + } + } + } + + @SuppressWarnings("unchecked") + private static final org.apache.avro.io.DatumWriter + WRITER$ = (org.apache.avro.io.DatumWriter)MODEL$.createDatumWriter(SCHEMA$); + + @Override public void writeExternal(java.io.ObjectOutput out) + throws java.io.IOException { + WRITER$.write(this, SpecificData.getEncoder(out)); + } + + @SuppressWarnings("unchecked") + private static final org.apache.avro.io.DatumReader + READER$ = (org.apache.avro.io.DatumReader)MODEL$.createDatumReader(SCHEMA$); + + @Override public void readExternal(java.io.ObjectInput in) + throws java.io.IOException { + READER$.read(this, SpecificData.getDecoder(in)); + } + +} diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java b/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java index 5adbb63693b1..8f9e4eccb853 100644 --- a/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java +++ b/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java @@ -39,6 +39,8 @@ import java.nio.file.Files; import java.nio.file.Paths; import java.sql.SQLException; +import java.sql.Timestamp; +import java.text.SimpleDateFormat; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -53,7 +55,14 @@ import java.util.concurrent.TimeUnit; import java.util.regex.Matcher; import java.util.regex.Pattern; - +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.io.BinaryEncoder; +import org.apache.avro.io.DatumWriter; +import org.apache.avro.io.EncoderFactory; +import org.apache.avro.specific.SpecificDatumWriter; import org.apache.commons.io.IOUtils; import org.apache.commons.io.output.ByteArrayOutputStream; import org.apache.commons.lang.StringUtils; @@ -112,6 +121,7 @@ import org.apache.hive.common.util.StreamPrinter; import org.apache.hive.druid.MiniDruidCluster; import org.apache.hive.kafka.SingleNodeKafkaCluster; +import org.apache.hive.kafka.Wikipedia; import org.apache.logging.log4j.util.Strings; import org.apache.tools.ant.BuildException; import org.apache.zookeeper.WatchedEvent; @@ -143,6 +153,7 @@ public class QTestUtil { static final Logger LOG = LoggerFactory.getLogger("QTestUtil"); private final static String defaultInitScript = "q_test_init.sql"; private final static String defaultCleanupScript = "q_test_cleanup.sql"; + private static SimpleDateFormat formatter = new SimpleDateFormat("MM/dd/yyyy HH:mm:ss"); private final String[] testOnlyCommands = new String[]{"crypto", "erasure"}; public static final String TEST_TMP_DIR_PROPERTY = "test.tmp.dir"; // typically target/tmp @@ -671,6 +682,7 @@ private void setupMiniCluster(HadoopShims shims, String confDir) throws "test-topic", new File(getScriptsDir(), "kafka_init_data.json") ); + kafkaCluster.createTopicWithData("wiki_kafka_avro_table", getAvroRows()); } if (clusterType.getCoreClusterType() == CoreClusterType.TEZ) { @@ -709,6 +721,48 @@ private void setupMiniCluster(HadoopShims shims, String confDir) throws } } + private static List getAvroRows() { + int numRows = 10; + List events; + final DatumWriter writer = new SpecificDatumWriter<>(Wikipedia.getClassSchema()); + events = + IntStream.rangeClosed(0, numRows) + .mapToObj(i -> Wikipedia.newBuilder() + // 1534736225090 -> 08/19/2018 20:37:05 + .setTimestamp(formatter.format(new Timestamp(1534736225090L + 1000 * 3600 * i))) + .setAdded(i * 300) + .setDeleted(-i) + .setIsrobot(i % 2 == 0) + .setChannel("chanel number " + i) + .setComment("comment number " + i) + .setCommentlength(i) + .setDiffurl(String.format("url %s", i)) + .setFlags("flag") + .setIsminor(i % 2 > 0) + .setIsanonymous(i % 3 != 0) + .setNamespace("namespace") + .setIsunpatrolled(new Boolean(i % 3 == 0)) + .setIsnew(new Boolean(i % 2 > 0)) + .setPage(String.format("page is %s", i * 100)) + .setDelta(i) + .setDeltabucket(i * 100.4) + .setUser("test-user-" + i) + .build()) + .map(genericRecord -> { + java.io.ByteArrayOutputStream out = new java.io.ByteArrayOutputStream(); + BinaryEncoder encoder = EncoderFactory.get().binaryEncoder(out, null); + try { + writer.write(genericRecord, encoder); + encoder.flush(); + out.close(); + } catch (IOException e) { + throw new RuntimeException(e); + } + return out.toByteArray(); + }) + .collect(Collectors.toList()); + return events; + } public void shutdown() throws Exception { if (System.getenv(QTEST_LEAVE_FILES) == null) { diff --git a/ql/src/test/queries/clientpositive/kafka_storage_handler.q b/ql/src/test/queries/clientpositive/kafka_storage_handler.q index 9dc5680e375c..1a9edb53c4bd 100644 --- a/ql/src/test/queries/clientpositive/kafka_storage_handler.q +++ b/ql/src/test/queries/clientpositive/kafka_storage_handler.q @@ -142,4 +142,87 @@ Select `__partition`, `__offset`, `__time`, `page`, `user`, `language`, `country `unpatrolled` , `anonymous` , `robot` , added , deleted , delta FROM kafka_table_2; -Select count(*) FROM kafka_table_2; \ No newline at end of file +Select count(*) FROM kafka_table_2; + + +CREATE EXTERNAL TABLE wiki_kafka_avro_table +STORED BY 'org.apache.hadoop.hive.kafka.KafkaStorageHandler' +TBLPROPERTIES +("kafka.topic" = "wiki_kafka_avro_table", +"kafka.bootstrap.servers"="localhost:9092", +"kafka.serde.class"="org.apache.hadoop.hive.serde2.avro.AvroSerDe", +'avro.schema.literal'='{ + "type" : "record", + "name" : "Wikipedia", + "namespace" : "org.apache.hive.kafka", + "version": "1", + "fields" : [ { + "name" : "isrobot", + "type" : "boolean" + }, { + "name" : "channel", + "type" : "string" + }, { + "name" : "timestamp", + "type" : "string" + }, { + "name" : "flags", + "type" : "string" + }, { + "name" : "isunpatrolled", + "type" : "boolean" + }, { + "name" : "page", + "type" : "string" + }, { + "name" : "diffurl", + "type" : "string" + }, { + "name" : "added", + "type" : "long" + }, { + "name" : "comment", + "type" : "string" + }, { + "name" : "commentlength", + "type" : "long" + }, { + "name" : "isnew", + "type" : "boolean" + }, { + "name" : "isminor", + "type" : "boolean" + }, { + "name" : "delta", + "type" : "long" + }, { + "name" : "isanonymous", + "type" : "boolean" + }, { + "name" : "user", + "type" : "string" + }, { + "name" : "deltabucket", + "type" : "double" + }, { + "name" : "deleted", + "type" : "long" + }, { + "name" : "namespace", + "type" : "string" + } ] +}' +); + +describe extended 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 count(*) from wiki_kafka_avro_table; + +select count(distinct `user`) from wiki_kafka_avro_table; + +select sum(deltabucket), min(commentlength) from wiki_kafka_avro_table; + +select cast ((`__timestamp`/1000) as timestamp) as kafka_record_ts, `__timestamp` as kafka_record_ts_long, `__partition`, `__offset`, `timestamp`, `user`, `page`, `deleted`, `deltabucket`, `isanonymous`, `commentlength` from wiki_kafka_avro_table where `__timestamp` > 1534750625090; + diff --git a/ql/src/test/results/clientpositive/druid/kafka_storage_handler.q.out b/ql/src/test/results/clientpositive/druid/kafka_storage_handler.q.out index ec7ca062e614..0d788315493b 100644 --- a/ql/src/test/results/clientpositive/druid/kafka_storage_handler.q.out +++ b/ql/src/test/results/clientpositive/druid/kafka_storage_handler.q.out @@ -572,3 +572,234 @@ POSTHOOK: type: QUERY POSTHOOK: Input: default@kafka_table_2 POSTHOOK: Output: hdfs://### HDFS PATH ### 10 +PREHOOK: query: CREATE EXTERNAL TABLE wiki_kafka_avro_table +STORED BY 'org.apache.hadoop.hive.kafka.KafkaStorageHandler' +TBLPROPERTIES +("kafka.topic" = "wiki_kafka_avro_table", +"kafka.bootstrap.servers"="localhost:9092", +"kafka.serde.class"="org.apache.hadoop.hive.serde2.avro.AvroSerDe", +'avro.schema.literal'='{ + "type" : "record", + "name" : "Wikipedia", + "namespace" : "org.apache.hive.kafka", + "version": "1", + "fields" : [ { + "name" : "isrobot", + "type" : "boolean" + }, { + "name" : "channel", + "type" : "string" + }, { + "name" : "timestamp", + "type" : "string" + }, { + "name" : "flags", + "type" : "string" + }, { + "name" : "isunpatrolled", + "type" : "boolean" + }, { + "name" : "page", + "type" : "string" + }, { + "name" : "diffurl", + "type" : "string" + }, { + "name" : "added", + "type" : "long" + }, { + "name" : "comment", + "type" : "string" + }, { + "name" : "commentlength", + "type" : "long" + }, { + "name" : "isnew", + "type" : "boolean" + }, { + "name" : "isminor", + "type" : "boolean" + }, { + "name" : "delta", + "type" : "long" + }, { + "name" : "isanonymous", + "type" : "boolean" + }, { + "name" : "user", + "type" : "string" + }, { + "name" : "deltabucket", + "type" : "double" + }, { + "name" : "deleted", + "type" : "long" + }, { + "name" : "namespace", + "type" : "string" + } ] +}' +) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@wiki_kafka_avro_table +POSTHOOK: query: CREATE EXTERNAL TABLE wiki_kafka_avro_table +STORED BY 'org.apache.hadoop.hive.kafka.KafkaStorageHandler' +TBLPROPERTIES +("kafka.topic" = "wiki_kafka_avro_table", +"kafka.bootstrap.servers"="localhost:9092", +"kafka.serde.class"="org.apache.hadoop.hive.serde2.avro.AvroSerDe", +'avro.schema.literal'='{ + "type" : "record", + "name" : "Wikipedia", + "namespace" : "org.apache.hive.kafka", + "version": "1", + "fields" : [ { + "name" : "isrobot", + "type" : "boolean" + }, { + "name" : "channel", + "type" : "string" + }, { + "name" : "timestamp", + "type" : "string" + }, { + "name" : "flags", + "type" : "string" + }, { + "name" : "isunpatrolled", + "type" : "boolean" + }, { + "name" : "page", + "type" : "string" + }, { + "name" : "diffurl", + "type" : "string" + }, { + "name" : "added", + "type" : "long" + }, { + "name" : "comment", + "type" : "string" + }, { + "name" : "commentlength", + "type" : "long" + }, { + "name" : "isnew", + "type" : "boolean" + }, { + "name" : "isminor", + "type" : "boolean" + }, { + "name" : "delta", + "type" : "long" + }, { + "name" : "isanonymous", + "type" : "boolean" + }, { + "name" : "user", + "type" : "string" + }, { + "name" : "deltabucket", + "type" : "double" + }, { + "name" : "deleted", + "type" : "long" + }, { + "name" : "namespace", + "type" : "string" + } ] +}' +) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@wiki_kafka_avro_table +PREHOOK: query: describe extended wiki_kafka_avro_table +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@wiki_kafka_avro_table +POSTHOOK: query: describe extended wiki_kafka_avro_table +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@wiki_kafka_avro_table +isrobot boolean from deserializer +channel string from deserializer +timestamp string from deserializer +flags string from deserializer +isunpatrolled boolean from deserializer +page string from deserializer +diffurl string from deserializer +added bigint from deserializer +comment string from deserializer +commentlength bigint from deserializer +isnew boolean from deserializer +isminor boolean from deserializer +delta bigint from deserializer +isanonymous boolean from deserializer +user string from deserializer +deltabucket double from deserializer +deleted bigint from deserializer +namespace string from deserializer +__partition int from deserializer +__offset bigint from deserializer +__timestamp bigint from deserializer + +#### A masked pattern was here #### +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 +PREHOOK: query: select count(*) from wiki_kafka_avro_table +PREHOOK: type: QUERY +PREHOOK: Input: default@wiki_kafka_avro_table +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select count(*) from wiki_kafka_avro_table +POSTHOOK: type: QUERY +POSTHOOK: Input: default@wiki_kafka_avro_table +POSTHOOK: Output: hdfs://### HDFS PATH ### +11 +PREHOOK: query: select count(distinct `user`) from wiki_kafka_avro_table +PREHOOK: type: QUERY +PREHOOK: Input: default@wiki_kafka_avro_table +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select count(distinct `user`) from wiki_kafka_avro_table +POSTHOOK: type: QUERY +POSTHOOK: Input: default@wiki_kafka_avro_table +POSTHOOK: Output: hdfs://### HDFS PATH ### +11 +PREHOOK: query: select sum(deltabucket), min(commentlength) from wiki_kafka_avro_table +PREHOOK: type: QUERY +PREHOOK: Input: default@wiki_kafka_avro_table +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: select sum(deltabucket), min(commentlength) from wiki_kafka_avro_table +POSTHOOK: type: QUERY +POSTHOOK: Input: default@wiki_kafka_avro_table +POSTHOOK: Output: hdfs://### HDFS PATH ### +5522.000000000001 0 +PREHOOK: query: select cast ((`__timestamp`/1000) as timestamp) as kafka_record_ts, `__timestamp` as kafka_record_ts_long, `__partition`, `__offset`, `timestamp`, `user`, `page`, `deleted`, `deltabucket`, `isanonymous`, `commentlength` from wiki_kafka_avro_table where `__timestamp` > 1534750625090 +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, `__timestamp` as kafka_record_ts_long, `__partition`, `__offset`, `timestamp`, `user`, `page`, `deleted`, `deltabucket`, `isanonymous`, `commentlength` from wiki_kafka_avro_table where `__timestamp` > 1534750625090 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@wiki_kafka_avro_table +POSTHOOK: Output: hdfs://### HDFS PATH ### +2018-08-20 08:37:05.09 1534754225090 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 1534757825090 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 1534761425090 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 1534765025090 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 1534768625090 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 1534772225090 0 10 08/20/2018 06:37:05 test-user-10 page is 1000 -10 1004.0 true 10 diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/avro/AvroLazyObjectInspector.java b/serde/src/java/org/apache/hadoop/hive/serde2/avro/AvroLazyObjectInspector.java index e1e4d8bc5e61..cd37bcc7effc 100644 --- a/serde/src/java/org/apache/hadoop/hive/serde2/avro/AvroLazyObjectInspector.java +++ b/serde/src/java/org/apache/hadoop/hive/serde2/avro/AvroLazyObjectInspector.java @@ -386,7 +386,7 @@ private LazyObject toLazyPrimitiveObject(Object obj, * Convert the given object to a lazy object using the given {@link ObjectInspector} * * @param obj Object to be converted to a {@link LazyObject} - * @param oi ObjectInspector used for the conversion + * @param objectInspector ObjectInspector used for the conversion * @return the created {@link LazyObject lazy object} * */ private Object toLazyListObject(Object obj, ObjectInspector objectInspector) { @@ -414,7 +414,7 @@ private Object toLazyListObject(Object obj, ObjectInspector objectInspector) { * Convert the given object to a lazy object using the given {@link ObjectInspector} * * @param obj Object to be converted to a {@link LazyObject} - * @param oi ObjectInspector used for the conversion + * @param objectInspector ObjectInspector used for the conversion * @return the created {@link LazyObject lazy object} * */ @SuppressWarnings({ "rawtypes", "unchecked" }) @@ -451,7 +451,7 @@ private Object toLazyMapObject(Object obj, ObjectInspector objectInspector) { * Convert the given object to a lazy object using the given {@link ObjectInspector} * * @param obj Object to be converted to a {@link LazyObject} - * @param oi ObjectInspector used for the conversion + * @param objectInspector ObjectInspector used for the conversion * @return the created {@link LazyObject lazy object} * */ private Object toLazyUnionObject(Object obj, ObjectInspector objectInspector) { From 3f86ad79ad2b6cae9a4b87d094a497c0c6fc9fbd Mon Sep 17 00:00:00 2001 From: Slim Bouguerra Date: Mon, 20 Aug 2018 13:19:09 -0700 Subject: [PATCH 26/33] more docs reframe the TODOS Change-Id: Ib84d603da833da1c76fbd14522e5c070849c6751 --- .../org/apache/hadoop/hive/kafka/GenericKafkaSerDe.java | 2 +- .../java/org/apache/hadoop/hive/kafka/KafkaScanTrimmer.java | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/GenericKafkaSerDe.java b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/GenericKafkaSerDe.java index 8b42cb7558de..2c9732975e79 100644 --- a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/GenericKafkaSerDe.java +++ b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/GenericKafkaSerDe.java @@ -133,7 +133,7 @@ public class GenericKafkaSerDe extends AbstractSerDe { // switch case the serde nature final Object row; if (delegateSerDe instanceof JsonSerDe) { - // @TODO Text constructor copies the data, this op is not needed + //@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(); diff --git a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaScanTrimmer.java b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaScanTrimmer.java index 8904548e1867..55e00b5d13ec 100644 --- a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaScanTrimmer.java +++ b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaScanTrimmer.java @@ -328,7 +328,6 @@ public Map computeOptimizedScan(ExprNodeG if (startOffset != -1 && endOffset == -1) { newInputSplit = new KafkaPullerInputSplit(tp.topic(), tp.partition(), - // @TODO make sure that this is okay //if the user ask for start offset > max offset will replace with last offset Math.min(startOffset, existingInputSplit.getEndOffset()), existingInputSplit.getEndOffset(), @@ -336,12 +335,13 @@ public Map computeOptimizedScan(ExprNodeG } else if (endOffset != -1 && startOffset == -1) { newInputSplit = new KafkaPullerInputSplit(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(), - //@TODO check this with team if we have ask for offset out of range what to do ? - // here am seeking to last offset + // non existing offset will be seeking last offset existingInputSplit.getEndOffset(), existingInputSplit.getEndOffset(), existingInputSplit.getPath()); } else { newInputSplit = From 4cfd31f006657ea764d921bde1d5983d1adbf3bb Mon Sep 17 00:00:00 2001 From: Slim Bouguerra Date: Wed, 22 Aug 2018 15:28:53 -0700 Subject: [PATCH 27/33] refactor clean KafkaScanTrimmer.java Change-Id: I2d37ab7fb5ed51bb68a010f0fa41ea33ef9e5fbc --- .../hadoop/hive/kafka/KafkaScanTrimmer.java | 43 +++++++++---------- 1 file changed, 20 insertions(+), 23 deletions(-) diff --git a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaScanTrimmer.java b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaScanTrimmer.java index 55e00b5d13ec..76415151ec16 100644 --- a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaScanTrimmer.java +++ b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaScanTrimmer.java @@ -53,6 +53,7 @@ import javax.annotation.Nullable; import java.util.HashMap; import java.util.Map; +import java.util.Objects; import java.util.Set; import java.util.function.Predicate; @@ -60,7 +61,7 @@ * Kafka Range trimmer, takes a full kafka scan and prune the scan based on a filter expression * it is a Best effort trimmer and it can not replace the filter it self, filtration still takes place in Hive executor. */ -public class KafkaScanTrimmer { +class KafkaScanTrimmer { private static final Logger LOG = LoggerFactory.getLogger(KafkaScanTrimmer.class); private final Map fullHouse; private final KafkaConsumer kafkaConsumer; @@ -69,7 +70,7 @@ public class KafkaScanTrimmer { * @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 */ - public KafkaScanTrimmer(Map fullHouse, KafkaConsumer kafkaConsumer) { + KafkaScanTrimmer(Map fullHouse, KafkaConsumer kafkaConsumer) { this.fullHouse = fullHouse; this.kafkaConsumer = kafkaConsumer; } @@ -82,8 +83,8 @@ public KafkaScanTrimmer(Map fullHouse, Ka * * @return tiny house of of the full house based on filter expression */ - public Map computeOptimizedScan(ExprNodeGenericFuncDesc filterExpression) { - Map optimizedScan = parse(filterExpression); + Map computeOptimizedScan(ExprNodeGenericFuncDesc filterExpression) { + Map optimizedScan = parseAndOptimize(filterExpression); if (LOG.isDebugEnabled()) { if (optimizedScan != null) { @@ -108,11 +109,11 @@ public Map computeOptimizedScan(ExprNodeG } /** - * @param expression filter to parse and trim the full scan + * @param expression filter to parseAndOptimize and trim the full scan * * @return Map of optimized kafka range scans or null if it is impossible to optimize. */ - @Nullable private Map parse(ExprNodeDesc expression) { + @Nullable private Map parseAndOptimize(ExprNodeDesc expression) { if (expression.getClass() != ExprNodeGenericFuncDesc.class) { return null; } @@ -210,6 +211,7 @@ public Map computeOptimizedScan(ExprNodeG if (columnDesc.getColumn().equals(KafkaStreamingUtils.TIMESTAMP_COLUMN)) { long timestamp = ((Number) constantDesc.getValue()).longValue(); + //noinspection unchecked return buildScanForTimesPredicate(fullHouse, operator, timestamp, flip, negation, kafkaConsumer); } return null; @@ -226,7 +228,8 @@ public Map computeOptimizedScan(ExprNodeG * * @return filtered kafka scan */ - @VisibleForTesting protected static Map buildScanFromPartitionPredicate( + + @VisibleForTesting static Map buildScanFromPartitionPredicate( Map fullScan, PredicateLeaf.Operator operator, int partitionConst, @@ -277,8 +280,7 @@ public Map computeOptimizedScan(ExprNodeG * * @return optimized kafka scan */ - @VisibleForTesting protected static Map buildScanFromOffsetPredicate( - Map fullScan, + @VisibleForTesting static Map buildScanFromOffsetPredicate(Map fullScan, PredicateLeaf.Operator operator, long offsetConst, boolean flip, @@ -287,11 +289,7 @@ public Map computeOptimizedScan(ExprNodeG final long startOffset; final long endOffset; - if (flip == negation) { - isEndBound = true; - } else { - isEndBound = false; - } + isEndBound = flip == negation; switch (operator) { case LESS_THAN_EQUALS: if (isEndBound) { @@ -367,13 +365,13 @@ public Map computeOptimizedScan(ExprNodeG return newScan; } - @Nullable protected static Map buildScanForTimesPredicate( + @Nullable private static Map buildScanForTimesPredicate( Map fullHouse, PredicateLeaf.Operator operator, long timestamp, boolean flip, boolean negation, - KafkaConsumer consumer) { + KafkaConsumer consumer) { long increment = (flip && operator == PredicateLeaf.Operator.LESS_THAN @@ -385,18 +383,17 @@ public Map computeOptimizedScan(ExprNodeG // Based on Kafka docs // 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); - final Map newScan = Maps.toMap(fullHouse.keySet(), tp -> { + return Maps.toMap(fullHouse.keySet(), tp -> { KafkaPullerInputSplit 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(tp.topic(), + return new KafkaPullerInputSplit(Objects.requireNonNull(tp).topic(), tp.partition(), startOffset, existing.getEndOffset(), existing.getPath()); }); - return newScan; } catch (Exception e) { LOG.error("Error while looking up offsets for time", e); //Bailout when can not figure out offsets for times. @@ -418,10 +415,10 @@ private Map pushAndOp(ExprNodeGenericFunc fullHouse.forEach((tp, input) -> currentScan.put(tp, KafkaPullerInputSplit.copyOf(input))); for (ExprNodeDesc child : expr.getChildren()) { - Map scan = parse(child); + Map scan = parseAndOptimize(child); if (scan != null) { Set currentKeys = ImmutableSet.copyOf(currentScan.keySet()); - currentKeys.stream().forEach(key -> { + currentKeys.forEach(key -> { KafkaPullerInputSplit newSplit = scan.get(key); KafkaPullerInputSplit oldSplit = currentScan.get(key); currentScan.remove(key); @@ -441,7 +438,7 @@ private Map pushAndOp(ExprNodeGenericFunc @Nullable private Map pushOrOp(ExprNodeGenericFuncDesc expr) { final Map currentScan = new HashMap<>(); for (ExprNodeDesc child : expr.getChildren()) { - Map scan = parse(child); + Map scan = parseAndOptimize(child); if (scan == null) { // if any of the children is unknown bailout return null; @@ -455,7 +452,7 @@ private Map pushAndOp(ExprNodeGenericFunc return currentScan; } - private static ExprNodeDesc getColumnExpr(ExprNodeDesc expr) { + @SuppressWarnings("Duplicates") private static ExprNodeDesc getColumnExpr(ExprNodeDesc expr) { if (expr instanceof ExprNodeColumnDesc) { return expr; } From dd9f22fece43057baa9d2b483e83352ab7460b46 Mon Sep 17 00:00:00 2001 From: Slim Bouguerra Date: Wed, 22 Aug 2018 15:57:43 -0700 Subject: [PATCH 28/33] Addressing review comments Change-Id: Ide0513e4c36d4294fbbd65934f6671f78a8479eb --- .../hive/kafka/KafkaPullerRecordReader.java | 3 - .../hive/kafka/KafkaStreamingUtils.java | 61 +++++++++++-------- .../hive/kafka/KafkaStreamingUtilsTest.java | 15 +++++ 3 files changed, 50 insertions(+), 29 deletions(-) diff --git a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaPullerRecordReader.java b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaPullerRecordReader.java index e023687bf261..1529bf3d2355 100644 --- a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaPullerRecordReader.java +++ b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaPullerRecordReader.java @@ -158,9 +158,6 @@ private synchronized void initialize(KafkaPullerInputSplit inputSplit, Configura } @Override public void close() throws IOException { - if (!started) { - return; - } LOG.trace("total read bytes [{}]", readBytes); if (consumer != null) { consumer.wakeup(); diff --git a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaStreamingUtils.java b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaStreamingUtils.java index 3df5d49a0891..5d9cf68cd429 100644 --- a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaStreamingUtils.java +++ b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaStreamingUtils.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hive.kafka; +import com.google.common.collect.ImmutableList; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -26,6 +27,7 @@ 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; @@ -36,9 +38,6 @@ import java.util.Set; import java.util.stream.Collectors; -import static org.apache.kafka.clients.consumer.ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG; -import static org.apache.kafka.clients.consumer.ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG; - /** * Constant, Table properties, Utilities class. */ @@ -83,6 +82,12 @@ public final class KafkaStreamingUtils { */ protected static final String CONSUMER_CONFIGURATION_PREFIX = "kafka.consumer"; + /** + * Set of Kafka properties that the user can not set via DDLs + */ + private static final HashSet FORBIDDEN_PROPERTIES = + new HashSet(ImmutableList.of(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, ConsumerConfig.AUTO_OFFSET_RESET_CONFIG)); + private KafkaStreamingUtils() { } @@ -94,20 +99,27 @@ private KafkaStreamingUtils() { static Properties consumerProperties(Configuration configuration) { final Properties props = new Properties(); // we are managing the commit offset - props.setProperty("enable.auto.commit", "false"); + props.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false"); // we are seeking in the stream so no reset - props.setProperty("auto.offset.reset", "none"); + 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(KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName()); - props.setProperty(VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName()); + 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()) { - props.setProperty(entry.getKey().substring(CONSUMER_CONFIGURATION_PREFIX.length() + 1), - entry.getValue()); + 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; } @@ -116,23 +128,20 @@ static void copyDependencyJars(Configuration conf, Class... classes) throws I Set jars = new HashSet<>(); FileSystem localFs = FileSystem.getLocal(conf); jars.addAll(conf.getStringCollection("tmpjars")); - jars.addAll(Arrays.asList(classes).stream().filter(aClass -> aClass != null) - .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())); + jars.addAll(Arrays.asList(classes).stream().filter(aClass -> aClass != null).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; diff --git a/kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaStreamingUtilsTest.java b/kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaStreamingUtilsTest.java index 28c532f61113..8d68ec27c8ca 100644 --- a/kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaStreamingUtilsTest.java +++ b/kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaStreamingUtilsTest.java @@ -19,6 +19,7 @@ package org.apache.hadoop.hive.kafka; import org.apache.hadoop.conf.Configuration; +import org.apache.kafka.clients.consumer.ConsumerConfig; import org.junit.Assert; import org.junit.Test; @@ -41,4 +42,18 @@ public KafkaStreamingUtilsTest() { Assert.assertEquals("40", properties.getProperty("fetch.max.wait.ms")); Assert.assertEquals("400", properties.getProperty("my.new.wait.ms")); } + + @Test(expected = IllegalArgumentException.class) public void canNotSetForbiddenProp() { + Configuration configuration = new Configuration(); + configuration.set("kafka.bootstrap.servers", "localhost:9090"); + configuration.set("kafka.consumer." + ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false"); + KafkaStreamingUtils.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"); + KafkaStreamingUtils.consumerProperties(configuration); + } } From d36477521599c599c0191041977f9f6a7c082439 Mon Sep 17 00:00:00 2001 From: Slim Bouguerra Date: Wed, 22 Aug 2018 17:09:58 -0700 Subject: [PATCH 29/33] clean up code after code review Change-Id: I99a7a487a8948d88700e8edd13dcc041ee2f4cc9 --- .../hadoop/hive/kafka/KafkaJsonSerDe.java | 2 +- .../hive/kafka/KafkaPullerInputFormat.java | 19 +++++----- .../hive/kafka/KafkaPullerInputSplit.java | 21 ++++++----- .../hive/kafka/KafkaPullerRecordReader.java | 37 ++++++++----------- .../hive/kafka/KafkaRecordIterator.java | 2 +- .../hive/kafka/KafkaRecordWritable.java | 14 +++---- .../hive/kafka/KafkaStorageHandler.java | 3 ++ .../hive/kafka/KafkaStreamingUtils.java | 31 +++++++++------- 8 files changed, 66 insertions(+), 63 deletions(-) diff --git a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaJsonSerDe.java b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaJsonSerDe.java index 3de64dca44d3..f383190083a5 100644 --- a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaJsonSerDe.java +++ b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaJsonSerDe.java @@ -81,7 +81,7 @@ public class KafkaJsonSerDe extends AbstractSerDe { private static final Logger LOG = LoggerFactory.getLogger(KafkaJsonSerDe.class); private static final DateTimeFormatter TS_PARSER = createAutoParser(); - protected static Function + static Function typeInfoToObjectInspector = typeInfo -> PrimitiveObjectInspectorFactory.getPrimitiveWritableObjectInspector( TypeInfoFactory.getPrimitiveTypeInfo(typeInfo.getTypeName())); diff --git a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaPullerInputFormat.java b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaPullerInputFormat.java index ed421f8ccd12..2d5637d43022 100644 --- a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaPullerInputFormat.java +++ b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaPullerInputFormat.java @@ -84,7 +84,7 @@ public class KafkaPullerInputFormat extends InputFormat buildFullScanFromKafka(String topic, - KafkaConsumer consumer, + KafkaConsumer consumer, Path[] tablePaths) { final Map starOffsetsMap; final Map endOffsetsMap; @@ -96,7 +96,7 @@ private static List buildFullScanFromKafka(String topic, if (LOG.isDebugEnabled()) { LOG.info("Found the following partitions [{}]", - topicPartitions.stream().map(topicPartition -> topicPartition.toString()).collect(Collectors.joining(","))); + topicPartitions.stream().map(TopicPartition::toString).collect(Collectors.joining(","))); starOffsetsMap.forEach((tp, start) -> LOG.info("TPartition [{}],Start offsets [{}]", tp, start)); endOffsetsMap.forEach((tp, end) -> LOG.info("TPartition [{}],End offsets [{}]", tp, end)); } @@ -123,6 +123,7 @@ private List computeSplits(Configuration configuration) JobConf jobConf = new JobConf(configuration); Path[] tablePaths = org.apache.hadoop.mapred.FileInputFormat.getInputPaths(jobConf); + //noinspection unchecked Future> futureFullHouse = execService.submit(() -> buildFullScanFromKafka(topic, consumer, tablePaths)); @@ -136,11 +137,11 @@ private List computeSplits(Configuration configuration) throw new IOException(e); } - final ImmutableMap.Builder + @SuppressWarnings("unchecked") final ImmutableMap.Builder fullHouseMapBuilder = new ImmutableMap.Builder(); - fullHouse.stream() - .forEach(input -> fullHouseMapBuilder.put(new TopicPartition(input.getTopic(), input.getPartition()), input)); + fullHouse.forEach(input -> fullHouseMapBuilder.put(new TopicPartition(input.getTopic(), input.getPartition()), + input)); final KafkaScanTrimmer kafkaScanTrimmer = new KafkaScanTrimmer(fullHouseMapBuilder.build(), consumer); final String filterExprSerialized = configuration.get(TableScanDesc.FILTER_EXPR_CONF_STR); @@ -152,7 +153,7 @@ private List computeSplits(Configuration configuration) trimmerWorker = () -> kafkaScanTrimmer.computeOptimizedScan(filterExpr) .entrySet() .stream() - .map(entry -> entry.getValue()) + .map(Map.Entry::getValue) .collect(Collectors.toList()); Future> futureTinyHouse = execService.submit(trimmerWorker); @@ -175,7 +176,7 @@ private List computeSplits(Configuration configuration) } } - private static List fetchTopicPartitions(String topic, KafkaConsumer consumer) { + private static List fetchTopicPartitions(String topic, KafkaConsumer consumer) { // this will block till REQUEST_TIMEOUT_MS_CONFIG = "request.timeout.ms" // then throws org.apache.kafka.common.errors.TimeoutException if can not fetch metadata // @TODO add retry logic maybe @@ -185,7 +186,7 @@ private static List fetchTopicPartitions(String topic, KafkaCons @Override public RecordReader getRecordReader(InputSplit inputSplit, JobConf jobConf, - Reporter reporter) throws IOException { + Reporter reporter) { return new KafkaPullerRecordReader((KafkaPullerInputSplit) inputSplit, jobConf); } @@ -198,7 +199,7 @@ private static List fetchTopicPartitions(String topic, KafkaCons @Override public org.apache.hadoop.mapreduce.RecordReader createRecordReader( org.apache.hadoop.mapreduce.InputSplit inputSplit, - TaskAttemptContext taskAttemptContext) throws IOException, InterruptedException { + TaskAttemptContext taskAttemptContext) { return new KafkaPullerRecordReader(); } } diff --git a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaPullerInputSplit.java b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaPullerInputSplit.java index a4f1de415398..697469c9e08e 100644 --- a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaPullerInputSplit.java +++ b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaPullerInputSplit.java @@ -34,14 +34,15 @@ /** * Kafka Hadoop Input Split Class. */ -public class KafkaPullerInputSplit extends FileSplit implements org.apache.hadoop.mapred.InputSplit { +@SuppressWarnings("WeakerAccess") public class KafkaPullerInputSplit extends FileSplit + implements org.apache.hadoop.mapred.InputSplit { private String topic; private long startOffset; private int partition; private long endOffset; public KafkaPullerInputSplit() { - super((Path) null, 0, 0, (String[]) null); + super(null, 0, 0, (String[]) null); } public KafkaPullerInputSplit(String topic, int partition, long startOffset, long endOffset, Path dummyPath) { @@ -60,7 +61,7 @@ public KafkaPullerInputSplit(String topic, int partition, long startOffset, long return 0; } - @Override public String[] getLocations() throws IOException { + @Override public String[] getLocations() { return new String[0]; } @@ -103,14 +104,14 @@ public long getEndOffset() { /** * Compute the intersection of 2 splits. Splits must share the same topic and partition number. * - * @param split1 - * @param split2 + * @param split1 left split + * @param split2 right split * * @return new split that represents range intersection or null if it is not overlapping */ @Nullable public static KafkaPullerInputSplit intersectRange(KafkaPullerInputSplit split1, KafkaPullerInputSplit split2) { - assert (split1.topic == split2.topic); + assert (split1.topic.equals(split2.topic)); assert (split1.partition == split2.partition); final long startOffset = Math.max(split1.getStartOffset(), split2.getStartOffset()); final long endOffset = Math.min(split1.getEndOffset(), split2.getEndOffset()); @@ -124,13 +125,13 @@ public long getEndOffset() { /** * Compute union of ranges between splits. Splits must share the same topic and partition * - * @param split1 - * @param split2 + * @param split1 left split + * @param split2 right split * * @return new split with a range including both splits. */ public static KafkaPullerInputSplit unionRange(KafkaPullerInputSplit split1, KafkaPullerInputSplit split2) { - assert (split1.topic == split2.topic); + 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()); @@ -179,7 +180,7 @@ public static KafkaPullerInputSplit copyOf(KafkaPullerInputSplit other) { other.getPath()); } - public KafkaPullerInputSplit clone() { + @SuppressWarnings("MethodDoesntCallSuperMethod") public KafkaPullerInputSplit clone() { return copyOf(this); } diff --git a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaPullerRecordReader.java b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaPullerRecordReader.java index 1529bf3d2355..28454ec3a155 100644 --- a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaPullerRecordReader.java +++ b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaPullerRecordReader.java @@ -38,7 +38,7 @@ /** * Kafka Records Reader implementation. */ -public class KafkaPullerRecordReader extends RecordReader +@SuppressWarnings("UnstableApiUsage") public class KafkaPullerRecordReader extends RecordReader implements org.apache.hadoop.mapred.RecordReader { private static final Logger LOG = LoggerFactory.getLogger(KafkaPullerRecordReader.class); @@ -49,17 +49,12 @@ public class KafkaPullerRecordReader extends RecordReader> recordsCursor = null; - private TopicPartition topicPartition; - private long startOffset; - private long endOffset; - private long totalNumberRecords = 0L; private long consumedRecords = 0L; private long readBytes = 0L; - private long pollTimeout; private volatile boolean started = false; - public KafkaPullerRecordReader() { + @SuppressWarnings("WeakerAccess") public KafkaPullerRecordReader() { } private void initConsumer() { @@ -69,28 +64,28 @@ private void initConsumer() { 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); - consumer = new KafkaConsumer(properties); + consumer = new KafkaConsumer<>(properties); closer.register(consumer); } } - public KafkaPullerRecordReader(KafkaPullerInputSplit inputSplit, Configuration jobConf) { + @SuppressWarnings("WeakerAccess") public KafkaPullerRecordReader(KafkaPullerInputSplit inputSplit, + Configuration jobConf) { initialize(inputSplit, jobConf); } private synchronized void initialize(KafkaPullerInputSplit inputSplit, Configuration jobConf) { if (!started) { this.config = jobConf; - this.startOffset = inputSplit.getStartOffset(); - this.endOffset = inputSplit.getEndOffset(); - this.topicPartition = new TopicPartition(inputSplit.getTopic(), inputSplit.getPartition()); + long startOffset = inputSplit.getStartOffset(); + long 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(); - pollTimeout = + long + pollTimeout = config.getLong(KafkaStreamingUtils.HIVE_KAFKA_POLL_TIMEOUT, KafkaStreamingUtils.DEFAULT_CONSUMER_POLL_TIMEOUT_MS); LOG.debug("Consumer poll timeout [{}] ms", pollTimeout); @@ -126,11 +121,11 @@ private synchronized void initialize(KafkaPullerInputSplit inputSplit, Configura return new KafkaRecordWritable(); } - @Override public long getPos() throws IOException { + @Override public long getPos() { return -1; } - @Override public boolean nextKeyValue() throws IOException { + @Override public boolean nextKeyValue() { currentWritableValue = new KafkaRecordWritable(); if (next(NullWritable.get(), currentWritableValue)) { return true; @@ -139,15 +134,15 @@ private synchronized void initialize(KafkaPullerInputSplit inputSplit, Configura return false; } - @Override public NullWritable getCurrentKey() throws IOException, InterruptedException { + @Override public NullWritable getCurrentKey() { return NullWritable.get(); } - @Override public KafkaRecordWritable getCurrentValue() throws IOException, InterruptedException { + @Override public KafkaRecordWritable getCurrentValue() { return Preconditions.checkNotNull(currentWritableValue); } - @Override public float getProgress() throws IOException { + @Override public float getProgress() { if (consumedRecords == 0) { return 0f; } diff --git a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaRecordIterator.java b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaRecordIterator.java index edcd9e6dd52c..7daa3e254462 100644 --- a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaRecordIterator.java +++ b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaRecordIterator.java @@ -73,7 +73,7 @@ public class KafkaRecordIterator implements Iterator consumer, + KafkaRecordIterator(Consumer consumer, TopicPartition topicPartition, @Nullable Long startOffset, @Nullable Long endOffset, diff --git a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaRecordWritable.java b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaRecordWritable.java index b6b8d391313a..5b6b2b047a6c 100644 --- a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaRecordWritable.java +++ b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaRecordWritable.java @@ -39,14 +39,14 @@ public class KafkaRecordWritable implements Writable { private long timestamp; private byte[] value; - public static KafkaRecordWritable fromKafkaRecord(ConsumerRecord consumerRecord) { + static KafkaRecordWritable fromKafkaRecord(ConsumerRecord consumerRecord) { return new KafkaRecordWritable(consumerRecord.partition(), consumerRecord.offset(), consumerRecord.timestamp(), consumerRecord.value()); } - public void set(ConsumerRecord consumerRecord) { + void set(ConsumerRecord consumerRecord) { this.partition = consumerRecord.partition(); this.timestamp = consumerRecord.timestamp(); this.offset = consumerRecord.offset(); @@ -60,7 +60,7 @@ private KafkaRecordWritable(int partition, long offset, long timestamp, byte[] v this.value = value; } - public KafkaRecordWritable() { + @SuppressWarnings("WeakerAccess") public KafkaRecordWritable() { } @Override public void write(DataOutput dataOutput) throws IOException { @@ -84,19 +84,19 @@ public KafkaRecordWritable() { } } - public int getPartition() { + int getPartition() { return partition; } - public long getOffset() { + long getOffset() { return offset; } - public long getTimestamp() { + long getTimestamp() { return timestamp; } - public byte[] getValue() { + byte[] getValue() { return value; } diff --git a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaStorageHandler.java b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaStorageHandler.java index 81f365e6344c..5847df5e7e6b 100644 --- a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaStorageHandler.java +++ b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaStorageHandler.java @@ -91,6 +91,9 @@ public class KafkaStorageHandler implements HiveStorageHandler { .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); diff --git a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaStreamingUtils.java b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaStreamingUtils.java index 5d9cf68cd429..ae33908f01f8 100644 --- a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaStreamingUtils.java +++ b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaStreamingUtils.java @@ -34,6 +34,7 @@ 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.stream.Collectors; @@ -41,52 +42,53 @@ /** * Constant, Table properties, Utilities class. */ -public final class KafkaStreamingUtils { +final class KafkaStreamingUtils { /** * MANDATORY Table property indicating kafka topic backing the table */ - public static final String HIVE_KAFKA_TOPIC = "kafka.topic"; + static final String HIVE_KAFKA_TOPIC = "kafka.topic"; /** * MANDATORY Table property indicating kafka broker(s) connection string. */ - public static final String HIVE_KAFKA_BOOTSTRAP_SERVERS = "kafka.bootstrap.servers"; + 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} */ - public static final String SERDE_CLASS_NAME = "kafka.serde.class"; + 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} */ - public static final String HIVE_KAFKA_POLL_TIMEOUT = "hive.kafka.poll.timeout.ms"; + static final String HIVE_KAFKA_POLL_TIMEOUT = "hive.kafka.poll.timeout.ms"; /** * default poll timeout for fetching metadata and record batch */ - public static final long DEFAULT_CONSUMER_POLL_TIMEOUT_MS = 5000L; // 5 seconds + static final long DEFAULT_CONSUMER_POLL_TIMEOUT_MS = 5000L; // 5 seconds /** * Record Timestamp column name, added as extra meta column of type long */ - public static final String TIMESTAMP_COLUMN = "__timestamp"; + static final String TIMESTAMP_COLUMN = "__timestamp"; /** * Record Kafka Partition column name added as extra meta column of type int */ - public static final String PARTITION_COLUMN = "__partition"; + static final String PARTITION_COLUMN = "__partition"; /** * Record offset column name added as extra metadata column to row as long */ - public static final String OFFSET_COLUMN = "__offset"; + static final String OFFSET_COLUMN = "__offset"; /** * 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 */ - protected static final String CONSUMER_CONFIGURATION_PREFIX = "kafka.consumer"; + static final String CONSUMER_CONFIGURATION_PREFIX = "kafka.consumer"; /** * Set of Kafka properties that the user can not set via DDLs */ - private static final HashSet FORBIDDEN_PROPERTIES = - new HashSet(ImmutableList.of(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, ConsumerConfig.AUTO_OFFSET_RESET_CONFIG)); + static final HashSet FORBIDDEN_PROPERTIES = + new HashSet<>(ImmutableList.of(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, + ConsumerConfig.AUTO_OFFSET_RESET_CONFIG)); private KafkaStreamingUtils() { } @@ -128,7 +130,7 @@ static void copyDependencyJars(Configuration conf, Class... classes) throws I Set jars = new HashSet<>(); FileSystem localFs = FileSystem.getLocal(conf); jars.addAll(conf.getStringCollection("tmpjars")); - jars.addAll(Arrays.asList(classes).stream().filter(aClass -> aClass != null).map(clazz -> { + 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."); @@ -146,12 +148,13 @@ static void copyDependencyJars(Configuration conf, Class... classes) throws I if (jars.isEmpty()) { return; } - conf.set("tmpjars", StringUtils.arrayToString(jars.toArray(new String[jars.size()]))); + 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); From 6bff3b24ccaf7936e3880e4838bb70035488a8ce Mon Sep 17 00:00:00 2001 From: Slim Bouguerra Date: Wed, 22 Aug 2018 18:16:11 -0700 Subject: [PATCH 30/33] added start and end offset markers Change-Id: I3a5f72a0a39003db4cb63f3f98909afded13d1cb --- .../hadoop/hive/kafka/GenericKafkaSerDe.java | 14 +- .../hive/kafka/KafkaPullerRecordReader.java | 8 +- .../hive/kafka/KafkaRecordWritable.java | 67 +++++-- .../hive/kafka/KafkaStreamingUtils.java | 10 + .../hive/kafka/KafkaRecordIteratorTest.java | 63 ++++--- .../hive/kafka/KafkaRecordWritableTest.java | 2 +- .../clientpositive/kafka_storage_handler.q | 24 ++- .../druid/kafka_storage_handler.q.out | 174 +++++++++++------- 8 files changed, 236 insertions(+), 126 deletions(-) diff --git a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/GenericKafkaSerDe.java b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/GenericKafkaSerDe.java index 2c9732975e79..bc300942928b 100644 --- a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/GenericKafkaSerDe.java +++ b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/GenericKafkaSerDe.java @@ -69,10 +69,16 @@ public class GenericKafkaSerDe extends AbstractSerDe { METADATA_COLUMN_NAMES = ImmutableList.of(KafkaStreamingUtils.PARTITION_COLUMN, KafkaStreamingUtils.OFFSET_COLUMN, - KafkaStreamingUtils.TIMESTAMP_COLUMN); + KafkaStreamingUtils.TIMESTAMP_COLUMN, + KafkaStreamingUtils.START_OFFSET_COUMN, + KafkaStreamingUtils.END_OFFSET_COUMN); private static final ImmutableList METADATA_PRIMITIVE_TYPE_INFO = - ImmutableList.of(TypeInfoFactory.intTypeInfo, TypeInfoFactory.longTypeInfo, TypeInfoFactory.longTypeInfo); + ImmutableList.of(TypeInfoFactory.intTypeInfo, + TypeInfoFactory.longTypeInfo, + TypeInfoFactory.longTypeInfo, + TypeInfoFactory.longTypeInfo, + TypeInfoFactory.longTypeInfo); private AbstractSerDe delegateSerDe; private ObjectInspector objectInspector; @@ -160,6 +166,10 @@ public class GenericKafkaSerDe extends AbstractSerDe { return new LongWritable(record.getOffset()); case KafkaStreamingUtils.TIMESTAMP_COLUMN: return new LongWritable(record.getTimestamp()); + case KafkaStreamingUtils.START_OFFSET_COUMN: + return new LongWritable(record.getStartOffset()); + case KafkaStreamingUtils.END_OFFSET_COUMN: + return new LongWritable(record.getEndOffset()); default: return delegateObjectInspector.getStructFieldData(row, delegateObjectInspector.getStructFieldRef(name)); } diff --git a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaPullerRecordReader.java b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaPullerRecordReader.java index 28454ec3a155..908ee5e29fb9 100644 --- a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaPullerRecordReader.java +++ b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaPullerRecordReader.java @@ -53,6 +53,8 @@ private long consumedRecords = 0L; private long readBytes = 0L; private volatile boolean started = false; + private long startOffset = -1L; + private long endOffset = Long.MAX_VALUE; @SuppressWarnings("WeakerAccess") public KafkaPullerRecordReader() { } @@ -77,8 +79,8 @@ private void initConsumer() { private synchronized void initialize(KafkaPullerInputSplit inputSplit, Configuration jobConf) { if (!started) { this.config = jobConf; - long startOffset = inputSplit.getStartOffset(); - long endOffset = inputSplit.getEndOffset(); + 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); @@ -105,7 +107,7 @@ private synchronized void initialize(KafkaPullerInputSplit inputSplit, Configura @Override public boolean next(NullWritable nullWritable, KafkaRecordWritable bytesWritable) { if (started && recordsCursor.hasNext()) { ConsumerRecord record = recordsCursor.next(); - bytesWritable.set(record); + bytesWritable.set(record, startOffset, endOffset); consumedRecords += 1; readBytes += record.serializedValueSize(); return true; diff --git a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaRecordWritable.java b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaRecordWritable.java index 5b6b2b047a6c..3088951aa521 100644 --- a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaRecordWritable.java +++ b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaRecordWritable.java @@ -36,28 +36,32 @@ public class KafkaRecordWritable implements Writable { private int partition; private long offset; + private long startOffset; + private long endOffset; private long timestamp; private byte[] value; - static KafkaRecordWritable fromKafkaRecord(ConsumerRecord consumerRecord) { - return new KafkaRecordWritable(consumerRecord.partition(), - consumerRecord.offset(), - consumerRecord.timestamp(), - consumerRecord.value()); - } - - void set(ConsumerRecord consumerRecord) { + void set(ConsumerRecord consumerRecord, long startOffset, long endOffset) { this.partition = consumerRecord.partition(); this.timestamp = consumerRecord.timestamp(); this.offset = consumerRecord.offset(); this.value = consumerRecord.value(); + this.startOffset = startOffset; + this.endOffset = endOffset; } - private KafkaRecordWritable(int partition, long offset, long timestamp, byte[] value) { + KafkaRecordWritable(int partition, + long offset, + long timestamp, + byte[] value, + long startOffset, + long endOffset) { this.partition = partition; this.offset = offset; this.timestamp = timestamp; this.value = value; + this.startOffset = startOffset; + this.endOffset = endOffset; } @SuppressWarnings("WeakerAccess") public KafkaRecordWritable() { @@ -67,6 +71,8 @@ private KafkaRecordWritable(int partition, long offset, long timestamp, byte[] v dataOutput.writeLong(timestamp); dataOutput.writeInt(partition); dataOutput.writeLong(offset); + dataOutput.writeLong(startOffset); + dataOutput.writeLong(endOffset); dataOutput.writeInt(value.length); dataOutput.write(value); } @@ -75,6 +81,8 @@ private KafkaRecordWritable(int partition, long offset, long timestamp, byte[] v timestamp = dataInput.readLong(); partition = dataInput.readInt(); offset = dataInput.readLong(); + startOffset = dataInput.readLong(); + endOffset = dataInput.readLong(); int size = dataInput.readInt(); if (size > 0) { value = new byte[size]; @@ -100,6 +108,14 @@ byte[] getValue() { return value; } + long getStartOffset() { + return startOffset; + } + + long getEndOffset() { + return endOffset; + } + @Override public boolean equals(Object o) { if (this == o) { return true; @@ -107,18 +123,35 @@ byte[] getValue() { if (!(o instanceof KafkaRecordWritable)) { return false; } - KafkaRecordWritable that = (KafkaRecordWritable) o; - return getPartition() == that.getPartition() - && getOffset() == that.getOffset() - && getTimestamp() == that.getTimestamp() - && Arrays.equals(getValue(), that.getValue()); + KafkaRecordWritable writable = (KafkaRecordWritable) o; + return partition == writable.partition + && offset == writable.offset + && startOffset == writable.startOffset + && endOffset == writable.endOffset + && timestamp == writable.timestamp + && Arrays.equals(value, writable.value); } @Override public int hashCode() { - - int result = Objects.hash(getPartition(), getOffset(), getTimestamp()); - result = 31 * result + Arrays.hashCode(getValue()); + int result = Objects.hash(partition, offset, startOffset, endOffset, timestamp); + result = 31 * result + Arrays.hashCode(value); return result; } + @Override public String toString() { + return "KafkaRecordWritable{" + + "partition=" + + partition + + ", offset=" + + offset + + ", startOffset=" + + startOffset + + ", endOffset=" + + endOffset + + ", timestamp=" + + timestamp + + ", value=" + + Arrays.toString(value) + + '}'; + } } diff --git a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaStreamingUtils.java b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaStreamingUtils.java index ae33908f01f8..308e11fdd891 100644 --- a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaStreamingUtils.java +++ b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaStreamingUtils.java @@ -77,6 +77,16 @@ final class KafkaStreamingUtils { * Record offset column name added as extra metadata column to row as long */ static final String OFFSET_COLUMN = "__offset"; + + /** + * Start offset given by the input split, this will reflect the actual start of TP or start given by split pruner + */ + static final String START_OFFSET_COUMN = "__start_offset"; + + /** + * End offset given by input split at run time + */ + static final String END_OFFSET_COUMN = "__end_offset"; /** * 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 diff --git a/kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaRecordIteratorTest.java b/kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaRecordIteratorTest.java index 2fd7c78204b1..5de51cd00a70 100644 --- a/kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaRecordIteratorTest.java +++ b/kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaRecordIteratorTest.java @@ -74,8 +74,9 @@ public class KafkaRecordIteratorTest { 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); + return new ConsumerRecord<>(TOPIC, 0, (long) number, 0L, null, 0L, 0, 0, KEY_BYTES, value); }).collect(Collectors.toList()); + public static final long POLL_TIMEOUT_MS = 900L; private static ZkUtils zkUtils; private static ZkClient zkClient; private static KafkaProducer producer; @@ -89,7 +90,7 @@ public class KafkaRecordIteratorTest { public KafkaRecordIteratorTest() { } - @BeforeClass public static void setupCluster() throws IOException, InterruptedException { + @BeforeClass public static void setupCluster() throws IOException { LOG.info("init embedded Zookeeper"); zkServer = new EmbeddedZookeeper(); zkConnect = "127.0.0.1:" + zkServer.port(); @@ -119,40 +120,42 @@ public KafkaRecordIteratorTest() { } @Test public void testHasNextAbsoluteStartEnd() { - this.kafkaRecordIterator = new KafkaRecordIterator(this.consumer, TOPIC_PARTITION, 0L, (long) RECORDS.size(), 200L); + this.kafkaRecordIterator = new KafkaRecordIterator(this.consumer, TOPIC_PARTITION, 0L, (long) RECORDS.size(), POLL_TIMEOUT_MS); this.compareIterator(RECORDS, this.kafkaRecordIterator); } @Test public void testHasNextGivenStartEnd() { - this.kafkaRecordIterator = new KafkaRecordIterator(this.consumer, TOPIC_PARTITION, 2L, 4L, 100L); + this.kafkaRecordIterator = new KafkaRecordIterator(this.consumer, TOPIC_PARTITION, 2L, 4L, POLL_TIMEOUT_MS); this.compareIterator(RECORDS.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, 100L); + this.kafkaRecordIterator = new KafkaRecordIterator(this.consumer, TOPIC_PARTITION, POLL_TIMEOUT_MS); this.compareIterator(RECORDS, this.kafkaRecordIterator); } @Test public void testHasNextLastRecord() { long startOffset = (long) (RECORDS.size() - 1); long lastOffset = (long) RECORDS.size(); - this.kafkaRecordIterator = new KafkaRecordIterator(this.consumer, TOPIC_PARTITION, startOffset, lastOffset, 100L); + this.kafkaRecordIterator = + new KafkaRecordIterator(this.consumer, TOPIC_PARTITION, startOffset, lastOffset, POLL_TIMEOUT_MS); this.compareIterator(RECORDS.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, 100L); + this.kafkaRecordIterator = new KafkaRecordIterator(this.consumer, TOPIC_PARTITION, 0L, 1L, POLL_TIMEOUT_MS); this.compareIterator(RECORDS.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, (Long) null, 10L, 100L); + this.kafkaRecordIterator = + new KafkaRecordIterator(this.consumer, TOPIC_PARTITION, null, 10L, POLL_TIMEOUT_MS); this.compareIterator(RECORDS.stream() .filter((consumerRecord) -> consumerRecord.offset() >= 0L && consumerRecord.offset() < 10L) .collect(Collectors.toList()), this.kafkaRecordIterator); @@ -160,27 +163,25 @@ public KafkaRecordIteratorTest() { @Test public void testHasNextNoEnd() { long lastOffset = (long) RECORDS.size(); - this.kafkaRecordIterator = new KafkaRecordIterator(this.consumer, TOPIC_PARTITION, 5L, (Long) null, 100L); + this.kafkaRecordIterator = + new KafkaRecordIterator(this.consumer, TOPIC_PARTITION, 5L, null, POLL_TIMEOUT_MS); this.compareIterator(RECORDS.stream() .filter((consumerRecord) -> consumerRecord.offset() >= 5L && consumerRecord.offset() < lastOffset) .collect(Collectors.toList()), this.kafkaRecordIterator); } @Test public void testRecordReader() throws IOException { - InputSplit inputSplits = new KafkaPullerInputSplit(TOPIC, 0, 0L, 50L, null); - KafkaPullerRecordReader recordReader = new KafkaPullerRecordReader((KafkaPullerInputSplit) inputSplits, this.conf); List serRecords = - RECORDS.stream().map((aRecord) -> KafkaRecordWritable.fromKafkaRecord(aRecord)).collect(Collectors.toList()); - - for (int i = 0; i < 50; ++i) { - KafkaRecordWritable record = new KafkaRecordWritable(); - Assert.assertTrue(recordReader.next((NullWritable) null, record)); - Assert.assertEquals(serRecords.get(i), record); - } - - recordReader.close(); - recordReader = new KafkaPullerRecordReader(); + RECORDS.stream() + .map((aRecord) -> new KafkaRecordWritable(aRecord.partition(), + aRecord.offset(), + aRecord.timestamp(), + aRecord.value(), + 50L, + 100L)) + .collect(Collectors.toList()); + KafkaPullerRecordReader recordReader = new KafkaPullerRecordReader(); TaskAttemptContext context = new TaskAttemptContextImpl(this.conf, new TaskAttemptID()); recordReader.initialize(new KafkaPullerInputSplit(TOPIC, 0, 50L, 100L, null), context); @@ -194,38 +195,44 @@ public KafkaRecordIteratorTest() { } @Test(expected = TimeoutException.class) public void testPullingBeyondLimit() { - this.kafkaRecordIterator = new KafkaRecordIterator(this.consumer, TOPIC_PARTITION, 0L, 101L, 100L); + this.kafkaRecordIterator = new KafkaRecordIterator(this.consumer, TOPIC_PARTITION, 0L, 101L, POLL_TIMEOUT_MS); this.compareIterator(RECORDS, this.kafkaRecordIterator); } @Test(expected = IllegalStateException.class) public void testPullingStartGreaterThanEnd() { - this.kafkaRecordIterator = new KafkaRecordIterator(this.consumer, TOPIC_PARTITION, 10L, 1L, 100L); + this.kafkaRecordIterator = new KafkaRecordIterator(this.consumer, TOPIC_PARTITION, 10L, 1L, POLL_TIMEOUT_MS); this.compareIterator(RECORDS, this.kafkaRecordIterator); } @Test(expected = TimeoutException.class) public void testPullingFromEmptyTopic() { - this.kafkaRecordIterator = new KafkaRecordIterator(this.consumer, new TopicPartition("noHere", 0), 0L, 100L, 100L); + this.kafkaRecordIterator = + new KafkaRecordIterator(this.consumer, new TopicPartition("noHere", 0), 0L, 100L, POLL_TIMEOUT_MS); this.compareIterator(RECORDS, this.kafkaRecordIterator); } @Test(expected = TimeoutException.class) public void testPullingFromEmptyPartition() { - this.kafkaRecordIterator = new KafkaRecordIterator(this.consumer, new TopicPartition(TOPIC, 1), 0L, 100L, 100L); + this.kafkaRecordIterator = + new KafkaRecordIterator(this.consumer, new TopicPartition(TOPIC, 1), 0L, 100L, POLL_TIMEOUT_MS); this.compareIterator(RECORDS, this.kafkaRecordIterator); } @Test public void testStartIsEqualEnd() { - this.kafkaRecordIterator = new KafkaRecordIterator(this.consumer, TOPIC_PARTITION, 10L, 10L, 100L); + this.kafkaRecordIterator = new KafkaRecordIterator(this.consumer, TOPIC_PARTITION, 10L, 10L, POLL_TIMEOUT_MS); this.compareIterator(ImmutableList.of(), this.kafkaRecordIterator); } @Test public void testStartIsTheLastOffset() { this.kafkaRecordIterator = - new KafkaRecordIterator(this.consumer, TOPIC_PARTITION, new Long(RECORD_NUMBER), new Long(RECORD_NUMBER), 100L); + new KafkaRecordIterator(this.consumer, + TOPIC_PARTITION, + new Long(RECORD_NUMBER), + new Long(RECORD_NUMBER), + POLL_TIMEOUT_MS); this.compareIterator(ImmutableList.of(), this.kafkaRecordIterator); } @Test public void testStartIsTheFirstOffset() { - this.kafkaRecordIterator = new KafkaRecordIterator(this.consumer, TOPIC_PARTITION, 0L, 0L, 100L); + this.kafkaRecordIterator = new KafkaRecordIterator(this.consumer, TOPIC_PARTITION, 0L, 0L, POLL_TIMEOUT_MS); this.compareIterator(ImmutableList.of(), this.kafkaRecordIterator); } diff --git a/kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaRecordWritableTest.java b/kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaRecordWritableTest.java index e28c924a9ad9..8f9df548f7cc 100644 --- a/kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaRecordWritableTest.java +++ b/kafka-handler/src/test/org/apache/hadoop/hive/kafka/KafkaRecordWritableTest.java @@ -37,7 +37,7 @@ public KafkaRecordWritableTest() { @Test public void testWriteReadFields() throws IOException { ConsumerRecord record = new ConsumerRecord("topic", 0, 3L, "key".getBytes(), "value".getBytes()); - KafkaRecordWritable kafkaRecordWritable = KafkaRecordWritable.fromKafkaRecord(record); + KafkaRecordWritable kafkaRecordWritable = new KafkaRecordWritable(record.partition(), record.offset(), record.timestamp(), record.value(), 0L, 100L); ByteArrayOutputStream baos = new ByteArrayOutputStream(); DataOutputStream w = new DataOutputStream(baos); kafkaRecordWritable.write(w); diff --git a/ql/src/test/queries/clientpositive/kafka_storage_handler.q b/ql/src/test/queries/clientpositive/kafka_storage_handler.q index 1a9edb53c4bd..8daa3e3bc05c 100644 --- a/ql/src/test/queries/clientpositive/kafka_storage_handler.q +++ b/ql/src/test/queries/clientpositive/kafka_storage_handler.q @@ -14,26 +14,32 @@ TBLPROPERTIES DESCRIBE EXTENDED kafka_table; -Select `__partition` , `__offset`,`__time`, `page`, `user`, `language`, `country`,`continent`, `namespace`, `newPage` , +Select `__partition` ,`__start_offset`,`__end_offset`, `__offset`,`__time`, `page`, `user`, `language`, `country`,`continent`, `namespace`, `newPage` , `unpatrolled` , `anonymous` , `robot` , added , deleted , delta FROM kafka_table; Select count(*) FROM kafka_table; -Select `__partition`, `__offset`,`__time`, `page`, `user`, `language`, `country`,`continent`, `namespace`, `newPage` , +Select `__partition`, `__offset`,`__start_offset`,`__end_offset`, `__time`, `page`, `user`, `language`, `country`,`continent`, `namespace`, `newPage` , `unpatrolled` , `anonymous` , `robot` , added , deleted , delta from kafka_table where `__timestamp` > 1533960760123; -Select `__partition`, `__offset`, `__time`, `page`, `user`, `language`, `country`,`continent`, `namespace`, `newPage` , +Select `__partition`, `__offset` ,`__start_offset`,`__end_offset`,`__time`, `page`, `user`, `language`, `country`,`continent`, `namespace`, `newPage` , `unpatrolled` , `anonymous` , `robot` , added , deleted , delta from kafka_table where `__timestamp` > 533960760123; -Select `__partition`, `__offset`,`__time`, `page`, `user`, `language`, `country`,`continent`, `namespace`, `newPage` , +Select `__partition`,`__start_offset`,`__end_offset`, `__offset`,`__time`, `page`, `user`, `language`, `country`,`continent`, `namespace`, `newPage` , `unpatrolled` , `anonymous` , `robot` , added , deleted , delta -from kafka_table where `__offset` > 7 and `__partition` = 0 OR -`__offset` = 4 and `__partition` = 0 OR `__offset` <= 1 and `__partition` = 0; +from kafka_table where (`__offset` > 7 and `__partition` = 0 and `__offset` <9 ) OR +`__offset` = 4 and `__partition` = 0 OR (`__offset` <= 1 and `__partition` = 0 and `__offset` > 0); + +Select `__partition`,`__start_offset`,`__end_offset`, `__offset`,`__time`, `page`, `user` from kafka_table where `__offset` = 5; + +Select `__partition`,`__start_offset`,`__end_offset`, `__offset`,`__time`, `page`, `user` from kafka_table where `__offset` < 5; + +Select `__partition`,`__start_offset`,`__end_offset`, `__offset`,`__time`, `page`, `user` from kafka_table where `__offset` > 5; -- Timestamp filter -Select `__partition`, `__offset`, `user` from kafka_table where +Select `__partition`,`__start_offset`,`__end_offset`, `__offset`, `user` from kafka_table where `__timestamp` > 1000 * to_unix_timestamp(CURRENT_TIMESTAMP - interval '1' HOURS) ; -- non existing partition @@ -224,5 +230,7 @@ select count(distinct `user`) from wiki_kafka_avro_table; select sum(deltabucket), min(commentlength) from wiki_kafka_avro_table; -select cast ((`__timestamp`/1000) as timestamp) as kafka_record_ts, `__timestamp` as kafka_record_ts_long, `__partition`, `__offset`, `timestamp`, `user`, `page`, `deleted`, `deltabucket`, `isanonymous`, `commentlength` from wiki_kafka_avro_table where `__timestamp` > 1534750625090; +select cast ((`__timestamp`/1000) as timestamp) as kafka_record_ts, `__timestamp` as kafka_record_ts_long, +`__partition`, `__start_offset`,`__end_offset`,`__offset`, `timestamp`, `user`, `page`, `deleted`, `deltabucket`, +`isanonymous`, `commentlength` from wiki_kafka_avro_table where `__timestamp` > 1534750625090; diff --git a/ql/src/test/results/clientpositive/druid/kafka_storage_handler.q.out b/ql/src/test/results/clientpositive/druid/kafka_storage_handler.q.out index 0d788315493b..3dec33d79088 100644 --- a/ql/src/test/results/clientpositive/druid/kafka_storage_handler.q.out +++ b/ql/src/test/results/clientpositive/druid/kafka_storage_handler.q.out @@ -47,28 +47,30 @@ delta bigint from deserializer __partition int from deserializer __offset bigint from deserializer __timestamp bigint from deserializer +__start_offset bigint from deserializer +__end_offset bigint from deserializer #### A masked pattern was here #### -PREHOOK: query: Select `__partition` , `__offset`,`__time`, `page`, `user`, `language`, `country`,`continent`, `namespace`, `newPage` , +PREHOOK: query: Select `__partition` ,`__start_offset`,`__end_offset`, `__offset`,`__time`, `page`, `user`, `language`, `country`,`continent`, `namespace`, `newPage` , `unpatrolled` , `anonymous` , `robot` , added , deleted , delta FROM kafka_table PREHOOK: type: QUERY PREHOOK: Input: default@kafka_table PREHOOK: Output: hdfs://### HDFS PATH ### -POSTHOOK: query: Select `__partition` , `__offset`,`__time`, `page`, `user`, `language`, `country`,`continent`, `namespace`, `newPage` , +POSTHOOK: query: Select `__partition` ,`__start_offset`,`__end_offset`, `__offset`,`__time`, `page`, `user`, `language`, `country`,`continent`, `namespace`, `newPage` , `unpatrolled` , `anonymous` , `robot` , added , deleted , delta FROM kafka_table POSTHOOK: type: QUERY POSTHOOK: Input: default@kafka_table POSTHOOK: Output: hdfs://### HDFS PATH ### -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 +0 0 10 0 NULL Gypsy Danger nuclear en United States North America article true true false false 57 200 -143 +0 0 10 1 NULL Striker Eureka speed en Australia Australia wikipedia true false false true 459 129 330 +0 0 10 2 NULL Cherno Alpha masterYi ru Russia Asia article true false false true 123 12 111 +0 0 10 3 NULL Crimson Typhoon triplets zh China Asia wikipedia false true false true 905 5 900 +0 0 10 4 NULL Coyote Tango stringer ja Japan Asia wikipedia false true false true 1 10 -9 +0 0 10 5 NULL Gypsy Danger nuclear en United States North America article true true false false 57 200 -143 +0 0 10 6 NULL Striker Eureka speed en Australia Australia wikipedia true false false true 459 129 330 +0 0 10 7 NULL Cherno Alpha masterYi ru Russia Asia article true false false true 123 12 111 +0 0 10 8 NULL Crimson Typhoon triplets zh China Asia wikipedia false true false true 905 5 900 +0 0 10 9 NULL Coyote Tango stringer ja Japan Asia wikipedia false true false true 1 10 -9 PREHOOK: query: Select count(*) FROM kafka_table PREHOOK: type: QUERY PREHOOK: Input: default@kafka_table @@ -78,89 +80,121 @@ POSTHOOK: type: QUERY POSTHOOK: Input: default@kafka_table POSTHOOK: Output: hdfs://### HDFS PATH ### 10 -PREHOOK: query: Select `__partition`, `__offset`,`__time`, `page`, `user`, `language`, `country`,`continent`, `namespace`, `newPage` , +PREHOOK: query: Select `__partition`, `__offset`,`__start_offset`,`__end_offset`, `__time`, `page`, `user`, `language`, `country`,`continent`, `namespace`, `newPage` , `unpatrolled` , `anonymous` , `robot` , added , deleted , delta from kafka_table where `__timestamp` > 1533960760123 PREHOOK: type: QUERY PREHOOK: Input: default@kafka_table PREHOOK: Output: hdfs://### HDFS PATH ### -POSTHOOK: query: Select `__partition`, `__offset`,`__time`, `page`, `user`, `language`, `country`,`continent`, `namespace`, `newPage` , +POSTHOOK: query: Select `__partition`, `__offset`,`__start_offset`,`__end_offset`, `__time`, `page`, `user`, `language`, `country`,`continent`, `namespace`, `newPage` , `unpatrolled` , `anonymous` , `robot` , added , deleted , delta from kafka_table where `__timestamp` > 1533960760123 POSTHOOK: type: QUERY POSTHOOK: Input: default@kafka_table POSTHOOK: Output: hdfs://### HDFS PATH ### -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 `__partition`, `__offset`, `__time`, `page`, `user`, `language`, `country`,`continent`, `namespace`, `newPage` , +0 0 0 10 NULL Gypsy Danger nuclear en United States North America article true true false false 57 200 -143 +0 1 0 10 NULL Striker Eureka speed en Australia Australia wikipedia true false false true 459 129 330 +0 2 0 10 NULL Cherno Alpha masterYi ru Russia Asia article true false false true 123 12 111 +0 3 0 10 NULL Crimson Typhoon triplets zh China Asia wikipedia false true false true 905 5 900 +0 4 0 10 NULL Coyote Tango stringer ja Japan Asia wikipedia false true false true 1 10 -9 +0 5 0 10 NULL Gypsy Danger nuclear en United States North America article true true false false 57 200 -143 +0 6 0 10 NULL Striker Eureka speed en Australia Australia wikipedia true false false true 459 129 330 +0 7 0 10 NULL Cherno Alpha masterYi ru Russia Asia article true false false true 123 12 111 +0 8 0 10 NULL Crimson Typhoon triplets zh China Asia wikipedia false true false true 905 5 900 +0 9 0 10 NULL Coyote Tango stringer ja Japan Asia wikipedia false true false true 1 10 -9 +PREHOOK: query: Select `__partition`, `__offset` ,`__start_offset`,`__end_offset`,`__time`, `page`, `user`, `language`, `country`,`continent`, `namespace`, `newPage` , `unpatrolled` , `anonymous` , `robot` , added , deleted , delta from kafka_table where `__timestamp` > 533960760123 PREHOOK: type: QUERY PREHOOK: Input: default@kafka_table PREHOOK: Output: hdfs://### HDFS PATH ### -POSTHOOK: query: Select `__partition`, `__offset`, `__time`, `page`, `user`, `language`, `country`,`continent`, `namespace`, `newPage` , +POSTHOOK: query: Select `__partition`, `__offset` ,`__start_offset`,`__end_offset`,`__time`, `page`, `user`, `language`, `country`,`continent`, `namespace`, `newPage` , `unpatrolled` , `anonymous` , `robot` , added , deleted , delta from kafka_table where `__timestamp` > 533960760123 POSTHOOK: type: QUERY POSTHOOK: Input: default@kafka_table POSTHOOK: Output: hdfs://### HDFS PATH ### -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 `__partition`, `__offset`,`__time`, `page`, `user`, `language`, `country`,`continent`, `namespace`, `newPage` , +0 0 0 10 NULL Gypsy Danger nuclear en United States North America article true true false false 57 200 -143 +0 1 0 10 NULL Striker Eureka speed en Australia Australia wikipedia true false false true 459 129 330 +0 2 0 10 NULL Cherno Alpha masterYi ru Russia Asia article true false false true 123 12 111 +0 3 0 10 NULL Crimson Typhoon triplets zh China Asia wikipedia false true false true 905 5 900 +0 4 0 10 NULL Coyote Tango stringer ja Japan Asia wikipedia false true false true 1 10 -9 +0 5 0 10 NULL Gypsy Danger nuclear en United States North America article true true false false 57 200 -143 +0 6 0 10 NULL Striker Eureka speed en Australia Australia wikipedia true false false true 459 129 330 +0 7 0 10 NULL Cherno Alpha masterYi ru Russia Asia article true false false true 123 12 111 +0 8 0 10 NULL Crimson Typhoon triplets zh China Asia wikipedia false true false true 905 5 900 +0 9 0 10 NULL Coyote Tango stringer ja Japan Asia wikipedia false true false true 1 10 -9 +PREHOOK: query: Select `__partition`,`__start_offset`,`__end_offset`, `__offset`,`__time`, `page`, `user`, `language`, `country`,`continent`, `namespace`, `newPage` , `unpatrolled` , `anonymous` , `robot` , added , deleted , delta -from kafka_table where `__offset` > 7 and `__partition` = 0 OR -`__offset` = 4 and `__partition` = 0 OR `__offset` <= 1 and `__partition` = 0 +from kafka_table where (`__offset` > 7 and `__partition` = 0 and `__offset` <9 ) OR +`__offset` = 4 and `__partition` = 0 OR (`__offset` <= 1 and `__partition` = 0 and `__offset` > 0) PREHOOK: type: QUERY PREHOOK: Input: default@kafka_table PREHOOK: Output: hdfs://### HDFS PATH ### -POSTHOOK: query: Select `__partition`, `__offset`,`__time`, `page`, `user`, `language`, `country`,`continent`, `namespace`, `newPage` , +POSTHOOK: query: Select `__partition`,`__start_offset`,`__end_offset`, `__offset`,`__time`, `page`, `user`, `language`, `country`,`continent`, `namespace`, `newPage` , `unpatrolled` , `anonymous` , `robot` , added , deleted , delta -from kafka_table where `__offset` > 7 and `__partition` = 0 OR -`__offset` = 4 and `__partition` = 0 OR `__offset` <= 1 and `__partition` = 0 +from kafka_table where (`__offset` > 7 and `__partition` = 0 and `__offset` <9 ) OR +`__offset` = 4 and `__partition` = 0 OR (`__offset` <= 1 and `__partition` = 0 and `__offset` > 0) POSTHOOK: type: QUERY POSTHOOK: Input: default@kafka_table POSTHOOK: Output: hdfs://### HDFS PATH ### -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 4 NULL Coyote Tango stringer ja Japan Asia wikipedia false true false true 1 10 -9 -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 `__partition`, `__offset`, `user` from kafka_table where +0 1 9 1 NULL Striker Eureka speed en Australia Australia wikipedia true false false true 459 129 330 +0 1 9 4 NULL Coyote Tango stringer ja Japan Asia wikipedia false true false true 1 10 -9 +0 1 9 8 NULL Crimson Typhoon triplets zh China Asia wikipedia false true false true 905 5 900 +PREHOOK: query: Select `__partition`,`__start_offset`,`__end_offset`, `__offset`,`__time`, `page`, `user` from kafka_table where `__offset` = 5 +PREHOOK: type: QUERY +PREHOOK: Input: default@kafka_table +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: Select `__partition`,`__start_offset`,`__end_offset`, `__offset`,`__time`, `page`, `user` from kafka_table where `__offset` = 5 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@kafka_table +POSTHOOK: Output: hdfs://### HDFS PATH ### +0 5 6 5 NULL Gypsy Danger nuclear +PREHOOK: query: Select `__partition`,`__start_offset`,`__end_offset`, `__offset`,`__time`, `page`, `user` from kafka_table where `__offset` < 5 +PREHOOK: type: QUERY +PREHOOK: Input: default@kafka_table +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: Select `__partition`,`__start_offset`,`__end_offset`, `__offset`,`__time`, `page`, `user` from kafka_table where `__offset` < 5 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@kafka_table +POSTHOOK: Output: hdfs://### HDFS PATH ### +0 0 5 0 NULL Gypsy Danger nuclear +0 0 5 1 NULL Striker Eureka speed +0 0 5 2 NULL Cherno Alpha masterYi +0 0 5 3 NULL Crimson Typhoon triplets +0 0 5 4 NULL Coyote Tango stringer +PREHOOK: query: Select `__partition`,`__start_offset`,`__end_offset`, `__offset`,`__time`, `page`, `user` from kafka_table where `__offset` > 5 +PREHOOK: type: QUERY +PREHOOK: Input: default@kafka_table +PREHOOK: Output: hdfs://### HDFS PATH ### +POSTHOOK: query: Select `__partition`,`__start_offset`,`__end_offset`, `__offset`,`__time`, `page`, `user` from kafka_table where `__offset` > 5 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@kafka_table +POSTHOOK: Output: hdfs://### HDFS PATH ### +0 6 10 6 NULL Striker Eureka speed +0 6 10 7 NULL Cherno Alpha masterYi +0 6 10 8 NULL Crimson Typhoon triplets +0 6 10 9 NULL Coyote Tango stringer +PREHOOK: query: Select `__partition`,`__start_offset`,`__end_offset`, `__offset`, `user` from kafka_table where `__timestamp` > 1000 * to_unix_timestamp(CURRENT_TIMESTAMP - interval '1' HOURS) PREHOOK: type: QUERY PREHOOK: Input: default@kafka_table PREHOOK: Output: hdfs://### HDFS PATH ### -POSTHOOK: query: Select `__partition`, `__offset`, `user` from kafka_table where +POSTHOOK: query: Select `__partition`,`__start_offset`,`__end_offset`, `__offset`, `user` from kafka_table where `__timestamp` > 1000 * to_unix_timestamp(CURRENT_TIMESTAMP - interval '1' HOURS) POSTHOOK: type: QUERY POSTHOOK: Input: default@kafka_table POSTHOOK: Output: hdfs://### HDFS PATH ### -0 0 nuclear -0 1 speed -0 2 masterYi -0 3 triplets -0 4 stringer -0 5 nuclear -0 6 speed -0 7 masterYi -0 8 triplets -0 9 stringer +0 0 10 0 nuclear +0 0 10 1 speed +0 0 10 2 masterYi +0 0 10 3 triplets +0 0 10 4 stringer +0 0 10 5 nuclear +0 0 10 6 speed +0 0 10 7 masterYi +0 0 10 8 triplets +0 0 10 9 stringer PREHOOK: query: Select count(*) from kafka_table where `__partition` = 1 PREHOOK: type: QUERY PREHOOK: Input: default@kafka_table @@ -741,6 +775,8 @@ namespace string from deserializer __partition int from deserializer __offset bigint from deserializer __timestamp bigint from deserializer +__start_offset bigint from deserializer +__end_offset bigint from deserializer #### A masked pattern was here #### 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 @@ -789,17 +825,21 @@ POSTHOOK: type: QUERY POSTHOOK: Input: default@wiki_kafka_avro_table POSTHOOK: Output: hdfs://### HDFS PATH ### 5522.000000000001 0 -PREHOOK: query: select cast ((`__timestamp`/1000) as timestamp) as kafka_record_ts, `__timestamp` as kafka_record_ts_long, `__partition`, `__offset`, `timestamp`, `user`, `page`, `deleted`, `deltabucket`, `isanonymous`, `commentlength` from wiki_kafka_avro_table where `__timestamp` > 1534750625090 +PREHOOK: query: select cast ((`__timestamp`/1000) as timestamp) as kafka_record_ts, `__timestamp` as kafka_record_ts_long, +`__partition`, `__start_offset`,`__end_offset`,`__offset`, `timestamp`, `user`, `page`, `deleted`, `deltabucket`, +`isanonymous`, `commentlength` from wiki_kafka_avro_table where `__timestamp` > 1534750625090 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, `__timestamp` as kafka_record_ts_long, `__partition`, `__offset`, `timestamp`, `user`, `page`, `deleted`, `deltabucket`, `isanonymous`, `commentlength` from wiki_kafka_avro_table where `__timestamp` > 1534750625090 +POSTHOOK: query: select cast ((`__timestamp`/1000) as timestamp) as kafka_record_ts, `__timestamp` as kafka_record_ts_long, +`__partition`, `__start_offset`,`__end_offset`,`__offset`, `timestamp`, `user`, `page`, `deleted`, `deltabucket`, +`isanonymous`, `commentlength` from wiki_kafka_avro_table where `__timestamp` > 1534750625090 POSTHOOK: type: QUERY POSTHOOK: Input: default@wiki_kafka_avro_table POSTHOOK: Output: hdfs://### HDFS PATH ### -2018-08-20 08:37:05.09 1534754225090 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 1534757825090 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 1534761425090 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 1534765025090 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 1534768625090 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 1534772225090 0 10 08/20/2018 06:37:05 test-user-10 page is 1000 -10 1004.0 true 10 +2018-08-20 08:37:05.09 1534754225090 0 5 11 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 1534757825090 0 5 11 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 1534761425090 0 5 11 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 1534765025090 0 5 11 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 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 10 08/20/2018 06:37:05 test-user-10 page is 1000 -10 1004.0 true 10 From 71184a9d9f4d0c0a7386eb2d4687fb4bc7bc2e5b Mon Sep 17 00:00:00 2001 From: Slim Bouguerra Date: Thu, 23 Aug 2018 09:46:53 -0700 Subject: [PATCH 31/33] fix docs for KafkaRecordWritable Change-Id: I849765d4c79e7af5cfc2c5145fd41c8fa8e7ac19 --- .../hive/kafka/KafkaRecordWritable.java | 21 ++++++++++++++++++- 1 file changed, 20 insertions(+), 1 deletion(-) diff --git a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaRecordWritable.java b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaRecordWritable.java index 3088951aa521..c6924ea480db 100644 --- a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaRecordWritable.java +++ b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaRecordWritable.java @@ -30,15 +30,34 @@ /** * Writable implementation of Kafka ConsumerRecord. * Serialized in the form - * kafkaRecordTimestamp(long) | kafkaPartition (int) | recordOffset (long) | value.size (int) | value (byte []) + * {@code timestamp} long| {@code partition} (int) | {@code offset} (long) | + * {@code startOffset} (long) | {@code endOffset} (long) | {@code value.size()} (int) | {@code value} (byte []) */ public class KafkaRecordWritable implements Writable { + /** + * Kafka partition id + */ private int partition; + /** + * Record Offset + */ private long offset; + /** + * Fist offset given by the input split used to pull the event {@link KafkaPullerInputSplit#getStartOffset()} + */ private long startOffset; + /** + * Last Offset given by the input split used to pull the event {@link KafkaPullerInputSplit#getEndOffset()} + */ private long endOffset; + /** + * Event timestamp provided by Kafka Record {@link ConsumerRecord#timestamp()} + */ private long timestamp; + /** + * Record value + */ private byte[] value; void set(ConsumerRecord consumerRecord, long startOffset, long endOffset) { From f5469102ec630a635fc0b422cc376514d7933a65 Mon Sep 17 00:00:00 2001 From: Slim Bouguerra Date: Thu, 23 Aug 2018 09:50:46 -0700 Subject: [PATCH 32/33] fix name of variable Change-Id: Ibb76b253c1f642d17e545130cdee256795d32695 --- .../org/apache/hadoop/hive/kafka/GenericKafkaSerDe.java | 8 ++++---- .../org/apache/hadoop/hive/kafka/KafkaStreamingUtils.java | 4 ++-- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/GenericKafkaSerDe.java b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/GenericKafkaSerDe.java index bc300942928b..e7ea53f4bc84 100644 --- a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/GenericKafkaSerDe.java +++ b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/GenericKafkaSerDe.java @@ -70,8 +70,8 @@ public class GenericKafkaSerDe extends AbstractSerDe { ImmutableList.of(KafkaStreamingUtils.PARTITION_COLUMN, KafkaStreamingUtils.OFFSET_COLUMN, KafkaStreamingUtils.TIMESTAMP_COLUMN, - KafkaStreamingUtils.START_OFFSET_COUMN, - KafkaStreamingUtils.END_OFFSET_COUMN); + KafkaStreamingUtils.START_OFFSET_COLUMN, + KafkaStreamingUtils.END_OFFSET_COLUMN); private static final ImmutableList METADATA_PRIMITIVE_TYPE_INFO = ImmutableList.of(TypeInfoFactory.intTypeInfo, @@ -166,9 +166,9 @@ public class GenericKafkaSerDe extends AbstractSerDe { return new LongWritable(record.getOffset()); case KafkaStreamingUtils.TIMESTAMP_COLUMN: return new LongWritable(record.getTimestamp()); - case KafkaStreamingUtils.START_OFFSET_COUMN: + case KafkaStreamingUtils.START_OFFSET_COLUMN: return new LongWritable(record.getStartOffset()); - case KafkaStreamingUtils.END_OFFSET_COUMN: + case KafkaStreamingUtils.END_OFFSET_COLUMN: return new LongWritable(record.getEndOffset()); default: return delegateObjectInspector.getStructFieldData(row, delegateObjectInspector.getStructFieldRef(name)); diff --git a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaStreamingUtils.java b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaStreamingUtils.java index 308e11fdd891..d2d0ebc1926b 100644 --- a/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaStreamingUtils.java +++ b/kafka-handler/src/java/org/apache/hadoop/hive/kafka/KafkaStreamingUtils.java @@ -81,12 +81,12 @@ final class KafkaStreamingUtils { /** * Start offset given by the input split, this will reflect the actual start of TP or start given by split pruner */ - static final String START_OFFSET_COUMN = "__start_offset"; + static final String START_OFFSET_COLUMN = "__start_offset"; /** * End offset given by input split at run time */ - static final String END_OFFSET_COUMN = "__end_offset"; + static final String END_OFFSET_COLUMN = "__end_offset"; /** * 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 From 980847d2b901cd74ec466851b62e7778bb24e8b0 Mon Sep 17 00:00:00 2001 From: Slim Bouguerra Date: Tue, 28 Aug 2018 11:23:43 -0700 Subject: [PATCH 33/33] add property to avoid deletion of records with old ts Change-Id: Ia64009c983557299b9bb31c52acb073aeb957d9e --- .../java/org/apache/hive/kafka/SingleNodeKafkaCluster.java | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/itests/qtest-druid/src/main/java/org/apache/hive/kafka/SingleNodeKafkaCluster.java b/itests/qtest-druid/src/main/java/org/apache/hive/kafka/SingleNodeKafkaCluster.java index 8dc26c6cf853..c9339b565e76 100644 --- a/itests/qtest-druid/src/main/java/org/apache/hive/kafka/SingleNodeKafkaCluster.java +++ b/itests/qtest-druid/src/main/java/org/apache/hive/kafka/SingleNodeKafkaCluster.java @@ -51,6 +51,9 @@ public SingleNodeKafkaCluster(String name, String logDir, Integer zkPort){ properties.setProperty("host.name", LOCALHOST); properties.setProperty("port", Integer.toString(BROKER_PORT)); properties.setProperty("log.dir", logDir); + // This property is very important, we are sending form records with a specific time + // Thus need to make sure that they don't get DELETED + properties.setProperty("log.retention.hours", String.valueOf(Integer.MAX_VALUE)); properties.setProperty("log.flush.interval.messages", String.valueOf(1)); properties.setProperty("offsets.topic.replication.factor", String.valueOf(1)); properties.setProperty("offsets.topic.num.partitions", String.valueOf(1)); @@ -120,10 +123,10 @@ public void createTopicWithData(String topic, List events) { // 1534736225090 -> 08/19/2018 20:37:05 IntStream.range(0, events.size()) .mapToObj(i -> new ProducerRecord<>(topic, - null, + 0, // 1534736225090 -> Mon Aug 20 2018 03:37:05 1534736225090L + 1000 * 3600 * i, - "key".getBytes(), + ("key-" + i).getBytes(), events.get(i))) .forEach(r -> producer.send(r)); }