From 539d2dbf3d315257c05e7a4c4be7b73b2f3edc6a Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Mon, 18 Nov 2019 15:12:08 -0800 Subject: [PATCH 1/8] Support inputFormat and inputSource for sampler --- .../druid/data/input/AbstractInputSource.java | 6 +- .../org/apache/druid/data/input/Firehose.java | 14 +- .../FirehoseFactoryToInputSourceAdaptor.java | 5 + .../druid/data/input/InputEntityReader.java | 9 - .../apache/druid/data/input/InputFormat.java | 4 +- .../data/input/InputRowListPlusJson.java | 63 +- .../apache/druid/data/input/InputSource.java | 6 +- .../input/IntermediateRowParsingReader.java | 28 +- .../druid/data/input/impl/ByteEntity.java | 5 + .../druid/data/input/impl/CsvReader.java | 5 +- .../input/impl/FileIteratingFirehose.java | 5 +- .../FirehoseToInputSourceReaderAdaptor.java | 4 +- .../data/input/impl/InlineInputSource.java | 76 ++ .../impl/InputEntityIteratingReader.java | 19 +- .../druid/data/input/impl/JsonReader.java | 6 +- .../data/input/impl/RegexInputFormat.java | 62 + .../druid/data/input/impl/RegexReader.java | 122 ++ .../data/input/impl/StringInputRowParser.java | 4 +- .../input/impl/TimedShutoffInputSource.java | 82 ++ .../impl/TimedShutoffInputSourceReader.java | 116 ++ .../java/util/common/CloseableIterators.java | 3 + .../common/parsers/CloseableIterator.java | 82 +- .../impl/TimedShutoffInputSourceTest.java | 57 + .../common/parsers/CloseableIteratorTest.java | 102 ++ ...ementalPublishingKafkaIndexTaskRunner.java | 4 +- .../druid/indexing/kafka/KafkaIndexTask.java | 2 +- .../kafka/KafkaIndexTaskIOConfig.java | 13 +- .../indexing/kafka/KafkaSamplerSpec.java | 50 +- .../kafka/supervisor/KafkaSupervisor.java | 5 +- .../supervisor/KafkaSupervisorIOConfig.java | 3 + .../indexing/kafka/KafkaIOConfigTest.java | 3 +- .../indexing/kafka/KafkaIndexTaskTest.java | 231 ++-- .../indexing/kafka/KafkaSamplerSpecTest.java | 92 +- .../kafka/supervisor/KafkaSupervisorTest.java | 41 +- .../indexing/kinesis/KinesisIndexTask.java | 3 +- .../kinesis/KinesisIndexTaskIOConfig.java | 7 +- .../kinesis/KinesisIndexTaskRunner.java | 2 +- .../indexing/kinesis/KinesisSamplerSpec.java | 67 +- .../kinesis/supervisor/KinesisSupervisor.java | 3 + .../supervisor/KinesisSupervisorIOConfig.java | 3 + .../indexing/kinesis/KinesisIOConfigTest.java | 1 + .../kinesis/KinesisIndexTaskTest.java | 168 ++- .../kinesis/KinesisSamplerSpecTest.java | 112 +- .../supervisor/KinesisSupervisorTest.java | 37 +- .../druid/indexing/common/task/IndexTask.java | 5 + .../overlord/sampler/FirehoseSampler.java | 299 ----- .../sampler/IndexTaskSamplerSpec.java | 54 +- .../overlord/sampler/InputSourceSampler.java | 230 ++++ .../overlord/sampler/SamplerCache.java | 179 --- .../overlord/sampler/SamplerConfig.java | 41 +- .../overlord/sampler/SamplerModule.java | 3 +- .../overlord/sampler/SamplerResponse.java | 36 +- .../RecordSupplierInputSource.java | 150 +++ .../SeekableStreamIndexTaskIOConfig.java | 20 +- .../SeekableStreamIndexTaskRunner.java | 60 +- .../SeekableStreamSamplerSpec.java | 200 +-- .../SeekableStreamSupervisorIOConfig.java | 25 + .../druid/indexing/common/TestFirehose.java | 86 +- .../overlord/sampler/FirehoseSamplerTest.java | 839 ------------- .../sampler/IndexTaskSamplerSpecTest.java | 50 +- .../sampler/InputSourceSamplerTest.java | 1100 +++++++++++++++++ .../overlord/sampler/SamplerCacheTest.java | 177 --- .../overlord/sampler/SamplerResponseTest.java | 10 +- .../RecordSupplierInputSourceTest.java | 219 ++++ .../SeekableStreamIndexTaskTestBase.java | 112 +- .../SeekableStreamSupervisorStateTest.java | 33 +- .../indexer/AbstractKafkaIndexerTest.java | 13 +- .../indexer/ITKafkaIndexingServiceTest.java | 24 +- ...KafkaIndexingServiceTransactionalTest.java | 24 +- .../kafka_supervisor_spec_input_format.json | 58 + ... kafka_supervisor_spec_legacy_parser.json} | 0 .../segment/incremental/IncrementalIndex.java | 2 +- .../segment/transform/TransformSpec.java | 8 +- .../druid/segment/transform/Transformer.java | 25 +- .../TransformingInputEntityReader.java | 51 + ...ava => TransformingInputSourceReader.java} | 4 +- .../druid/segment/indexing/DataSchema.java | 4 +- .../granularity/ArbitraryGranularitySpec.java | 3 +- .../realtime/firehose/InlineFirehose.java | 5 +- .../firehose/InlineFirehoseFactory.java | 2 +- .../firehose/TimedShutoffFirehoseFactory.java | 21 +- .../realtime/firehose/InlineFirehoseTest.java | 38 +- 82 files changed, 3595 insertions(+), 2347 deletions(-) create mode 100644 core/src/main/java/org/apache/druid/data/input/impl/InlineInputSource.java create mode 100644 core/src/main/java/org/apache/druid/data/input/impl/RegexInputFormat.java create mode 100644 core/src/main/java/org/apache/druid/data/input/impl/RegexReader.java create mode 100644 core/src/main/java/org/apache/druid/data/input/impl/TimedShutoffInputSource.java create mode 100644 core/src/main/java/org/apache/druid/data/input/impl/TimedShutoffInputSourceReader.java create mode 100644 core/src/test/java/org/apache/druid/data/input/impl/TimedShutoffInputSourceTest.java create mode 100644 core/src/test/java/org/apache/druid/java/util/common/parsers/CloseableIteratorTest.java delete mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/FirehoseSampler.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/InputSourceSampler.java delete mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/SamplerCache.java create mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/RecordSupplierInputSource.java delete mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/FirehoseSamplerTest.java create mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/InputSourceSamplerTest.java delete mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/SamplerCacheTest.java create mode 100644 indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/RecordSupplierInputSourceTest.java create mode 100644 integration-tests/src/test/resources/indexer/kafka_supervisor_spec_input_format.json rename integration-tests/src/test/resources/indexer/{kafka_supervisor_spec.json => kafka_supervisor_spec_legacy_parser.json} (100%) create mode 100644 processing/src/main/java/org/apache/druid/segment/transform/TransformingInputEntityReader.java rename processing/src/main/java/org/apache/druid/segment/transform/{TransformingReader.java => TransformingInputSourceReader.java} (90%) diff --git a/core/src/main/java/org/apache/druid/data/input/AbstractInputSource.java b/core/src/main/java/org/apache/druid/data/input/AbstractInputSource.java index 94ddcd6b79e7..1dee243c2006 100644 --- a/core/src/main/java/org/apache/druid/data/input/AbstractInputSource.java +++ b/core/src/main/java/org/apache/druid/data/input/AbstractInputSource.java @@ -35,7 +35,7 @@ public abstract class AbstractInputSource implements InputSource public InputSourceReader reader( InputRowSchema inputRowSchema, @Nullable InputFormat inputFormat, - @Nullable File temporaryDirectory + File temporaryDirectory ) { if (needsFormat()) { @@ -52,13 +52,13 @@ public InputSourceReader reader( protected InputSourceReader formattableReader( InputRowSchema inputRowSchema, InputFormat inputFormat, - @Nullable File temporaryDirectory + File temporaryDirectory ) { throw new UnsupportedOperationException("Implement this method properly if needsFormat() = true"); } - protected InputSourceReader fixedFormatReader(InputRowSchema inputRowSchema, @Nullable File temporaryDirectory) + protected InputSourceReader fixedFormatReader(InputRowSchema inputRowSchema, File temporaryDirectory) { throw new UnsupportedOperationException("Implement this method properly if needsFormat() = false"); } diff --git a/core/src/main/java/org/apache/druid/data/input/Firehose.java b/core/src/main/java/org/apache/druid/data/input/Firehose.java index 6957c4b2ad3b..a95c55df74d1 100644 --- a/core/src/main/java/org/apache/druid/data/input/Firehose.java +++ b/core/src/main/java/org/apache/druid/data/input/Firehose.java @@ -67,20 +67,22 @@ public interface Firehose extends Closeable InputRow nextRow() throws IOException; /** - * Returns an InputRowPlusRaw object containing the InputRow plus the raw, unparsed data corresponding to the next row - * available. Used in the sampler to provide the caller with information to assist in configuring a parse spec. If a - * ParseException is thrown by the parser, it should be caught and returned in the InputRowPlusRaw so we will be able - * to provide information on the raw row which failed to be parsed. Should only be called if hasMore returns true. + * Returns an {@link InputRowListPlusJson} object containing the InputRow plus the raw, unparsed data corresponding to + * the next row available. Used in the sampler to provide the caller with information to assist in configuring a parse + * spec. If a ParseException is thrown by the parser, it should be caught and returned in the InputRowListPlusJson so + * we will be able to provide information on the raw row which failed to be parsed. Should only be called if hasMore + * returns true. * - * @return an InputRowPlusRaw which may contain any of: an InputRow, the raw data, or a ParseException + * @return an InputRowListPlusJson which may contain any of: an InputRow, map of the raw data, or a ParseException */ + @Deprecated default InputRowListPlusJson nextRowWithRaw() throws IOException { try { return InputRowListPlusJson.of(nextRow(), null); } catch (ParseException e) { - return InputRowListPlusJson.of((byte[]) null, e); + return InputRowListPlusJson.of(null, e); } } diff --git a/core/src/main/java/org/apache/druid/data/input/FirehoseFactoryToInputSourceAdaptor.java b/core/src/main/java/org/apache/druid/data/input/FirehoseFactoryToInputSourceAdaptor.java index 11466ad6583f..37ef3ca9eee9 100644 --- a/core/src/main/java/org/apache/druid/data/input/FirehoseFactoryToInputSourceAdaptor.java +++ b/core/src/main/java/org/apache/druid/data/input/FirehoseFactoryToInputSourceAdaptor.java @@ -45,6 +45,11 @@ public FiniteFirehoseFactory getFirehoseFactory() return firehoseFactory; } + public InputRowParser getInputRowParser() + { + return inputRowParser; + } + @Override public boolean isSplittable() { diff --git a/core/src/main/java/org/apache/druid/data/input/InputEntityReader.java b/core/src/main/java/org/apache/druid/data/input/InputEntityReader.java index 2dd351c898a5..7152a751924b 100644 --- a/core/src/main/java/org/apache/druid/data/input/InputEntityReader.java +++ b/core/src/main/java/org/apache/druid/data/input/InputEntityReader.java @@ -19,8 +19,6 @@ package org.apache.druid.data.input; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.databind.ObjectWriter; import org.apache.druid.guice.annotations.UnstableApi; import org.apache.druid.java.util.common.parsers.CloseableIterator; @@ -36,13 +34,6 @@ @UnstableApi public interface InputEntityReader { - /** - * Default JSON writer for sampler. This writer can be used to create an {@link InputRowListPlusJson}. - * Note that this writer uses the default serializer of Jackson. You may want to create a custom writer - * to serialize your custom types. - */ - ObjectWriter DEFAULT_JSON_WRITER = new ObjectMapper().writerWithDefaultPrettyPrinter(); - CloseableIterator read() throws IOException; CloseableIterator sample() throws IOException; diff --git a/core/src/main/java/org/apache/druid/data/input/InputFormat.java b/core/src/main/java/org/apache/druid/data/input/InputFormat.java index 682a7bdaa954..2ea04bd1e0b5 100644 --- a/core/src/main/java/org/apache/druid/data/input/InputFormat.java +++ b/core/src/main/java/org/apache/druid/data/input/InputFormat.java @@ -26,6 +26,7 @@ import org.apache.druid.data.input.impl.CsvInputFormat; import org.apache.druid.data.input.impl.JsonInputFormat; import org.apache.druid.data.input.impl.NestedInputFormat; +import org.apache.druid.data.input.impl.RegexInputFormat; import org.apache.druid.data.input.impl.SplittableInputSource; import org.apache.druid.guice.annotations.UnstableApi; @@ -42,7 +43,8 @@ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") @JsonSubTypes(value = { @Type(name = "csv", value = CsvInputFormat.class), - @Type(name = "json", value = JsonInputFormat.class) + @Type(name = "json", value = JsonInputFormat.class), + @Type(name = "regex", value = RegexInputFormat.class) }) public interface InputFormat { diff --git a/core/src/main/java/org/apache/druid/data/input/InputRowListPlusJson.java b/core/src/main/java/org/apache/druid/data/input/InputRowListPlusJson.java index 27c42b3b06fa..dc47098ae3c7 100644 --- a/core/src/main/java/org/apache/druid/data/input/InputRowListPlusJson.java +++ b/core/src/main/java/org/apache/druid/data/input/InputRowListPlusJson.java @@ -19,12 +19,13 @@ package org.apache.druid.data.input; -import com.google.common.collect.Iterables; +import com.google.common.base.Preconditions; import org.apache.druid.java.util.common.parsers.ParseException; import javax.annotation.Nullable; import java.util.Collections; import java.util.List; +import java.util.Map; public class InputRowListPlusJson { @@ -32,72 +33,47 @@ public class InputRowListPlusJson private final List inputRows; @Nullable - private final byte[] raw; - - @Nullable - private final String rawJson; + private final Map rawColumns; @Nullable private final ParseException parseException; - public static InputRowListPlusJson of(@Nullable InputRow inputRow, @Nullable byte[] raw) - { - return new InputRowListPlusJson(inputRow == null ? null : Collections.singletonList(inputRow), raw, null, null); - } - - public static InputRowListPlusJson of(@Nullable List inputRows, @Nullable String jsonRaw) + public static InputRowListPlusJson of(@Nullable InputRow inputRow, Map rawColumns) { - return new InputRowListPlusJson(inputRows, null, jsonRaw, null); + return of(inputRow == null ? null : Collections.singletonList(inputRow), rawColumns); } - public static InputRowListPlusJson of(@Nullable byte[] raw, @Nullable ParseException parseException) + public static InputRowListPlusJson of(@Nullable List inputRows, Map rawColumns) { - return new InputRowListPlusJson(null, raw, null, parseException); + return new InputRowListPlusJson(inputRows, Preconditions.checkNotNull(rawColumns, "rawColumns"), null); } - public static InputRowListPlusJson of(@Nullable String jsonRaw, @Nullable ParseException parseException) + public static InputRowListPlusJson of(@Nullable Map rawColumns, ParseException parseException) { - return new InputRowListPlusJson(null, null, jsonRaw, parseException); + return new InputRowListPlusJson(null, rawColumns, Preconditions.checkNotNull(parseException, "parseException")); } - private InputRowListPlusJson(@Nullable List inputRows, @Nullable byte[] raw, @Nullable String rawJson, @Nullable ParseException parseException) + private InputRowListPlusJson( + @Nullable List inputRows, + @Nullable Map rawColumns, + @Nullable ParseException parseException + ) { this.inputRows = inputRows; - this.raw = raw; - this.rawJson = rawJson; + this.rawColumns = rawColumns; this.parseException = parseException; } - @Nullable - public InputRow getInputRow() - { - return inputRows == null ? null : Iterables.getOnlyElement(inputRows); - } - @Nullable public List getInputRows() { return inputRows; } - /** - * The raw, unparsed event (as opposed to an {@link InputRow} which is the output of a parser). The interface default - * for {@link Firehose#nextRowWithRaw()} sets this to null, so this will only be non-null if nextRowWithRaw() is - * overridden by an implementation, such as in - * {@link org.apache.druid.data.input.impl.FileIteratingFirehose#nextRowWithRaw()}. Note that returning the raw row - * does not make sense for some sources (e.g. non-row based types), so clients should be able to handle this field - * being unset. - */ - @Nullable - public byte[] getRaw() - { - return raw; - } - @Nullable - public String getRawJson() + public Map getRawValues() { - return rawJson; + return rawColumns; } @Nullable @@ -105,9 +81,4 @@ public ParseException getParseException() { return parseException; } - - public boolean isEmpty() - { - return (inputRows == null || inputRows.isEmpty()) && raw == null && rawJson == null && parseException == null; - } } diff --git a/core/src/main/java/org/apache/druid/data/input/InputSource.java b/core/src/main/java/org/apache/druid/data/input/InputSource.java index 6109c592604c..56992a12f2a8 100644 --- a/core/src/main/java/org/apache/druid/data/input/InputSource.java +++ b/core/src/main/java/org/apache/druid/data/input/InputSource.java @@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonSubTypes.Type; import com.fasterxml.jackson.annotation.JsonTypeInfo; import org.apache.druid.data.input.impl.HttpInputSource; +import org.apache.druid.data.input.impl.InlineInputSource; import org.apache.druid.data.input.impl.LocalInputSource; import org.apache.druid.guice.annotations.UnstableApi; @@ -48,7 +49,8 @@ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") @JsonSubTypes(value = { @Type(name = "local", value = LocalInputSource.class), - @Type(name = "http", value = HttpInputSource.class) + @Type(name = "http", value = HttpInputSource.class), + @Type(name = "inline", value = InlineInputSource.class) }) public interface InputSource { @@ -76,6 +78,6 @@ public interface InputSource InputSourceReader reader( InputRowSchema inputRowSchema, @Nullable InputFormat inputFormat, - @Nullable File temporaryDirectory + File temporaryDirectory ); } diff --git a/core/src/main/java/org/apache/druid/data/input/IntermediateRowParsingReader.java b/core/src/main/java/org/apache/druid/data/input/IntermediateRowParsingReader.java index 313498198eb8..103992a99c02 100644 --- a/core/src/main/java/org/apache/druid/data/input/IntermediateRowParsingReader.java +++ b/core/src/main/java/org/apache/druid/data/input/IntermediateRowParsingReader.java @@ -25,6 +25,7 @@ import java.io.IOException; import java.util.List; +import java.util.Map; /** * {@link InputEntityReader} that parses bytes into some intermediate rows first, and then into {@link InputRow}s. @@ -59,31 +60,30 @@ public CloseableIterator sample() throws IOException { return intermediateRowIterator().map(row -> { - final String json; + final Map rawColumns; try { - json = toJson(row); + rawColumns = toMap(row); } - catch (IOException e) { - throw new RuntimeException(e); + catch (Exception e) { + return InputRowListPlusJson.of(null, new ParseException(e, "Unable to parse row [%s] into JSON", row)); } try { - return InputRowListPlusJson.of(parseInputRows(row), json); + return InputRowListPlusJson.of(parseInputRows(row), rawColumns); } catch (ParseException e) { - return InputRowListPlusJson.of(json, e); + return InputRowListPlusJson.of(rawColumns, e); } catch (IOException e) { - return InputRowListPlusJson.of(json, new ParseException(e, "Unable to parse row [%s]", row)); + return InputRowListPlusJson.of(rawColumns, new ParseException(e, "Unable to parse row [%s] into inputRow", row)); } }); } /** * Creates an iterator of intermediate rows. The returned rows will be consumed by {@link #parseInputRows} and - * {@link #toJson}. + * {@link #toMap}. */ - protected abstract CloseableIterator intermediateRowIterator() - throws IOException; + protected abstract CloseableIterator intermediateRowIterator() throws IOException; /** * Parses the given intermediate row into a list of {@link InputRow}s. @@ -91,10 +91,8 @@ protected abstract CloseableIterator intermediateRowIterator() protected abstract List parseInputRows(T intermediateRow) throws IOException, ParseException; /** - * Converts the given intermediate row into a JSON string. The returned JSON will be used by FirehoseSampler. - * Implementations can use any method to convert the given row into JSON string. If the implementation knows how - * to convert the given row into {@link java.util.Map}, {@link InputEntityReader#DEFAULT_JSON_WRITER} can be - * used to convert the Map into JSON string. + * Converts the given intermediate row into a {@link Map}. The returned JSON will be used by FirehoseSampler. + * Implementations can use any method to convert the given row into a Map. */ - protected abstract String toJson(T intermediateRow) throws IOException; + protected abstract Map toMap(T intermediateRow) throws IOException; } diff --git a/core/src/main/java/org/apache/druid/data/input/impl/ByteEntity.java b/core/src/main/java/org/apache/druid/data/input/impl/ByteEntity.java index f356b2084dd0..52c187ebdad3 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/ByteEntity.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/ByteEntity.java @@ -43,6 +43,11 @@ public ByteEntity(byte[] bytes) this(ByteBuffer.wrap(bytes)); } + public ByteBuffer getBuffer() + { + return buffer; + } + @Override @Nullable public URI getUri() diff --git a/core/src/main/java/org/apache/druid/data/input/impl/CsvReader.java b/core/src/main/java/org/apache/druid/data/input/impl/CsvReader.java index 2fb3d8f17940..3f52a1a31eb4 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/CsvReader.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/CsvReader.java @@ -89,10 +89,9 @@ public List parseInputRows(String line) throws IOException, ParseExcep } @Override - public String toJson(String intermediateRow) throws IOException + public Map toMap(String intermediateRow) throws IOException { - final Map zipped = parseLine(intermediateRow); - return DEFAULT_JSON_WRITER.writeValueAsString(zipped); + return parseLine(intermediateRow); } private Map parseLine(String line) throws IOException diff --git a/core/src/main/java/org/apache/druid/data/input/impl/FileIteratingFirehose.java b/core/src/main/java/org/apache/druid/data/input/impl/FileIteratingFirehose.java index 591e8f5ab4ac..74f1d4c7392d 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/FileIteratingFirehose.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/FileIteratingFirehose.java @@ -23,7 +23,6 @@ import org.apache.druid.data.input.Firehose; import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.InputRowListPlusJson; -import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.parsers.ParseException; import javax.annotation.Nullable; @@ -90,10 +89,10 @@ public InputRowListPlusJson nextRowWithRaw() throws IOException String raw = lineIterator.next(); try { - return InputRowListPlusJson.of(parser.parse(raw), StringUtils.toUtf8(raw)); + return InputRowListPlusJson.of(parser.parse(raw), parser.parseString(raw)); } catch (ParseException e) { - return InputRowListPlusJson.of(StringUtils.toUtf8(raw), e); + return InputRowListPlusJson.of(parser.parseString(raw), e); } } diff --git a/core/src/main/java/org/apache/druid/data/input/impl/FirehoseToInputSourceReaderAdaptor.java b/core/src/main/java/org/apache/druid/data/input/impl/FirehoseToInputSourceReaderAdaptor.java index 2fb3cde38d6f..8dd1ea3b8438 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/FirehoseToInputSourceReaderAdaptor.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/FirehoseToInputSourceReaderAdaptor.java @@ -38,12 +38,12 @@ public class FirehoseToInputSourceReaderAdaptor implements InputSourceReader public FirehoseToInputSourceReaderAdaptor( FirehoseFactory firehoseFactory, - InputRowParser inputRowPlusRaw, + InputRowParser inputRowParser, File temporaryDirectory ) { this.firehoseFactory = firehoseFactory; - this.inputRowParser = inputRowPlusRaw; + this.inputRowParser = inputRowParser; this.temporaryDirectory = temporaryDirectory; } diff --git a/core/src/main/java/org/apache/druid/data/input/impl/InlineInputSource.java b/core/src/main/java/org/apache/druid/data/input/impl/InlineInputSource.java new file mode 100644 index 000000000000..244facec5501 --- /dev/null +++ b/core/src/main/java/org/apache/druid/data/input/impl/InlineInputSource.java @@ -0,0 +1,76 @@ +/* + * 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.druid.data.input.impl; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.data.input.AbstractInputSource; +import org.apache.druid.data.input.InputFormat; +import org.apache.druid.data.input.InputRowSchema; +import org.apache.druid.data.input.InputSourceReader; +import org.apache.druid.java.util.common.StringUtils; + +import javax.annotation.Nullable; +import java.io.File; +import java.util.stream.Stream; + +public class InlineInputSource extends AbstractInputSource +{ + private final String data; + + @JsonCreator + public InlineInputSource(@JsonProperty("data") String data) + { + this.data = data; + } + + @JsonProperty + public String getData() + { + return data; + } + + @Override + public boolean isSplittable() + { + return false; + } + + @Override + public boolean needsFormat() + { + return true; + } + + @Override + protected InputSourceReader formattableReader( + InputRowSchema inputRowSchema, + InputFormat inputFormat, + @Nullable File temporaryDirectory + ) + { + return new InputEntityIteratingReader( + inputRowSchema, + inputFormat, + Stream.of(new ByteEntity(StringUtils.toUtf8(data))), + temporaryDirectory + ); + } +} diff --git a/core/src/main/java/org/apache/druid/data/input/impl/InputEntityIteratingReader.java b/core/src/main/java/org/apache/druid/data/input/impl/InputEntityIteratingReader.java index 385bc5f14591..ec0090e2a284 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/InputEntityIteratingReader.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/InputEntityIteratingReader.java @@ -31,7 +31,6 @@ import java.io.File; import java.io.IOException; -import java.util.Iterator; import java.util.function.Function; import java.util.stream.Stream; @@ -43,19 +42,29 @@ public class InputEntityIteratingReader implements InputSourceReader { private final InputRowSchema inputRowSchema; private final InputFormat inputFormat; - private final Iterator sourceIterator; + private final CloseableIterator sourceIterator; private final File temporaryDirectory; - InputEntityIteratingReader( + public InputEntityIteratingReader( InputRowSchema inputRowSchema, InputFormat inputFormat, Stream sourceStream, File temporaryDirectory ) + { + this(inputRowSchema, inputFormat, CloseableIterators.withEmptyBaggage(sourceStream.iterator()), temporaryDirectory); + } + + public InputEntityIteratingReader( + InputRowSchema inputRowSchema, + InputFormat inputFormat, + CloseableIterator sourceIterator, + File temporaryDirectory + ) { this.inputRowSchema = inputRowSchema; this.inputFormat = inputFormat; - this.sourceIterator = sourceStream.iterator(); + this.sourceIterator = sourceIterator; this.temporaryDirectory = temporaryDirectory; } @@ -91,6 +100,6 @@ public CloseableIterator sample() private CloseableIterator createIterator(Function> rowPopulator) { - return CloseableIterators.withEmptyBaggage(sourceIterator).flatMap(rowPopulator); + return sourceIterator.flatMap(rowPopulator); } } diff --git a/core/src/main/java/org/apache/druid/data/input/impl/JsonReader.java b/core/src/main/java/org/apache/druid/data/input/impl/JsonReader.java index ac2b38c2977b..50617fce4d20 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/JsonReader.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/JsonReader.java @@ -64,10 +64,10 @@ public List parseInputRows(String line) throws IOException, ParseExcep } @Override - public String toJson(String intermediateRow) throws IOException + public Map toMap(String intermediateRow) throws IOException { - final JsonNode document = mapper.readValue(intermediateRow, JsonNode.class); - return DEFAULT_JSON_WRITER.writeValueAsString(document); + //noinspection unchecked + return mapper.readValue(intermediateRow, Map.class); } @Override diff --git a/core/src/main/java/org/apache/druid/data/input/impl/RegexInputFormat.java b/core/src/main/java/org/apache/druid/data/input/impl/RegexInputFormat.java new file mode 100644 index 000000000000..239ee6378bfb --- /dev/null +++ b/core/src/main/java/org/apache/druid/data/input/impl/RegexInputFormat.java @@ -0,0 +1,62 @@ +/* + * 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.druid.data.input.impl; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.data.input.InputEntity; +import org.apache.druid.data.input.InputEntityReader; +import org.apache.druid.data.input.InputFormat; +import org.apache.druid.data.input.InputRowSchema; + +import javax.annotation.Nullable; +import java.io.File; +import java.util.List; + +public class RegexInputFormat implements InputFormat +{ + private final String pattern; + private final String listDelimiter; + private final List columns; + + @JsonCreator + public RegexInputFormat( + @JsonProperty("pattern") String pattern, + @JsonProperty("listDelimiter") @Nullable String listDelimiter, + @JsonProperty("columns") @Nullable List columns + ) + { + this.pattern = pattern; + this.listDelimiter = listDelimiter; + this.columns = columns; + } + + @Override + public boolean isSplittable() + { + return false; + } + + @Override + public InputEntityReader createReader(InputRowSchema inputRowSchema, InputEntity source, File temporaryDirectory) + { + return new RegexReader(inputRowSchema, source, temporaryDirectory, pattern, listDelimiter, columns); + } +} diff --git a/core/src/main/java/org/apache/druid/data/input/impl/RegexReader.java b/core/src/main/java/org/apache/druid/data/input/impl/RegexReader.java new file mode 100644 index 000000000000..c7105c8afd0a --- /dev/null +++ b/core/src/main/java/org/apache/druid/data/input/impl/RegexReader.java @@ -0,0 +1,122 @@ +/* + * 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.druid.data.input.impl; + +import com.google.common.base.Function; +import com.google.common.base.Splitter; +import com.google.common.collect.Iterables; +import org.apache.druid.data.input.InputEntity; +import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.InputRowSchema; +import org.apache.druid.data.input.TextReader; +import org.apache.druid.java.util.common.collect.Utils; +import org.apache.druid.java.util.common.parsers.ParseException; +import org.apache.druid.java.util.common.parsers.ParserUtils; +import org.apache.druid.java.util.common.parsers.Parsers; + +import javax.annotation.Nullable; +import java.io.File; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +public class RegexReader extends TextReader +{ + private final String pattern; + private final Pattern compiled; + private final Function multiValueFunction; + + private List columns; + + RegexReader( + InputRowSchema inputRowSchema, + InputEntity source, + File temporaryDirectory, + String pattern, + @Nullable String listDelimiter, + @Nullable List columns + ) + { + super(inputRowSchema, source, temporaryDirectory); + this.pattern = pattern; + this.compiled = Pattern.compile(pattern); + final String finalListDelimeter = listDelimiter == null ? Parsers.DEFAULT_LIST_DELIMITER : listDelimiter; + this.multiValueFunction = ParserUtils.getMultiValueFunction(finalListDelimeter, Splitter.on(finalListDelimeter)); + this.columns = columns; + } + + @Override + public List parseInputRows(String intermediateRow) throws ParseException + { + return Collections.singletonList(MapInputRowParser.parse(getInputRowSchema(), parseLine(intermediateRow))); + } + + @Override + protected Map toMap(String intermediateRow) + { + return parseLine(intermediateRow); + } + + private Map parseLine(String line) + { + try { + final Matcher matcher = compiled.matcher(line); + + if (!matcher.matches()) { + throw new ParseException("Incorrect Regex: %s . No match found.", pattern); + } + + final List values = new ArrayList<>(); + for (int i = 1; i <= matcher.groupCount(); i++) { + values.add(matcher.group(i)); + } + + if (columns == null) { + columns = ParserUtils.generateFieldNames(matcher.groupCount()); + } + + return Utils.zipMapPartial(columns, Iterables.transform(values, multiValueFunction)); + } + catch (Exception e) { + throw new ParseException(e, "Unable to parse row [%s]", line); + } + } + + @Override + public int getNumHeaderLinesToSkip() + { + return 0; + } + + @Override + public boolean needsToProcessHeaderLine() + { + return false; + } + + @Override + public void processHeaderLine(String line) + { + // do nothing + } +} diff --git a/core/src/main/java/org/apache/druid/data/input/impl/StringInputRowParser.java b/core/src/main/java/org/apache/druid/data/input/impl/StringInputRowParser.java index 07761c0898db..b83bd868b9e3 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/StringInputRowParser.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/StringInputRowParser.java @@ -99,7 +99,7 @@ public StringInputRowParser withParseSpec(ParseSpec parseSpec) return new StringInputRowParser(parseSpec, getEncoding()); } - private Map buildStringKeyMap(ByteBuffer input) + public Map buildStringKeyMap(ByteBuffer input) { int payloadSize = input.remaining(); @@ -149,7 +149,7 @@ public InputRow parse(@Nullable String input) } @Nullable - private Map parseString(@Nullable String inputString) + public Map parseString(@Nullable String inputString) { initializeParser(); return parser.parseToMap(inputString); diff --git a/core/src/main/java/org/apache/druid/data/input/impl/TimedShutoffInputSource.java b/core/src/main/java/org/apache/druid/data/input/impl/TimedShutoffInputSource.java new file mode 100644 index 000000000000..bc72580b8668 --- /dev/null +++ b/core/src/main/java/org/apache/druid/data/input/impl/TimedShutoffInputSource.java @@ -0,0 +1,82 @@ +/* + * 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.druid.data.input.impl; + +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.data.input.InputFormat; +import org.apache.druid.data.input.InputRowSchema; +import org.apache.druid.data.input.InputSource; +import org.apache.druid.data.input.InputSourceReader; +import org.joda.time.DateTime; + +import javax.annotation.Nullable; +import java.io.File; + +public class TimedShutoffInputSource implements InputSource +{ + private final InputSource delegate; + private final DateTime shutoffTime; + + public TimedShutoffInputSource( + @JsonProperty("delegate") InputSource delegate, + @JsonProperty("shutoffTime") DateTime shutoffTime + ) + { + this.delegate = delegate; + this.shutoffTime = shutoffTime; + } + + @JsonProperty + public InputSource getDelegate() + { + return delegate; + } + + @JsonProperty + public DateTime getShutoffTime() + { + return shutoffTime; + } + + @Override + public boolean isSplittable() + { + return delegate.isSplittable(); + } + + @Override + public boolean needsFormat() + { + return delegate.needsFormat(); + } + + @Override + public InputSourceReader reader( + InputRowSchema inputRowSchema, + @Nullable InputFormat inputFormat, + File temporaryDirectory + ) + { + return new TimedShutoffInputSourceReader( + delegate.reader(inputRowSchema, inputFormat, temporaryDirectory), + shutoffTime + ); + } +} diff --git a/core/src/main/java/org/apache/druid/data/input/impl/TimedShutoffInputSourceReader.java b/core/src/main/java/org/apache/druid/data/input/impl/TimedShutoffInputSourceReader.java new file mode 100644 index 000000000000..58a3ace1e542 --- /dev/null +++ b/core/src/main/java/org/apache/druid/data/input/impl/TimedShutoffInputSourceReader.java @@ -0,0 +1,116 @@ +/* + * 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.druid.data.input.impl; + +import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.InputRowListPlusJson; +import org.apache.druid.data.input.InputSourceReader; +import org.apache.druid.java.util.common.concurrent.Execs; +import org.apache.druid.java.util.common.io.Closer; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.java.util.common.parsers.CloseableIterator; +import org.joda.time.DateTime; + +import java.io.IOException; +import java.util.NoSuchElementException; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; + +public class TimedShutoffInputSourceReader implements InputSourceReader +{ + private static final Logger LOG = new Logger(TimedShutoffInputSourceReader.class); + + private final InputSourceReader delegate; + private final DateTime shutoffTime; + + public TimedShutoffInputSourceReader(InputSourceReader delegate, DateTime shutoffTime) + { + this.delegate = delegate; + this.shutoffTime = shutoffTime; + } + + @Override + public CloseableIterator read() throws IOException + { + final ScheduledExecutorService shutdownExec = Execs.scheduledSingleThreaded("timed-shutoff-reader-%d"); + final CloseableIterator delegateIterator = delegate.read(); + return decorateShutdownTimeout(shutdownExec, delegateIterator); + } + + @Override + public CloseableIterator sample() throws IOException + { + final ScheduledExecutorService shutdownExec = Execs.scheduledSingleThreaded("timed-shutoff-reader-%d"); + final CloseableIterator delegateIterator = delegate.sample(); + return decorateShutdownTimeout(shutdownExec, delegateIterator); + } + + private CloseableIterator decorateShutdownTimeout( + ScheduledExecutorService exec, + CloseableIterator delegateIterator + ) + { + final Closer closer = Closer.create(); + closer.register(delegateIterator); + closer.register(exec::shutdownNow); + final CloseableIterator wrappingIterator = new CloseableIterator() + { + volatile boolean closed; + + @Override + public boolean hasNext() + { + return !closed && delegateIterator.hasNext(); + } + + @Override + public T next() + { + if (!hasNext()) { + throw new NoSuchElementException(); + } + return delegateIterator.next(); + } + + @Override + public void close() throws IOException + { + closed = true; + closer.close(); + } + }; + exec.schedule( + () -> { + LOG.info("Closing delegate inputSource."); + + try { + wrappingIterator.close(); + } + catch (IOException e) { + LOG.warn(e, "Failed to close delegate inputSource, ignoring."); + } + }, + shutoffTime.getMillis() - System.currentTimeMillis(), + TimeUnit.MILLISECONDS + ); + + return wrappingIterator; + } +} diff --git a/core/src/main/java/org/apache/druid/java/util/common/CloseableIterators.java b/core/src/main/java/org/apache/druid/java/util/common/CloseableIterators.java index afca4f984cac..22a9c988bd0e 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/CloseableIterators.java +++ b/core/src/main/java/org/apache/druid/java/util/common/CloseableIterators.java @@ -65,6 +65,9 @@ public static CloseableIterator wrap(Iterator innerIterator, @Nullable @Override public boolean hasNext() { + if (closed) { + return false; + } return innerIterator.hasNext(); } diff --git a/core/src/main/java/org/apache/druid/java/util/common/parsers/CloseableIterator.java b/core/src/main/java/org/apache/druid/java/util/common/parsers/CloseableIterator.java index 4cf5addb6428..3753b3fda4d1 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/parsers/CloseableIterator.java +++ b/core/src/main/java/org/apache/druid/java/util/common/parsers/CloseableIterator.java @@ -19,6 +19,7 @@ package org.apache.druid.java.util.common.parsers; +import javax.annotation.Nullable; import java.io.Closeable; import java.io.IOException; import java.io.UncheckedIOException; @@ -65,12 +66,38 @@ default CloseableIterator flatMap(Function> funct return new CloseableIterator() { - CloseableIterator iterator = null; + CloseableIterator iterator = findNextIeteratorIfNecessary(); + + @Nullable + private CloseableIterator findNextIeteratorIfNecessary() + { + while ((iterator == null || !iterator.hasNext()) && delegate.hasNext()) { + if (iterator != null) { + try { + iterator.close(); + } + catch (IOException e) { + throw new UncheckedIOException(e); + } + } + try { + iterator = function.apply(delegate.next()); + if (iterator.hasNext()) { + return iterator; + } + } + catch (Exception e) { + iterator = new ExceptionThrowingIterator<>(e); + return iterator; + } + } + return null; + } @Override public boolean hasNext() { - return (iterator != null && iterator.hasNext()) || delegate.hasNext(); + return iterator != null && iterator.hasNext(); } @Override @@ -79,18 +106,12 @@ public R next() if (!hasNext()) { throw new NoSuchElementException(); } - if (iterator == null || !iterator.hasNext()) { - if (iterator != null) { - try { - iterator.close(); - } - catch (IOException e) { - throw new UncheckedIOException(e); - } - } - iterator = function.apply(delegate.next()); + try { + return iterator.next(); + } + finally { + findNextIeteratorIfNecessary(); } - return iterator.next(); } @Override @@ -100,4 +121,39 @@ public void close() throws IOException } }; } + + class ExceptionThrowingIterator implements CloseableIterator + { + private final Exception exception; + + private boolean thrown = false; + + private ExceptionThrowingIterator(Exception exception) + { + this.exception = exception; + } + + @Override + public boolean hasNext() + { + return !thrown; + } + + @Override + public T next() + { + thrown = true; + if (exception instanceof RuntimeException) { + throw (RuntimeException) exception; + } else { + throw new RuntimeException(exception); + } + } + + @Override + public void close() throws IOException + { + // do nothing + } + } } diff --git a/core/src/test/java/org/apache/druid/data/input/impl/TimedShutoffInputSourceTest.java b/core/src/test/java/org/apache/druid/data/input/impl/TimedShutoffInputSourceTest.java new file mode 100644 index 000000000000..205f85613674 --- /dev/null +++ b/core/src/test/java/org/apache/druid/data/input/impl/TimedShutoffInputSourceTest.java @@ -0,0 +1,57 @@ +/* + * 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.druid.data.input.impl; + +import com.google.common.collect.ImmutableList; +import org.apache.druid.data.input.InputFormat; +import org.apache.druid.data.input.InputRowListPlusJson; +import org.apache.druid.data.input.InputRowSchema; +import org.apache.druid.data.input.InputSource; +import org.apache.druid.data.input.InputSourceReader; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.parsers.CloseableIterator; +import org.junit.Assert; +import org.junit.Test; + +import java.io.IOException; +import java.util.Collections; + +public class TimedShutoffInputSourceTest +{ + @Test + public void testTimeoutShutoff() throws IOException, InterruptedException + { + final int timeoutMs = 2000; + final InputSource inputSource = new TimedShutoffInputSource( + new InlineInputSource("this,is,test\nthis,data,has\n3,rows,\n"), + DateTimes.nowUtc().plusMillis(timeoutMs) + ); + final InputFormat inputFormat = new CsvInputFormat(ImmutableList.of("col1", "col2", "col3"), null, false, 0); + final InputSourceReader reader = inputSource.reader( + new InputRowSchema(new TimestampSpec(null, null, null), new DimensionsSpec(null), Collections.emptyList()), + inputFormat, + null + ); + try (CloseableIterator iterator = reader.sample()) { + Thread.sleep(timeoutMs + 1000); + Assert.assertFalse(iterator.hasNext()); + } + } +} diff --git a/core/src/test/java/org/apache/druid/java/util/common/parsers/CloseableIteratorTest.java b/core/src/test/java/org/apache/druid/java/util/common/parsers/CloseableIteratorTest.java new file mode 100644 index 000000000000..5434e2de3909 --- /dev/null +++ b/core/src/test/java/org/apache/druid/java/util/common/parsers/CloseableIteratorTest.java @@ -0,0 +1,102 @@ +/* + * 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.druid.java.util.common.parsers; + +import org.apache.druid.java.util.common.CloseableIterators; +import org.junit.Assert; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.NoSuchElementException; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +public class CloseableIteratorTest +{ + @Test + public void testMap() + { + final CloseableIterator> actual = generateTestIterator(8) + .map(list -> { + final List newList = new ArrayList<>(list.size()); + for (Integer i : list) { + newList.add(i * 10); + } + return newList; + }); + final Iterator> expected = IntStream.range(0, 8) + .mapToObj(i -> IntStream.range(0, i).map(j -> j * 10).boxed().collect(Collectors.toList())) + .iterator(); + while (expected.hasNext() && actual.hasNext()) { + Assert.assertEquals(expected.next(), actual.next()); + } + Assert.assertFalse(actual.hasNext()); + Assert.assertFalse(expected.hasNext()); + } + + @Test + public void testFlatMap() + { + final CloseableIterator actual = generateTestIterator(8) + .flatMap(list -> CloseableIterators.withEmptyBaggage(list.iterator())); + final Iterator expected = IntStream + .range(0, 8) + .flatMap(i -> IntStream.range(0, i)) + .iterator(); + while (expected.hasNext() && actual.hasNext()) { + Assert.assertEquals(expected.next(), actual.next()); + } + Assert.assertFalse(actual.hasNext()); + Assert.assertFalse(expected.hasNext()); + } + + private static CloseableIterator> generateTestIterator(int numIterates) + { + return new CloseableIterator>() + { + private int cnt = 0; + + @Override + public boolean hasNext() + { + return cnt < numIterates; + } + + @Override + public List next() + { + if (!hasNext()) { + throw new NoSuchElementException(); + } + final List integers = IntStream.range(0, cnt).boxed().collect(Collectors.toList()); + cnt++; + return integers; + } + + @Override + public void close() + { + // do nothing + } + }; + } +} diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java index 449878715908..35bc5675788c 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/IncrementalPublishingKafkaIndexTaskRunner.java @@ -66,9 +66,9 @@ public class IncrementalPublishingKafkaIndexTaskRunner extends SeekableStreamInd private static final EmittingLogger log = new EmittingLogger(IncrementalPublishingKafkaIndexTaskRunner.class); private final KafkaIndexTask task; - public IncrementalPublishingKafkaIndexTaskRunner( + IncrementalPublishingKafkaIndexTaskRunner( KafkaIndexTask task, - InputRowParser parser, + @Nullable InputRowParser parser, AuthorizerMapper authorizerMapper, Optional chatHandlerProvider, CircularBuffer savedParseExceptions, diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java index 57f7493b702b..9f7f9977efb2 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java @@ -138,7 +138,7 @@ protected SeekableStreamIndexTaskRunner createTaskRunner() //noinspection unchecked return new IncrementalPublishingKafkaIndexTaskRunner( this, - Preconditions.checkNotNull(dataSchema.getParser(), "inputRowParser"), + dataSchema.getParser(), authorizerMapper, chatHandlerProvider, savedParseExceptions, diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskIOConfig.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskIOConfig.java index 37366f3cf2aa..ece3fa5ba82b 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskIOConfig.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskIOConfig.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; +import org.apache.druid.data.input.InputFormat; import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorIOConfig; import org.apache.druid.indexing.seekablestream.SeekableStreamEndSequenceNumbers; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskIOConfig; @@ -54,7 +55,8 @@ public KafkaIndexTaskIOConfig( @JsonProperty("pollTimeout") Long pollTimeout, @JsonProperty("useTransaction") Boolean useTransaction, @JsonProperty("minimumMessageTime") DateTime minimumMessageTime, - @JsonProperty("maximumMessageTime") DateTime maximumMessageTime + @JsonProperty("maximumMessageTime") DateTime maximumMessageTime, + @JsonProperty("inputFormat") @Nullable InputFormat inputFormat ) { super( @@ -66,7 +68,8 @@ public KafkaIndexTaskIOConfig( endSequenceNumbers == null ? endPartitions : endSequenceNumbers, useTransaction, minimumMessageTime, - maximumMessageTime + maximumMessageTime, + inputFormat ); this.consumerProperties = Preconditions.checkNotNull(consumerProperties, "consumerProperties"); @@ -93,7 +96,8 @@ public KafkaIndexTaskIOConfig( Long pollTimeout, Boolean useTransaction, DateTime minimumMessageTime, - DateTime maximumMessageTime + DateTime maximumMessageTime, + InputFormat inputFormat ) { this( @@ -107,7 +111,8 @@ public KafkaIndexTaskIOConfig( pollTimeout, useTransaction, minimumMessageTime, - maximumMessageTime + maximumMessageTime, + inputFormat ); } diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaSamplerSpec.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaSamplerSpec.java index ae98c229e5b4..3981728cfc7d 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaSamplerSpec.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaSamplerSpec.java @@ -23,15 +23,13 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.druid.data.input.Firehose; -import org.apache.druid.data.input.impl.InputRowParser; import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorIOConfig; import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorSpec; -import org.apache.druid.indexing.overlord.sampler.FirehoseSampler; +import org.apache.druid.indexing.overlord.sampler.InputSourceSampler; import org.apache.druid.indexing.overlord.sampler.SamplerConfig; import org.apache.druid.indexing.seekablestream.SeekableStreamSamplerSpec; -import org.apache.druid.indexing.seekablestream.common.RecordSupplier; +import javax.annotation.Nullable; import java.util.HashMap; import java.util.Map; @@ -42,47 +40,33 @@ public class KafkaSamplerSpec extends SeekableStreamSamplerSpec @JsonCreator public KafkaSamplerSpec( @JsonProperty("spec") final KafkaSupervisorSpec ingestionSpec, - @JsonProperty("samplerConfig") final SamplerConfig samplerConfig, - @JacksonInject FirehoseSampler firehoseSampler, + @JsonProperty("samplerConfig") @Nullable final SamplerConfig samplerConfig, + @JacksonInject InputSourceSampler inputSourceSampler, @JacksonInject ObjectMapper objectMapper ) { - super(ingestionSpec, samplerConfig, firehoseSampler); + super(ingestionSpec, samplerConfig, inputSourceSampler); this.objectMapper = objectMapper; } @Override - protected Firehose getFirehose(InputRowParser parser) + protected KafkaRecordSupplier createRecordSupplier() { - return new KafkaSamplerFirehose(parser); - } - - protected class KafkaSamplerFirehose extends SeekableStreamSamplerFirehose - { - private KafkaSamplerFirehose(InputRowParser parser) - { - super(parser); - } - - @Override - protected RecordSupplier getRecordSupplier() - { - ClassLoader currCtxCl = Thread.currentThread().getContextClassLoader(); - try { - Thread.currentThread().setContextClassLoader(getClass().getClassLoader()); + ClassLoader currCtxCl = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(getClass().getClassLoader()); - final Map props = new HashMap<>(((KafkaSupervisorIOConfig) ioConfig).getConsumerProperties()); + final Map props = new HashMap<>(((KafkaSupervisorIOConfig) ioConfig).getConsumerProperties()); - props.put("enable.auto.commit", "false"); - props.put("auto.offset.reset", "none"); - props.put("request.timeout.ms", Integer.toString(samplerConfig.getTimeoutMs())); + props.put("enable.auto.commit", "false"); + props.put("auto.offset.reset", "none"); + props.put("request.timeout.ms", Integer.toString(samplerConfig.getTimeoutMs())); - return new KafkaRecordSupplier(props, objectMapper); - } - finally { - Thread.currentThread().setContextClassLoader(currCtxCl); - } + return new KafkaRecordSupplier(props, objectMapper); + } + finally { + Thread.currentThread().setContextClassLoader(currCtxCl); } } } diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java index f747c24b3165..98ba1696e021 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisor.java @@ -226,7 +226,10 @@ protected SeekableStreamIndexTaskIOConfig createTaskIoConfig( kafkaIoConfig.getPollTimeout(), true, minimumMessageTime, - maximumMessageTime + maximumMessageTime, + ioConfig.getInputFormat( + spec.getDataSchema().getParser() == null ? null : spec.getDataSchema().getParser().getParseSpec() + ) ); } diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorIOConfig.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorIOConfig.java index af377a7f39ca..a1360b5c63f3 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorIOConfig.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorIOConfig.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; +import org.apache.druid.data.input.InputFormat; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorIOConfig; import org.apache.druid.java.util.common.StringUtils; import org.joda.time.DateTime; @@ -44,6 +45,7 @@ public class KafkaSupervisorIOConfig extends SeekableStreamSupervisorIOConfig @JsonCreator public KafkaSupervisorIOConfig( @JsonProperty("topic") String topic, + @JsonProperty("inputFormat") InputFormat inputFormat, @JsonProperty("replicas") Integer replicas, @JsonProperty("taskCount") Integer taskCount, @JsonProperty("taskDuration") Period taskDuration, @@ -60,6 +62,7 @@ public KafkaSupervisorIOConfig( { super( Preconditions.checkNotNull(topic, "topic"), + inputFormat, replicas, taskCount, taskDuration, diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIOConfigTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIOConfigTest.java index ac93c1e5b6ac..4de3c06e78ae 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIOConfigTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIOConfigTest.java @@ -316,7 +316,8 @@ public void testDeserializeToOldIoConfig() throws IOException 100L, true, DateTimes.nowUtc(), - DateTimes.nowUtc() + DateTimes.nowUtc(), + null ); final byte[] json = mapper.writeValueAsBytes(currentConfig); diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java index c835bb950385..e8bde11469c6 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java @@ -341,6 +341,54 @@ public void testRunAfterDataInserted() throws Exception KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, true, null, + null, + INPUT_FORMAT + ) + ); + + final ListenableFuture future = runTask(task); + + // Wait for task to exit + Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); + + // Check metrics + Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed()); + Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); + Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); + + // Check published metadata and segments in deep storage + assertEqualsExceptVersion( + ImmutableList.of( + sdd("2010/P1D", 0, ImmutableList.of("c")), + sdd("2011/P1D", 0, ImmutableList.of("d", "e")) + ), + publishedDescriptors() + ); + Assert.assertEquals( + new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L))), + metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource()) + ); + } + + @Test(timeout = 60_000L) + public void testRunAfterDataInsertedWithLegacyParser() throws Exception + { + // Insert data + insertData(); + + final KafkaIndexTask task = createTask( + null, + OLD_DATA_SCHEMA, + new KafkaIndexTaskIOConfig( + 0, + "sequence0", + new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 2L), ImmutableSet.of()), + new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L)), + kafkaServer.consumerProperties(), + KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, + true, + null, + null, null ) ); @@ -365,7 +413,7 @@ public void testRunAfterDataInserted() throws Exception ); Assert.assertEquals( new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L))), - metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource()) ); } @@ -383,7 +431,8 @@ public void testRunBeforeDataInserted() throws Exception KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, true, null, - null + null, + INPUT_FORMAT ) ); @@ -415,7 +464,7 @@ public void testRunBeforeDataInserted() throws Exception ); Assert.assertEquals( new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L))), - metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource()) ); } @@ -435,7 +484,8 @@ public void testRunAfterDataInsertedLiveReport() throws Exception KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, true, null, - null + null, + INPUT_FORMAT ) ); final ListenableFuture future = runTask(task); @@ -512,7 +562,8 @@ public void testIncrementalHandOff() throws Exception KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, true, null, - null + null, + INPUT_FORMAT ) ); final ListenableFuture future = runTask(task); @@ -530,7 +581,7 @@ public void testIncrementalHandOff() throws Exception Assert.assertTrue( checkpointRequestsHash.contains( Objects.hash( - DATA_SCHEMA.getDataSource(), + NEW_DATA_SCHEMA.getDataSource(), 0, new KafkaDataSourceMetadata(startPartitions) ) @@ -559,7 +610,7 @@ public void testIncrementalHandOff() throws Exception new KafkaDataSourceMetadata( new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 10L, 1, 2L)) ), - metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource()) ); } @@ -615,7 +666,8 @@ public void testIncrementalHandOffMaxTotalRows() throws Exception KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, true, null, - null + null, + INPUT_FORMAT ) ); final ListenableFuture future = runTask(task); @@ -652,7 +704,7 @@ public void testIncrementalHandOffMaxTotalRows() throws Exception Assert.assertTrue( checkpointRequestsHash.contains( Objects.hash( - DATA_SCHEMA.getDataSource(), + NEW_DATA_SCHEMA.getDataSource(), 0, new KafkaDataSourceMetadata(startPartitions) ) @@ -661,7 +713,7 @@ public void testIncrementalHandOffMaxTotalRows() throws Exception Assert.assertTrue( checkpointRequestsHash.contains( Objects.hash( - DATA_SCHEMA.getDataSource(), + NEW_DATA_SCHEMA.getDataSource(), 0, new KafkaDataSourceMetadata( new SeekableStreamStartSequenceNumbers<>(topic, currentOffsets, ImmutableSet.of()) @@ -692,14 +744,14 @@ public void testIncrementalHandOffMaxTotalRows() throws Exception new KafkaDataSourceMetadata( new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 10L, 1, 2L)) ), - metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource()) ); Assert.assertEquals( new KafkaDataSourceMetadata( new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 10L, 1, 2L)) ), - metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource()) ); } @@ -741,7 +793,8 @@ public void testTimeBasedIncrementalHandOff() throws Exception KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, true, null, - null + null, + INPUT_FORMAT ) ); final ListenableFuture future = runTask(task); @@ -759,7 +812,7 @@ public void testTimeBasedIncrementalHandOff() throws Exception Assert.assertTrue( checkpointRequestsHash.contains( Objects.hash( - DATA_SCHEMA.getDataSource(), + NEW_DATA_SCHEMA.getDataSource(), 0, new KafkaDataSourceMetadata(startPartitions) ) @@ -783,7 +836,7 @@ public void testTimeBasedIncrementalHandOff() throws Exception new KafkaDataSourceMetadata( new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 2L, 1, 0L)) ), - metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource()) ); } @@ -821,7 +874,8 @@ public void testIncrementalHandOffReadsThroughEndOffsets() throws Exception KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, true, null, - null + null, + INPUT_FORMAT ) ); final KafkaIndexTask staleReplica = createTask( @@ -835,7 +889,8 @@ public void testIncrementalHandOffReadsThroughEndOffsets() throws Exception KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, true, null, - null + null, + INPUT_FORMAT ) ); @@ -902,7 +957,8 @@ public void testRunWithMinimumMessageTime() throws Exception KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, true, DateTimes.of("2010"), - null + null, + INPUT_FORMAT ) ); @@ -934,7 +990,7 @@ public void testRunWithMinimumMessageTime() throws Exception ); Assert.assertEquals( new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L))), - metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource()) ); } @@ -952,7 +1008,8 @@ public void testRunWithMaximumMessageTime() throws Exception KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, true, null, - DateTimes.of("2010") + DateTimes.of("2010"), + INPUT_FORMAT ) ); @@ -985,7 +1042,7 @@ public void testRunWithMaximumMessageTime() throws Exception ); Assert.assertEquals( new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L))), - metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource()) ); } @@ -994,7 +1051,7 @@ public void testRunWithTransformSpec() throws Exception { final KafkaIndexTask task = createTask( null, - DATA_SCHEMA.withTransformSpec( + NEW_DATA_SCHEMA.withTransformSpec( new TransformSpec( new SelectorDimFilter("dim1", "b", null), ImmutableList.of( @@ -1011,7 +1068,8 @@ public void testRunWithTransformSpec() throws Exception KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, true, null, - null + null, + INPUT_FORMAT ) ); @@ -1038,7 +1096,7 @@ public void testRunWithTransformSpec() throws Exception assertEqualsExceptVersion(ImmutableList.of(sdd("2009/P1D", 0)), publishedDescriptors); Assert.assertEquals( new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L))), - metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource()) ); // Check segments in deep storage @@ -1063,7 +1121,8 @@ public void testRunOnNothing() throws Exception KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, true, null, - null + null, + INPUT_FORMAT ) ); @@ -1100,7 +1159,8 @@ public void testHandoffConditionTimeoutWhenHandoffOccurs() throws Exception KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, true, null, - null + null, + INPUT_FORMAT ) ); @@ -1124,7 +1184,7 @@ public void testHandoffConditionTimeoutWhenHandoffOccurs() throws Exception ); Assert.assertEquals( new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L))), - metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource()) ); } @@ -1148,7 +1208,8 @@ public void testHandoffConditionTimeoutWhenHandoffDoesNotOccur() throws Exceptio KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, true, null, - null + null, + INPUT_FORMAT ) ); @@ -1174,7 +1235,7 @@ public void testHandoffConditionTimeoutWhenHandoffDoesNotOccur() throws Exceptio new KafkaDataSourceMetadata( new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L)) ), - metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource()) ); } @@ -1201,7 +1262,8 @@ public void testReportParseExceptions() throws Exception KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, true, null, - null + null, + INPUT_FORMAT ) ); @@ -1217,7 +1279,7 @@ public void testReportParseExceptions() throws Exception // Check published metadata Assert.assertEquals(ImmutableList.of(), publishedDescriptors()); - Assert.assertNull(metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())); + Assert.assertNull(metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource())); } @Test(timeout = 60_000L) @@ -1241,7 +1303,8 @@ public void testMultipleParseExceptionsSuccess() throws Exception KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, true, null, - null + null, + INPUT_FORMAT ) ); @@ -1266,7 +1329,7 @@ public void testMultipleParseExceptionsSuccess() throws Exception ); Assert.assertEquals( new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 13L))), - metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource()) ); IngestionStatsAndErrorsTaskReportData reportData = getTaskReportData(); @@ -1318,7 +1381,8 @@ public void testMultipleParseExceptionsFailure() throws Exception KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, true, null, - null + null, + INPUT_FORMAT ) ); @@ -1338,7 +1402,7 @@ public void testMultipleParseExceptionsFailure() throws Exception // Check published metadata Assert.assertEquals(ImmutableList.of(), publishedDescriptors()); - Assert.assertNull(metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())); + Assert.assertNull(metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource())); IngestionStatsAndErrorsTaskReportData reportData = getTaskReportData(); @@ -1378,7 +1442,8 @@ public void testRunReplicas() throws Exception KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, true, null, - null + null, + INPUT_FORMAT ) ); final KafkaIndexTask task2 = createTask( @@ -1392,7 +1457,8 @@ public void testRunReplicas() throws Exception KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, true, null, - null + null, + INPUT_FORMAT ) ); @@ -1424,7 +1490,7 @@ public void testRunReplicas() throws Exception ); Assert.assertEquals( new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L))), - metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource()) ); } @@ -1442,7 +1508,8 @@ public void testRunConflicting() throws Exception KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, true, null, - null + null, + INPUT_FORMAT ) ); final KafkaIndexTask task2 = createTask( @@ -1456,7 +1523,8 @@ public void testRunConflicting() throws Exception KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, true, null, - null + null, + INPUT_FORMAT ) ); @@ -1490,7 +1558,7 @@ public void testRunConflicting() throws Exception ); Assert.assertEquals( new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L))), - metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource()) ); } @@ -1508,7 +1576,8 @@ public void testRunConflictingWithoutTransactions() throws Exception KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, false, null, - null + null, + INPUT_FORMAT ) ); final KafkaIndexTask task2 = createTask( @@ -1522,7 +1591,8 @@ public void testRunConflictingWithoutTransactions() throws Exception KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, false, null, - null + null, + INPUT_FORMAT ) ); @@ -1537,7 +1607,7 @@ public void testRunConflictingWithoutTransactions() throws Exception SegmentDescriptorAndExpectedDim1Values desc1 = sdd("2010/P1D", 0, ImmutableList.of("c")); SegmentDescriptorAndExpectedDim1Values desc2 = sdd("2011/P1D", 0, ImmutableList.of("d", "e")); assertEqualsExceptVersion(ImmutableList.of(desc1, desc2), publishedDescriptors()); - Assert.assertNull(metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())); + Assert.assertNull(metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource())); // Run second task final ListenableFuture future2 = runTask(task2); @@ -1555,7 +1625,7 @@ public void testRunConflictingWithoutTransactions() throws Exception SegmentDescriptorAndExpectedDim1Values desc3 = sdd("2011/P1D", 1, ImmutableList.of("d", "e")); SegmentDescriptorAndExpectedDim1Values desc4 = sdd("2013/P1D", 0, ImmutableList.of("f")); assertEqualsExceptVersion(ImmutableList.of(desc1, desc2, desc3, desc4), publishedDescriptors()); - Assert.assertNull(metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())); + Assert.assertNull(metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource())); } @Test(timeout = 60_000L) @@ -1572,7 +1642,8 @@ public void testRunOneTaskTwoPartitions() throws Exception KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, true, null, - null + null, + INPUT_FORMAT ) ); @@ -1601,7 +1672,7 @@ public void testRunOneTaskTwoPartitions() throws Exception new KafkaDataSourceMetadata( new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L, 1, 2L)) ), - metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource()) ); } @@ -1619,7 +1690,8 @@ public void testRunTwoTasksTwoPartitions() throws Exception KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, true, null, - null + null, + INPUT_FORMAT ) ); final KafkaIndexTask task2 = createTask( @@ -1633,7 +1705,8 @@ public void testRunTwoTasksTwoPartitions() throws Exception KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, true, null, - null + null, + INPUT_FORMAT ) ); @@ -1668,7 +1741,7 @@ public void testRunTwoTasksTwoPartitions() throws Exception new KafkaDataSourceMetadata( new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L, 1, 1L)) ), - metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource()) ); } @@ -1686,7 +1759,8 @@ public void testRestore() throws Exception KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, true, null, - null + null, + INPUT_FORMAT ) ); @@ -1726,7 +1800,8 @@ public void testRestore() throws Exception KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, true, null, - null + null, + INPUT_FORMAT ) ); @@ -1764,7 +1839,7 @@ public void testRestore() throws Exception ); Assert.assertEquals( new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 6L))), - metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource()) ); } @@ -1787,7 +1862,8 @@ public void testRestoreAfterPersistingSequences() throws Exception KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, true, null, - null + null, + INPUT_FORMAT ) ); @@ -1835,7 +1911,8 @@ public void testRestoreAfterPersistingSequences() throws Exception KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, true, null, - null + null, + INPUT_FORMAT ) ); @@ -1878,7 +1955,7 @@ public void testRestoreAfterPersistingSequences() throws Exception ); Assert.assertEquals( new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 10L))), - metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource()) ); } @@ -1896,7 +1973,8 @@ public void testRunWithPauseAndResume() throws Exception KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, true, null, - null + null, + INPUT_FORMAT ) ); @@ -1967,7 +2045,7 @@ public void testRunWithPauseAndResume() throws Exception ); Assert.assertEquals( new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 6L))), - metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource()) ); } @@ -1985,7 +2063,8 @@ public void testRunWithOffsetOutOfRangeExceptionAndPause() throws Exception KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, true, null, - null + null, + INPUT_FORMAT ) ); @@ -2020,7 +2099,8 @@ public void testRunWithOffsetOutOfRangeExceptionAndNextOffsetGreaterThanLeastAva KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, true, null, - null + null, + INPUT_FORMAT ) ); @@ -2065,7 +2145,8 @@ public void testRunContextSequenceAheadOfStartingOffsets() throws Exception KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, true, null, - null + null, + INPUT_FORMAT ), context ); @@ -2090,7 +2171,7 @@ public void testRunContextSequenceAheadOfStartingOffsets() throws Exception ); Assert.assertEquals( new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L))), - metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource()) ); } @@ -2111,7 +2192,8 @@ public void testRunWithDuplicateRequest() throws Exception KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, true, null, - null + null, + INPUT_FORMAT ) ); @@ -2146,7 +2228,8 @@ public void testRunTransactionModeRollback() throws Exception KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, true, null, - null + null, + INPUT_FORMAT ) ); @@ -2233,7 +2316,7 @@ public void testRunTransactionModeRollback() throws Exception ); Assert.assertEquals( new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 13L))), - metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource()) ); } @@ -2271,7 +2354,8 @@ public void testCanStartFromLaterThanEarliestOffset() throws Exception KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, true, null, - null + null, + INPUT_FORMAT ) ); final ListenableFuture future = runTask(task); @@ -2285,7 +2369,7 @@ public void testSerde() throws Exception final KafkaIndexTask task = createTask( "taskid", - DATA_SCHEMA.withTransformSpec( + NEW_DATA_SCHEMA.withTransformSpec( new TransformSpec( null, ImmutableList.of(new ExpressionTransform("beep", "nofunc()", ExprMacroTable.nil())) @@ -2300,7 +2384,8 @@ public void testSerde() throws Exception KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, true, null, - null + null, + INPUT_FORMAT ) ); @@ -2311,7 +2396,7 @@ public void testSerde() throws Exception private List scanData(final Task task, QuerySegmentSpec spec) { ScanQuery query = new Druids.ScanQueryBuilder().dataSource( - DATA_SCHEMA.getDataSource()).intervals(spec).build(); + NEW_DATA_SCHEMA.getDataSource()).intervals(spec).build(); return task.getQueryRunner(query).run(QueryPlus.wrap(query)).toList(); } @@ -2332,7 +2417,7 @@ private KafkaIndexTask createTask( final KafkaIndexTaskIOConfig ioConfig ) throws JsonProcessingException { - return createTask(taskId, DATA_SCHEMA, ioConfig); + return createTask(taskId, NEW_DATA_SCHEMA, ioConfig); } private KafkaIndexTask createTask( @@ -2341,7 +2426,7 @@ private KafkaIndexTask createTask( final Map context ) throws JsonProcessingException { - return createTask(taskId, DATA_SCHEMA, ioConfig, context); + return createTask(taskId, NEW_DATA_SCHEMA, ioConfig, context); } private KafkaIndexTask createTask( @@ -2411,10 +2496,12 @@ private static DataSchema cloneDataSchema(final DataSchema dataSchema) { return new DataSchema( dataSchema.getDataSource(), - dataSchema.getParserMap(), + dataSchema.getTimestampSpec(), + dataSchema.getDimensionsSpec(), dataSchema.getAggregators(), dataSchema.getGranularitySpec(), dataSchema.getTransformSpec(), + dataSchema.getParserMap(), OBJECT_MAPPER ); } diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaSamplerSpecTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaSamplerSpecTest.java index 5ede763b8d62..0bf97dcb5160 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaSamplerSpecTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaSamplerSpecTest.java @@ -23,19 +23,16 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import org.apache.curator.test.TestingCluster; -import org.apache.druid.client.cache.MapCache; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.FloatDimensionSchema; -import org.apache.druid.data.input.impl.JSONParseSpec; +import org.apache.druid.data.input.impl.JsonInputFormat; import org.apache.druid.data.input.impl.LongDimensionSchema; import org.apache.druid.data.input.impl.StringDimensionSchema; -import org.apache.druid.data.input.impl.StringInputRowParser; import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorIOConfig; import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorSpec; import org.apache.druid.indexing.kafka.test.TestBroker; -import org.apache.druid.indexing.overlord.sampler.FirehoseSampler; -import org.apache.druid.indexing.overlord.sampler.SamplerCache; +import org.apache.druid.indexing.overlord.sampler.InputSourceSampler; import org.apache.druid.indexing.overlord.sampler.SamplerConfig; import org.apache.druid.indexing.overlord.sampler.SamplerResponse; import org.apache.druid.java.util.common.StringUtils; @@ -54,11 +51,9 @@ import org.junit.BeforeClass; import org.junit.Test; -import java.nio.charset.StandardCharsets; import java.util.Arrays; import java.util.Iterator; import java.util.List; -import java.util.Map; public class KafkaSamplerSpecTest { @@ -66,35 +61,24 @@ public class KafkaSamplerSpecTest private static final String TOPIC = "sampling"; private static final DataSchema DATA_SCHEMA = new DataSchema( "test_ds", - OBJECT_MAPPER.convertValue( - new StringInputRowParser( - new JSONParseSpec( - new TimestampSpec("timestamp", "iso", null), - new DimensionsSpec( - Arrays.asList( - new StringDimensionSchema("dim1"), - new StringDimensionSchema("dim1t"), - new StringDimensionSchema("dim2"), - new LongDimensionSchema("dimLong"), - new FloatDimensionSchema("dimFloat") - ), - null, - null - ), - new JSONPathSpec(true, ImmutableList.of()), - ImmutableMap.of() - ), - StandardCharsets.UTF_8.name() + new TimestampSpec("timestamp", "iso", null), + new DimensionsSpec( + Arrays.asList( + new StringDimensionSchema("dim1"), + new StringDimensionSchema("dim1t"), + new StringDimensionSchema("dim2"), + new LongDimensionSchema("dimLong"), + new FloatDimensionSchema("dimFloat") ), - Map.class + null, + null ), new AggregatorFactory[]{ new DoubleSumAggregatorFactory("met1sum", "met1"), new CountAggregatorFactory("rows") }, new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null), - null, - OBJECT_MAPPER + null ); private static TestingCluster zkServer; @@ -139,6 +123,7 @@ public void testSample() null, new KafkaSupervisorIOConfig( TOPIC, + new JsonInputFormat(new JSONPathSpec(true, ImmutableList.of()), ImmutableMap.of()), null, null, null, @@ -167,22 +152,28 @@ public void testSample() KafkaSamplerSpec samplerSpec = new KafkaSamplerSpec( supervisorSpec, - new SamplerConfig(5, null, null, null), - new FirehoseSampler(OBJECT_MAPPER, new SamplerCache(MapCache.create(100000))), + new SamplerConfig(5, null), + new InputSourceSampler(), OBJECT_MAPPER ); SamplerResponse response = samplerSpec.sample(); - Assert.assertNotNull(response.getCacheKey()); - Assert.assertEquals(5, (int) response.getNumRowsRead()); - Assert.assertEquals(3, (int) response.getNumRowsIndexed()); + Assert.assertEquals(5, response.getNumRowsRead()); + Assert.assertEquals(3, response.getNumRowsIndexed()); Assert.assertEquals(5, response.getData().size()); Iterator it = response.getData().iterator(); Assert.assertEquals(new SamplerResponse.SamplerResponseRow( - "{\"timestamp\":\"2008\",\"dim1\":\"a\",\"dim2\":\"y\",\"dimLong\":\"10\",\"dimFloat\":\"20.0\",\"met1\":\"1.0\"}", + ImmutableMap.builder() + .put("timestamp", "2008") + .put("dim1", "a") + .put("dim2", "y") + .put("dimLong", "10") + .put("dimFloat", "20.0") + .put("met1", "1.0") + .build(), ImmutableMap.builder() .put("__time", 1199145600000L) .put("dim1", "a") @@ -196,7 +187,14 @@ public void testSample() null ), it.next()); Assert.assertEquals(new SamplerResponse.SamplerResponseRow( - "{\"timestamp\":\"2009\",\"dim1\":\"b\",\"dim2\":\"y\",\"dimLong\":\"10\",\"dimFloat\":\"20.0\",\"met1\":\"1.0\"}", + ImmutableMap.builder() + .put("timestamp", "2009") + .put("dim1", "b") + .put("dim2", "y") + .put("dimLong", "10") + .put("dimFloat", "20.0") + .put("met1", "1.0") + .build(), ImmutableMap.builder() .put("__time", 1230768000000L) .put("dim1", "b") @@ -210,7 +208,14 @@ public void testSample() null ), it.next()); Assert.assertEquals(new SamplerResponse.SamplerResponseRow( - "{\"timestamp\":\"2010\",\"dim1\":\"c\",\"dim2\":\"y\",\"dimLong\":\"10\",\"dimFloat\":\"20.0\",\"met1\":\"1.0\"}", + ImmutableMap.builder() + .put("timestamp", "2010") + .put("dim1", "c") + .put("dim2", "y") + .put("dimLong", "10") + .put("dimFloat", "20.0") + .put("met1", "1.0") + .build(), ImmutableMap.builder() .put("__time", 1262304000000L) .put("dim1", "c") @@ -224,16 +229,23 @@ public void testSample() null ), it.next()); Assert.assertEquals(new SamplerResponse.SamplerResponseRow( - "{\"timestamp\":\"246140482-04-24T15:36:27.903Z\",\"dim1\":\"x\",\"dim2\":\"z\",\"dimLong\":\"10\",\"dimFloat\":\"20.0\",\"met1\":\"1.0\"}", + ImmutableMap.builder() + .put("timestamp", "246140482-04-24T15:36:27.903Z") + .put("dim1", "x") + .put("dim2", "z") + .put("dimLong", "10") + .put("dimFloat", "20.0") + .put("met1", "1.0") + .build(), null, true, "Timestamp cannot be represented as a long: [MapBasedInputRow{timestamp=246140482-04-24T15:36:27.903Z, event={timestamp=246140482-04-24T15:36:27.903Z, dim1=x, dim2=z, dimLong=10, dimFloat=20.0, met1=1.0}, dimensions=[dim1, dim1t, dim2, dimLong, dimFloat]}]" ), it.next()); Assert.assertEquals(new SamplerResponse.SamplerResponseRow( - "unparseable", + null, null, true, - "Unable to parse row [unparseable]" + "Unable to parse row [unparseable] into JSON" ), it.next()); Assert.assertFalse(it.hasNext()); diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java index 0662e6e4ef46..f841e9a6357a 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java @@ -32,11 +32,11 @@ import kafka.admin.RackAwareMode; import kafka.utils.ZkUtils; import org.apache.curator.test.TestingCluster; +import org.apache.druid.data.input.InputFormat; import org.apache.druid.data.input.impl.DimensionSchema; import org.apache.druid.data.input.impl.DimensionsSpec; -import org.apache.druid.data.input.impl.JSONParseSpec; +import org.apache.druid.data.input.impl.JsonInputFormat; import org.apache.druid.data.input.impl.StringDimensionSchema; -import org.apache.druid.data.input.impl.StringInputRowParser; import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.indexer.TaskLocation; import org.apache.druid.indexer.TaskStatus; @@ -116,7 +116,6 @@ import java.io.File; import java.io.IOException; -import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -131,6 +130,10 @@ public class KafkaSupervisorTest extends EasyMockSupport { private static final ObjectMapper OBJECT_MAPPER = TestHelper.makeJsonMapper(); + private static final InputFormat INPUT_FORMAT = new JsonInputFormat( + new JSONPathSpec(true, ImmutableList.of()), + ImmutableMap.of() + ); private static final String TOPIC_PREFIX = "testTopic"; private static final String DATASOURCE = "testDS"; private static final int NUM_PARTITIONS = 3; @@ -256,7 +259,8 @@ public void testCreateBaseTaskContexts() throws JsonProcessingException null, null, null, - null + null, + INPUT_FORMAT ), new KafkaIndexTaskTuningConfig( null, @@ -3384,6 +3388,7 @@ private TestableKafkaSupervisor getTestableSupervisor( consumerProperties.put("bootstrap.servers", kafkaHost); KafkaSupervisorIOConfig kafkaSupervisorIOConfig = new KafkaSupervisorIOConfig( topic, + INPUT_FORMAT, replicas, taskCount, new Period(duration), @@ -3492,6 +3497,7 @@ private TestableKafkaSupervisor getTestableSupervisorCustomIsTaskCurrent( consumerProperties.put("isolation.level", "read_committed"); KafkaSupervisorIOConfig kafkaSupervisorIOConfig = new KafkaSupervisorIOConfig( topic, + INPUT_FORMAT, replicas, taskCount, new Period(duration), @@ -3604,6 +3610,7 @@ private KafkaSupervisor getSupervisor( consumerProperties.put("isolation.level", "read_committed"); KafkaSupervisorIOConfig kafkaSupervisorIOConfig = new KafkaSupervisorIOConfig( topic, + INPUT_FORMAT, replicas, taskCount, new Period(duration), @@ -3673,21 +3680,11 @@ private static DataSchema getDataSchema(String dataSource) return new DataSchema( dataSource, - OBJECT_MAPPER.convertValue( - new StringInputRowParser( - new JSONParseSpec( - new TimestampSpec("timestamp", "iso", null), - new DimensionsSpec( - dimensions, - null, - null - ), - new JSONPathSpec(true, ImmutableList.of()), - ImmutableMap.of() - ), - StandardCharsets.UTF_8.name() - ), - Map.class + new TimestampSpec("timestamp", "iso", null), + new DimensionsSpec( + dimensions, + null, + null ), new AggregatorFactory[]{new CountAggregatorFactory("rows")}, new UniformGranularitySpec( @@ -3695,8 +3692,7 @@ private static DataSchema getDataSchema(String dataSource) Granularities.NONE, ImmutableList.of() ), - null, - OBJECT_MAPPER + null ); } @@ -3771,7 +3767,8 @@ private KafkaIndexTask createKafkaIndexTask( KafkaSupervisorIOConfig.DEFAULT_POLL_TIMEOUT_MILLIS, true, minimumMessageTime, - maximumMessageTime + maximumMessageTime, + INPUT_FORMAT ), Collections.emptyMap(), null, diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java index a52a9ebb5591..e8f49bd34aa3 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java @@ -22,7 +22,6 @@ import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.base.Preconditions; import org.apache.druid.common.aws.AWSCredentialsConfig; import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; import org.apache.druid.indexing.common.task.TaskResource; @@ -78,7 +77,7 @@ protected SeekableStreamIndexTaskRunner createTaskRunner() //noinspection unchecked return new KinesisIndexTaskRunner( this, - Preconditions.checkNotNull(dataSchema.getParser(), "inputRowParser"), + dataSchema.getParser(), authorizerMapper, chatHandlerProvider, savedParseExceptions, diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskIOConfig.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskIOConfig.java index 2967ef66474f..cabc99334aaa 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskIOConfig.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskIOConfig.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; +import org.apache.druid.data.input.InputFormat; import org.apache.druid.indexing.seekablestream.SeekableStreamEndSequenceNumbers; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskIOConfig; import org.apache.druid.indexing.seekablestream.SeekableStreamStartSequenceNumbers; @@ -63,6 +64,7 @@ public KinesisIndexTaskIOConfig( @JsonProperty("useTransaction") Boolean useTransaction, @JsonProperty("minimumMessageTime") DateTime minimumMessageTime, @JsonProperty("maximumMessageTime") DateTime maximumMessageTime, + @JsonProperty("inputFormat") @Nullable InputFormat inputFormat, @JsonProperty("endpoint") String endpoint, @JsonProperty("recordsPerFetch") Integer recordsPerFetch, @JsonProperty("fetchDelayMillis") Integer fetchDelayMillis, @@ -78,7 +80,8 @@ public KinesisIndexTaskIOConfig( endSequenceNumbers == null ? endPartitions : endSequenceNumbers, useTransaction, minimumMessageTime, - maximumMessageTime + maximumMessageTime, + inputFormat ); Preconditions.checkArgument( getEndSequenceNumbers().getPartitionSequenceNumberMap() @@ -104,6 +107,7 @@ public KinesisIndexTaskIOConfig( Boolean useTransaction, DateTime minimumMessageTime, DateTime maximumMessageTime, + InputFormat inputFormat, String endpoint, Integer recordsPerFetch, Integer fetchDelayMillis, @@ -123,6 +127,7 @@ public KinesisIndexTaskIOConfig( useTransaction, minimumMessageTime, maximumMessageTime, + inputFormat, endpoint, recordsPerFetch, fetchDelayMillis, diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java index c46d223e5cf3..c2d6f488e2a7 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskRunner.java @@ -62,7 +62,7 @@ public class KinesisIndexTaskRunner extends SeekableStreamIndexTaskRunner parser, + @Nullable InputRowParser parser, AuthorizerMapper authorizerMapper, Optional chatHandlerProvider, CircularBuffer savedParseExceptions, diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisSamplerSpec.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisSamplerSpec.java index 1fd36f279ad7..dbe77a21ee98 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisSamplerSpec.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisSamplerSpec.java @@ -24,15 +24,14 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.inject.name.Named; import org.apache.druid.common.aws.AWSCredentialsConfig; -import org.apache.druid.data.input.Firehose; -import org.apache.druid.data.input.impl.InputRowParser; import org.apache.druid.indexing.kinesis.supervisor.KinesisSupervisorIOConfig; import org.apache.druid.indexing.kinesis.supervisor.KinesisSupervisorSpec; import org.apache.druid.indexing.kinesis.supervisor.KinesisSupervisorTuningConfig; -import org.apache.druid.indexing.overlord.sampler.FirehoseSampler; +import org.apache.druid.indexing.overlord.sampler.InputSourceSampler; import org.apache.druid.indexing.overlord.sampler.SamplerConfig; import org.apache.druid.indexing.seekablestream.SeekableStreamSamplerSpec; -import org.apache.druid.indexing.seekablestream.common.RecordSupplier; + +import javax.annotation.Nullable; public class KinesisSamplerSpec extends SeekableStreamSamplerSpec { @@ -41,52 +40,38 @@ public class KinesisSamplerSpec extends SeekableStreamSamplerSpec @JsonCreator public KinesisSamplerSpec( @JsonProperty("spec") final KinesisSupervisorSpec ingestionSpec, - @JsonProperty("samplerConfig") final SamplerConfig samplerConfig, - @JacksonInject FirehoseSampler firehoseSampler, + @JsonProperty("samplerConfig") @Nullable final SamplerConfig samplerConfig, + @JacksonInject InputSourceSampler inputSourceSampler, @JacksonInject @Named("kinesis") AWSCredentialsConfig awsCredentialsConfig ) { - super(ingestionSpec, samplerConfig, firehoseSampler); + super(ingestionSpec, samplerConfig, inputSourceSampler); this.awsCredentialsConfig = awsCredentialsConfig; } @Override - protected Firehose getFirehose(InputRowParser parser) + protected KinesisRecordSupplier createRecordSupplier() { - return new KinesisSamplerFirehose(parser); - } - - protected class KinesisSamplerFirehose extends SeekableStreamSamplerFirehose - { - protected KinesisSamplerFirehose(InputRowParser parser) - { - super(parser); - } - - @Override - protected RecordSupplier getRecordSupplier() - { - KinesisSupervisorIOConfig ioConfig = (KinesisSupervisorIOConfig) KinesisSamplerSpec.this.ioConfig; - KinesisSupervisorTuningConfig tuningConfig = ((KinesisSupervisorTuningConfig) KinesisSamplerSpec.this.tuningConfig); + KinesisSupervisorIOConfig ioConfig = (KinesisSupervisorIOConfig) KinesisSamplerSpec.this.ioConfig; + KinesisSupervisorTuningConfig tuningConfig = ((KinesisSupervisorTuningConfig) KinesisSamplerSpec.this.tuningConfig); - return new KinesisRecordSupplier( - KinesisRecordSupplier.getAmazonKinesisClient( - ioConfig.getEndpoint(), - awsCredentialsConfig, - ioConfig.getAwsAssumedRoleArn(), - ioConfig.getAwsExternalId() - ), - ioConfig.getRecordsPerFetch(), - ioConfig.getFetchDelayMillis(), - 1, - ioConfig.isDeaggregate(), - tuningConfig.getRecordBufferSize(), - tuningConfig.getRecordBufferOfferTimeout(), - tuningConfig.getRecordBufferFullWait(), - tuningConfig.getFetchSequenceNumberTimeout(), - tuningConfig.getMaxRecordsPerPoll() - ); - } + return new KinesisRecordSupplier( + KinesisRecordSupplier.getAmazonKinesisClient( + ioConfig.getEndpoint(), + awsCredentialsConfig, + ioConfig.getAwsAssumedRoleArn(), + ioConfig.getAwsExternalId() + ), + ioConfig.getRecordsPerFetch(), + ioConfig.getFetchDelayMillis(), + 1, + ioConfig.isDeaggregate(), + tuningConfig.getRecordBufferSize(), + tuningConfig.getRecordBufferOfferTimeout(), + tuningConfig.getRecordBufferFullWait(), + tuningConfig.getFetchSequenceNumberTimeout(), + tuningConfig.getMaxRecordsPerPoll() + ); } } diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java index 74640df82f31..f7454a4b885c 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisor.java @@ -146,6 +146,9 @@ protected SeekableStreamIndexTaskIOConfig createTaskIoConfig( true, minimumMessageTime, maximumMessageTime, + ioConfig.getInputFormat( + spec.getDataSchema().getParser() == null ? null : spec.getDataSchema().getParser().getParseSpec() + ), ioConfig.getEndpoint(), ioConfig.getRecordsPerFetch(), ioConfig.getFetchDelayMillis(), diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfig.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfig.java index c69910cde8fb..f68e0b7a09b3 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfig.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfig.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; +import org.apache.druid.data.input.InputFormat; import org.apache.druid.indexing.kinesis.KinesisIndexTaskIOConfig; import org.apache.druid.indexing.kinesis.KinesisRegion; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorIOConfig; @@ -52,6 +53,7 @@ public class KinesisSupervisorIOConfig extends SeekableStreamSupervisorIOConfig @JsonCreator public KinesisSupervisorIOConfig( @JsonProperty("stream") String stream, + @JsonProperty("inputFormat") InputFormat inputFormat, @JsonProperty("endpoint") String endpoint, @JsonProperty("region") KinesisRegion region, @JsonProperty("replicas") Integer replicas, @@ -73,6 +75,7 @@ public KinesisSupervisorIOConfig( { super( Preconditions.checkNotNull(stream, "stream"), + inputFormat, replicas, taskCount, taskDuration, diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIOConfigTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIOConfigTest.java index 22393a82ff34..ed064f0c5fcc 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIOConfigTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIOConfigTest.java @@ -269,6 +269,7 @@ public void testDeserializeToOldIoConfig() throws IOException true, DateTimes.nowUtc(), DateTimes.nowUtc(), + null, "endpoint", 1000, 2000, diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java index 274549891b92..a334a790bdba 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java @@ -340,6 +340,7 @@ public void testRunAfterDataInserted() throws Exception true, null, null, + INPUT_FORMAT, "awsEndpoint", null, null, @@ -374,7 +375,74 @@ public void testRunAfterDataInserted() throws Exception new KinesisDataSourceMetadata( new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "4")) ), - metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource()) + ); + } + + @Test(timeout = 120_000L) + public void testRunAfterDataInsertedWithLegacyParser() throws Exception + { + recordSupplier.assign(EasyMock.anyObject()); + EasyMock.expectLastCall().anyTimes(); + + EasyMock.expect(recordSupplier.getEarliestSequenceNumber(EasyMock.anyObject())).andReturn("0").anyTimes(); + + recordSupplier.seek(EasyMock.anyObject(), EasyMock.anyString()); + EasyMock.expectLastCall().anyTimes(); + + EasyMock.expect(recordSupplier.poll(EasyMock.anyLong())).andReturn(records.subList(2, 5)).once(); + + recordSupplier.close(); + EasyMock.expectLastCall().once(); + + replayAll(); + + final KinesisIndexTask task = createTask( + null, + OLD_DATA_SCHEMA, + new KinesisIndexTaskIOConfig( + 0, + "sequence0", + new SeekableStreamStartSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "2"), ImmutableSet.of()), + new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "4")), + true, + null, + null, + null, + "awsEndpoint", + null, + null, + null, + null, + false + ) + ); + + final ListenableFuture future = runTask(task); + + // Wait for task to exit + Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode()); + + verifyAll(); + + // Check metrics + Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getProcessed()); + Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); + Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); + + // Check published metadata and segments in deep storage + assertEqualsExceptVersion( + ImmutableList.of( + sdd("2010/P1D", 0, ImmutableList.of("c")), + sdd("2011/P1D", 0, ImmutableList.of("d", "e")) + ), + publishedDescriptors() + ); + Assert.assertEquals( + new KinesisDataSourceMetadata( + new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "4")) + ), + metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource()) ); } @@ -410,6 +478,7 @@ public void testRunBeforeDataInserted() throws Exception true, null, null, + INPUT_FORMAT, "awsEndpoint", null, null, @@ -443,7 +512,7 @@ public void testRunBeforeDataInserted() throws Exception new KinesisDataSourceMetadata( new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID0, "1")) ), - metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource()) ); } @@ -498,6 +567,7 @@ public void testIncrementalHandOff() throws Exception true, null, null, + INPUT_FORMAT, "awsEndpoint", null, null, @@ -522,7 +592,7 @@ public void testIncrementalHandOff() throws Exception Assert.assertTrue( checkpointRequestsHash.contains( Objects.hash( - DATA_SCHEMA.getDataSource(), + NEW_DATA_SCHEMA.getDataSource(), 0, new KinesisDataSourceMetadata(startPartitions) ) @@ -554,7 +624,7 @@ public void testIncrementalHandOff() throws Exception ImmutableMap.of(SHARD_ID1, "9", SHARD_ID0, "1") ) ), - metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource()) ); } @@ -616,6 +686,7 @@ public void testIncrementalHandOffMaxTotalRows() throws Exception true, null, null, + INPUT_FORMAT, "awsEndpoint", null, null, @@ -652,7 +723,7 @@ public void testIncrementalHandOffMaxTotalRows() throws Exception Assert.assertTrue( checkpointRequestsHash.contains( Objects.hash( - DATA_SCHEMA.getDataSource(), + NEW_DATA_SCHEMA.getDataSource(), 0, new KinesisDataSourceMetadata(startPartitions) ) @@ -661,7 +732,7 @@ public void testIncrementalHandOffMaxTotalRows() throws Exception Assert.assertTrue( checkpointRequestsHash.contains( Objects.hash( - DATA_SCHEMA.getDataSource(), + NEW_DATA_SCHEMA.getDataSource(), 0, new KinesisDataSourceMetadata( new SeekableStreamStartSequenceNumbers<>(STREAM, currentOffsets, currentOffsets.keySet())) @@ -688,7 +759,7 @@ public void testIncrementalHandOffMaxTotalRows() throws Exception ); Assert.assertEquals( new KinesisDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "10"))), - metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource()) ); } @@ -721,6 +792,7 @@ public void testRunWithMinimumMessageTime() throws Exception true, DateTimes.of("2010"), null, + INPUT_FORMAT, "awsEndpoint", null, null, @@ -757,7 +829,7 @@ public void testRunWithMinimumMessageTime() throws Exception ); Assert.assertEquals( new KinesisDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "4"))), - metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource()) ); } @@ -790,6 +862,7 @@ public void testRunWithMaximumMessageTime() throws Exception true, null, DateTimes.of("2010"), + INPUT_FORMAT, "awsEndpoint", null, null, @@ -828,7 +901,7 @@ public void testRunWithMaximumMessageTime() throws Exception Assert.assertEquals( new KinesisDataSourceMetadata( new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "4"))), - metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource()) ); } @@ -853,7 +926,7 @@ public void testRunWithTransformSpec() throws Exception final KinesisIndexTask task = createTask( null, - DATA_SCHEMA.withTransformSpec( + NEW_DATA_SCHEMA.withTransformSpec( new TransformSpec( new SelectorDimFilter("dim1", "b", null), ImmutableList.of( @@ -869,6 +942,7 @@ public void testRunWithTransformSpec() throws Exception true, null, null, + INPUT_FORMAT, "awsEndpoint", null, null, @@ -900,7 +974,7 @@ public void testRunWithTransformSpec() throws Exception Assert.assertEquals( new KinesisDataSourceMetadata( new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "4"))), - metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource()) ); // Check segments in deep storage @@ -940,6 +1014,7 @@ public void testRunOnSingletonRange() throws Exception true, null, null, + INPUT_FORMAT, "awsEndpoint", null, null, @@ -996,6 +1071,7 @@ public void testHandoffConditionTimeoutWhenHandoffOccurs() throws Exception true, null, null, + INPUT_FORMAT, "awsEndpoint", null, null, @@ -1029,7 +1105,7 @@ public void testHandoffConditionTimeoutWhenHandoffOccurs() throws Exception new KinesisDataSourceMetadata( new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "4")) ), - metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource()) ); } @@ -1065,6 +1141,7 @@ public void testHandoffConditionTimeoutWhenHandoffDoesNotOccur() throws Exceptio true, null, null, + INPUT_FORMAT, "awsEndpoint", null, null, @@ -1098,7 +1175,7 @@ public void testHandoffConditionTimeoutWhenHandoffDoesNotOccur() throws Exceptio new KinesisDataSourceMetadata( new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "4")) ), - metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource()) ); } @@ -1137,6 +1214,7 @@ public void testReportParseExceptions() throws Exception true, null, null, + INPUT_FORMAT, "awsEndpoint", null, null, @@ -1160,7 +1238,7 @@ public void testReportParseExceptions() throws Exception // Check published metadata Assert.assertEquals(ImmutableList.of(), publishedDescriptors()); - Assert.assertNull(metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())); + Assert.assertNull(metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource())); } @@ -1196,6 +1274,7 @@ public void testMultipleParseExceptionsSuccess() throws Exception true, null, null, + INPUT_FORMAT, "awsEndpoint", null, null, @@ -1230,7 +1309,7 @@ public void testMultipleParseExceptionsSuccess() throws Exception new KinesisDataSourceMetadata( new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "12")) ), - metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource()) ); IngestionStatsAndErrorsTaskReportData reportData = getTaskReportData(); @@ -1295,6 +1374,7 @@ public void testMultipleParseExceptionsFailure() throws Exception true, null, null, + INPUT_FORMAT, "awsEndpoint", null, null, @@ -1321,7 +1401,7 @@ public void testMultipleParseExceptionsFailure() throws Exception // Check published metadata Assert.assertEquals(ImmutableList.of(), publishedDescriptors()); - Assert.assertNull(metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())); + Assert.assertNull(metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource())); IngestionStatsAndErrorsTaskReportData reportData = getTaskReportData(); @@ -1377,6 +1457,7 @@ public void testRunReplicas() throws Exception true, null, null, + INPUT_FORMAT, "awsEndpoint", null, null, @@ -1395,6 +1476,7 @@ public void testRunReplicas() throws Exception true, null, null, + INPUT_FORMAT, "awsEndpoint", null, null, @@ -1433,7 +1515,7 @@ public void testRunReplicas() throws Exception new KinesisDataSourceMetadata( new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "4")) ), - metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource()) ); } @@ -1469,6 +1551,7 @@ public void testRunConflicting() throws Exception true, null, null, + INPUT_FORMAT, "awsEndpoint", null, null, @@ -1487,6 +1570,7 @@ public void testRunConflicting() throws Exception true, null, null, + INPUT_FORMAT, "awsEndpoint", null, null, @@ -1523,7 +1607,7 @@ public void testRunConflicting() throws Exception ); Assert.assertEquals( new KinesisDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "4"))), - metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource()) ); } @@ -1559,6 +1643,7 @@ public void testRunConflictingWithoutTransactions() throws Exception false, null, null, + INPUT_FORMAT, "awsEndpoint", null, null, @@ -1577,6 +1662,7 @@ public void testRunConflictingWithoutTransactions() throws Exception false, null, null, + INPUT_FORMAT, "awsEndpoint", null, null, @@ -1594,7 +1680,7 @@ public void testRunConflictingWithoutTransactions() throws Exception SegmentDescriptorAndExpectedDim1Values desc1 = sdd("2010/P1D", 0, ImmutableList.of("c")); SegmentDescriptorAndExpectedDim1Values desc2 = sdd("2011/P1D", 0, ImmutableList.of("d", "e")); assertEqualsExceptVersion(ImmutableList.of(desc1, desc2), publishedDescriptors()); - Assert.assertNull(metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())); + Assert.assertNull(metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource())); // Run second task final ListenableFuture future2 = runTask(task2); @@ -1614,7 +1700,7 @@ public void testRunConflictingWithoutTransactions() throws Exception SegmentDescriptorAndExpectedDim1Values desc3 = sdd("2011/P1D", 1, ImmutableList.of("d", "e")); SegmentDescriptorAndExpectedDim1Values desc4 = sdd("2013/P1D", 0, ImmutableList.of("f")); assertEqualsExceptVersion(ImmutableList.of(desc1, desc2, desc3, desc4), publishedDescriptors()); - Assert.assertNull(metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())); + Assert.assertNull(metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource())); } @@ -1651,6 +1737,7 @@ public void testRunOneTaskTwoPartitions() throws Exception true, null, null, + INPUT_FORMAT, "awsEndpoint", null, null, @@ -1689,7 +1776,7 @@ public void testRunOneTaskTwoPartitions() throws Exception new KinesisDataSourceMetadata( new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "4", SHARD_ID0, "1")) ), - metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource()) ); } @@ -1725,6 +1812,7 @@ public void testRunTwoTasksTwoPartitions() throws Exception true, null, null, + INPUT_FORMAT, "awsEndpoint", null, null, @@ -1743,6 +1831,7 @@ public void testRunTwoTasksTwoPartitions() throws Exception true, null, null, + INPUT_FORMAT, "awsEndpoint", null, null, @@ -1783,7 +1872,7 @@ public void testRunTwoTasksTwoPartitions() throws Exception new KinesisDataSourceMetadata( new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "4", SHARD_ID0, "1")) ), - metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource()) ); } @@ -1816,6 +1905,7 @@ public void testRestore() throws Exception true, null, null, + INPUT_FORMAT, "awsEndpoint", null, null, @@ -1865,6 +1955,7 @@ public void testRestore() throws Exception true, null, null, + INPUT_FORMAT, "awsEndpoint", null, null, @@ -1906,7 +1997,7 @@ public void testRestore() throws Exception Assert.assertEquals( new KinesisDataSourceMetadata( new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "5"))), - metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource()) ); } @@ -1952,6 +2043,7 @@ public void testRestoreAfterPersistingSequences() throws Exception true, null, null, + INPUT_FORMAT, "awsEndpoint", null, null, @@ -2013,6 +2105,7 @@ public void testRestoreAfterPersistingSequences() throws Exception true, null, null, + INPUT_FORMAT, "awsEndpoint", null, null, @@ -2053,7 +2146,7 @@ public void testRestoreAfterPersistingSequences() throws Exception new KinesisDataSourceMetadata( new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "6")) ), - metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource()) ); } @@ -2082,6 +2175,7 @@ public void testRunWithPauseAndResume() throws Exception true, null, null, + INPUT_FORMAT, "awsEndpoint", null, null, @@ -2156,7 +2250,7 @@ public void testRunWithPauseAndResume() throws Exception STREAM, ImmutableMap.of(SHARD_ID1, currentOffsets.get(SHARD_ID1)) )), - metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource()) ); } @@ -2196,7 +2290,7 @@ public void testRunContextSequenceAheadOfStartingOffsets() throws Exception final KinesisIndexTask task = createTask( "task1", - DATA_SCHEMA, + NEW_DATA_SCHEMA, new KinesisIndexTaskIOConfig( 0, "sequence0", @@ -2205,6 +2299,7 @@ public void testRunContextSequenceAheadOfStartingOffsets() throws Exception true, null, null, + INPUT_FORMAT, "awsEndpoint", null, null, @@ -2234,7 +2329,7 @@ public void testRunContextSequenceAheadOfStartingOffsets() throws Exception ); Assert.assertEquals( new KinesisDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "4"))), - metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource()) ); } @@ -2302,6 +2397,7 @@ public void testIncrementalHandOffReadsThroughEndOffsets() throws Exception true, null, null, + INPUT_FORMAT, "awsEndpoint", null, null, @@ -2311,14 +2407,14 @@ public void testIncrementalHandOffReadsThroughEndOffsets() throws Exception ); final KinesisIndexTask normalReplica = createTask( null, - DATA_SCHEMA, + NEW_DATA_SCHEMA, ioConfig, null ); ((TestableKinesisIndexTask) normalReplica).setLocalSupplier(recordSupplier1); final KinesisIndexTask staleReplica = createTask( null, - DATA_SCHEMA, + NEW_DATA_SCHEMA, ioConfig, null ); @@ -2390,7 +2486,7 @@ public void testIncrementalHandOffReadsThroughEndOffsets() throws Exception new KinesisDataSourceMetadata( new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "9")) ), - metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource()) ); } @@ -2409,7 +2505,7 @@ public void testSequencesFromContext() throws IOException final KinesisIndexTask task = createTask( "task1", - DATA_SCHEMA, + NEW_DATA_SCHEMA, new KinesisIndexTaskIOConfig( 0, "sequence0", @@ -2422,6 +2518,7 @@ public void testSequencesFromContext() throws IOException true, null, null, + INPUT_FORMAT, "awsEndpoint", null, null, @@ -2507,6 +2604,7 @@ public void testEndOfShard() throws Exception true, null, null, + INPUT_FORMAT, "awsEndpoint", null, null, @@ -2544,7 +2642,7 @@ public void testEndOfShard() throws Exception ImmutableMap.of(SHARD_ID1, KinesisSequenceNumber.END_OF_SHARD_MARKER) ) ), - metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource()) ); } @@ -2553,7 +2651,7 @@ private KinesisIndexTask createTask( final KinesisIndexTaskIOConfig ioConfig ) throws JsonProcessingException { - return createTask(taskId, DATA_SCHEMA, ioConfig, null); + return createTask(taskId, NEW_DATA_SCHEMA, ioConfig, null); } private KinesisIndexTask createTask( @@ -2642,10 +2740,12 @@ private static DataSchema cloneDataSchema(final DataSchema dataSchema) { return new DataSchema( dataSchema.getDataSource(), - dataSchema.getParserMap(), + dataSchema.getTimestampSpec(), + dataSchema.getDimensionsSpec(), dataSchema.getAggregators(), dataSchema.getGranularitySpec(), dataSchema.getTransformSpec(), + dataSchema.getParserMap(), OBJECT_MAPPER ); } diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisSamplerSpecTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisSamplerSpecTest.java index 3cfcd97208ea..c84f400f2232 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisSamplerSpecTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisSamplerSpecTest.java @@ -23,25 +23,19 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; -import org.apache.druid.client.cache.MapCache; import org.apache.druid.common.aws.AWSCredentialsConfig; -import org.apache.druid.data.input.Firehose; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.FloatDimensionSchema; -import org.apache.druid.data.input.impl.InputRowParser; -import org.apache.druid.data.input.impl.JSONParseSpec; +import org.apache.druid.data.input.impl.JsonInputFormat; import org.apache.druid.data.input.impl.LongDimensionSchema; import org.apache.druid.data.input.impl.StringDimensionSchema; -import org.apache.druid.data.input.impl.StringInputRowParser; import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.indexing.kinesis.supervisor.KinesisSupervisorIOConfig; import org.apache.druid.indexing.kinesis.supervisor.KinesisSupervisorSpec; -import org.apache.druid.indexing.overlord.sampler.FirehoseSampler; -import org.apache.druid.indexing.overlord.sampler.SamplerCache; +import org.apache.druid.indexing.overlord.sampler.InputSourceSampler; import org.apache.druid.indexing.overlord.sampler.SamplerConfig; import org.apache.druid.indexing.overlord.sampler.SamplerResponse; import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; -import org.apache.druid.indexing.seekablestream.common.RecordSupplier; import org.apache.druid.indexing.seekablestream.common.StreamPartition; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; @@ -49,7 +43,6 @@ import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; -import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.easymock.EasyMock; @@ -57,49 +50,35 @@ import org.junit.Assert; import org.junit.Test; -import java.nio.charset.StandardCharsets; import java.util.Arrays; import java.util.Collections; import java.util.Iterator; import java.util.List; -import java.util.Map; public class KinesisSamplerSpecTest extends EasyMockSupport { - private static final ObjectMapper OBJECT_MAPPER = TestHelper.makeJsonMapper(); private static final String STREAM = "sampling"; private static final String SHARD_ID = "1"; private static final DataSchema DATA_SCHEMA = new DataSchema( "test_ds", - OBJECT_MAPPER.convertValue( - new StringInputRowParser( - new JSONParseSpec( - new TimestampSpec("timestamp", "iso", null), - new DimensionsSpec( - Arrays.asList( - new StringDimensionSchema("dim1"), - new StringDimensionSchema("dim1t"), - new StringDimensionSchema("dim2"), - new LongDimensionSchema("dimLong"), - new FloatDimensionSchema("dimFloat") - ), - null, - null - ), - new JSONPathSpec(true, ImmutableList.of()), - ImmutableMap.of() - ), - StandardCharsets.UTF_8.name() + new TimestampSpec("timestamp", "iso", null), + new DimensionsSpec( + Arrays.asList( + new StringDimensionSchema("dim1"), + new StringDimensionSchema("dim1t"), + new StringDimensionSchema("dim2"), + new LongDimensionSchema("dimLong"), + new FloatDimensionSchema("dimFloat") ), - Map.class + null, + null ), new AggregatorFactory[]{ new DoubleSumAggregatorFactory("met1sum", "met1"), new CountAggregatorFactory("rows") }, new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null), - null, - OBJECT_MAPPER + null ); private final KinesisRecordSupplier recordSupplier = mock(KinesisRecordSupplier.class); @@ -149,6 +128,7 @@ public void testSample() throws Exception null, new KinesisSupervisorIOConfig( STREAM, + new JsonInputFormat(new JSONPathSpec(true, ImmutableList.of()), ImmutableMap.of()), null, null, null, @@ -183,8 +163,8 @@ public void testSample() throws Exception KinesisSamplerSpec samplerSpec = new TestableKinesisSamplerSpec( supervisorSpec, - new SamplerConfig(5, null, null, null), - new FirehoseSampler(OBJECT_MAPPER, new SamplerCache(MapCache.create(100000))), + new SamplerConfig(5, null), + new InputSourceSampler(), null ); @@ -192,15 +172,21 @@ public void testSample() throws Exception verifyAll(); - Assert.assertNotNull(response.getCacheKey()); - Assert.assertEquals(5, (int) response.getNumRowsRead()); - Assert.assertEquals(3, (int) response.getNumRowsIndexed()); + Assert.assertEquals(5, response.getNumRowsRead()); + Assert.assertEquals(3, response.getNumRowsIndexed()); Assert.assertEquals(5, response.getData().size()); Iterator it = response.getData().iterator(); Assert.assertEquals(new SamplerResponse.SamplerResponseRow( - "{\"timestamp\":\"2008\",\"dim1\":\"a\",\"dim2\":\"y\",\"dimLong\":\"10\",\"dimFloat\":\"20.0\",\"met1\":\"1.0\"}", + ImmutableMap.builder() + .put("timestamp", "2008") + .put("dim1", "a") + .put("dim2", "y") + .put("dimLong", "10") + .put("dimFloat", "20.0") + .put("met1", "1.0") + .build(), ImmutableMap.builder() .put("__time", 1199145600000L) .put("dim1", "a") @@ -214,7 +200,14 @@ public void testSample() throws Exception null ), it.next()); Assert.assertEquals(new SamplerResponse.SamplerResponseRow( - "{\"timestamp\":\"2009\",\"dim1\":\"b\",\"dim2\":\"y\",\"dimLong\":\"10\",\"dimFloat\":\"20.0\",\"met1\":\"1.0\"}", + ImmutableMap.builder() + .put("timestamp", "2009") + .put("dim1", "b") + .put("dim2", "y") + .put("dimLong", "10") + .put("dimFloat", "20.0") + .put("met1", "1.0") + .build(), ImmutableMap.builder() .put("__time", 1230768000000L) .put("dim1", "b") @@ -228,7 +221,14 @@ public void testSample() throws Exception null ), it.next()); Assert.assertEquals(new SamplerResponse.SamplerResponseRow( - "{\"timestamp\":\"2010\",\"dim1\":\"c\",\"dim2\":\"y\",\"dimLong\":\"10\",\"dimFloat\":\"20.0\",\"met1\":\"1.0\"}", + ImmutableMap.builder() + .put("timestamp", "2010") + .put("dim1", "c") + .put("dim2", "y") + .put("dimLong", "10") + .put("dimFloat", "20.0") + .put("met1", "1.0") + .build(), ImmutableMap.builder() .put("__time", 1262304000000L) .put("dim1", "c") @@ -242,16 +242,23 @@ public void testSample() throws Exception null ), it.next()); Assert.assertEquals(new SamplerResponse.SamplerResponseRow( - "{\"timestamp\":\"246140482-04-24T15:36:27.903Z\",\"dim1\":\"x\",\"dim2\":\"z\",\"dimLong\":\"10\",\"dimFloat\":\"20.0\",\"met1\":\"1.0\"}", + ImmutableMap.builder() + .put("timestamp", "246140482-04-24T15:36:27.903Z") + .put("dim1", "x") + .put("dim2", "z") + .put("dimLong", "10") + .put("dimFloat", "20.0") + .put("met1", "1.0") + .build(), null, true, "Timestamp cannot be represented as a long: [MapBasedInputRow{timestamp=246140482-04-24T15:36:27.903Z, event={timestamp=246140482-04-24T15:36:27.903Z, dim1=x, dim2=z, dimLong=10, dimFloat=20.0, met1=1.0}, dimensions=[dim1, dim1t, dim2, dimLong, dimFloat]}]" ), it.next()); Assert.assertEquals(new SamplerResponse.SamplerResponseRow( - "unparseable", + null, null, true, - "Unable to parse row [unparseable]" + "Unable to parse row [unparseable] into JSON" ), it.next()); Assert.assertFalse(it.hasNext()); @@ -281,24 +288,17 @@ private class TestableKinesisSamplerSpec extends KinesisSamplerSpec private TestableKinesisSamplerSpec( KinesisSupervisorSpec ingestionSpec, SamplerConfig samplerConfig, - FirehoseSampler firehoseSampler, + InputSourceSampler inputSourceSampler, AWSCredentialsConfig awsCredentialsConfig ) { - super(ingestionSpec, samplerConfig, firehoseSampler, awsCredentialsConfig); + super(ingestionSpec, samplerConfig, inputSourceSampler, awsCredentialsConfig); } @Override - protected Firehose getFirehose(InputRowParser parser) + protected KinesisRecordSupplier createRecordSupplier() { - return new KinesisSamplerFirehose(parser) - { - @Override - protected RecordSupplier getRecordSupplier() - { - return recordSupplier; - } - }; + return recordSupplier; } } } diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java index b9a1ed6e01aa..be79204399a6 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java @@ -26,11 +26,11 @@ import com.google.common.collect.ImmutableSet; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; +import org.apache.druid.data.input.InputFormat; import org.apache.druid.data.input.impl.DimensionSchema; import org.apache.druid.data.input.impl.DimensionsSpec; -import org.apache.druid.data.input.impl.JSONParseSpec; +import org.apache.druid.data.input.impl.JsonInputFormat; import org.apache.druid.data.input.impl.StringDimensionSchema; -import org.apache.druid.data.input.impl.StringInputRowParser; import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.indexer.TaskLocation; import org.apache.druid.indexer.TaskStatus; @@ -99,7 +99,6 @@ import org.junit.Test; import java.io.File; -import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -111,6 +110,10 @@ public class KinesisSupervisorTest extends EasyMockSupport { private static final ObjectMapper OBJECT_MAPPER = TestHelper.makeJsonMapper(); + private static final InputFormat INPUT_FORMAT = new JsonInputFormat( + new JSONPathSpec(true, ImmutableList.of()), + ImmutableMap.of() + ); private static final String DATASOURCE = "testDS"; private static final int TEST_CHAT_THREADS = 3; private static final long TEST_CHAT_RETRIES = 9L; @@ -4596,6 +4599,7 @@ private TestableKinesisSupervisor getTestableSupervisor( { KinesisSupervisorIOConfig kinesisSupervisorIOConfig = new KinesisSupervisorIOConfig( STREAM, + INPUT_FORMAT, "awsEndpoint", null, replicas, @@ -4733,6 +4737,7 @@ private TestableKinesisSupervisor getTestableSupervisor( { KinesisSupervisorIOConfig kinesisSupervisorIOConfig = new KinesisSupervisorIOConfig( STREAM, + INPUT_FORMAT, "awsEndpoint", null, replicas, @@ -4818,6 +4823,7 @@ private TestableKinesisSupervisor getTestableSupervisorCustomIsTaskCurrent( { KinesisSupervisorIOConfig kinesisSupervisorIOConfig = new KinesisSupervisorIOConfig( STREAM, + INPUT_FORMAT, "awsEndpoint", null, replicas, @@ -4905,6 +4911,7 @@ private KinesisSupervisor getSupervisor( { KinesisSupervisorIOConfig kinesisSupervisorIOConfig = new KinesisSupervisorIOConfig( STREAM, + INPUT_FORMAT, "awsEndpoint", null, replicas, @@ -4981,21 +4988,11 @@ private static DataSchema getDataSchema(String dataSource) return new DataSchema( dataSource, - OBJECT_MAPPER.convertValue( - new StringInputRowParser( - new JSONParseSpec( - new TimestampSpec("timestamp", "iso", null), - new DimensionsSpec( - dimensions, - null, - null - ), - new JSONPathSpec(true, ImmutableList.of()), - ImmutableMap.of() - ), - StandardCharsets.UTF_8.name() - ), - Map.class + new TimestampSpec("timestamp", "iso", null), + new DimensionsSpec( + dimensions, + null, + null ), new AggregatorFactory[]{new CountAggregatorFactory("rows")}, new UniformGranularitySpec( @@ -5003,8 +5000,7 @@ private static DataSchema getDataSchema(String dataSource) Granularities.NONE, ImmutableList.of() ), - null, - OBJECT_MAPPER + null ); } @@ -5076,6 +5072,7 @@ private KinesisIndexTask createKinesisIndexTask( true, minimumMessageTime, maximumMessageTime, + INPUT_FORMAT, "awsEndpoint", null, null, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java index b3fecb19c979..3725656aad84 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java @@ -65,6 +65,7 @@ import org.apache.druid.indexing.common.stats.RowIngestionMeters; import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; import org.apache.druid.indexing.common.stats.TaskRealtimeMetricsMonitor; +import org.apache.druid.indexing.overlord.sampler.InputSourceSampler; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Intervals; @@ -1145,6 +1146,10 @@ public InputSource getInputSource() return inputSource; } + /** + * Returns {@link InputFormat}. Can be null if {@link DataSchema#parserMap} is specified. + * Also can be null in {@link InputSourceSampler}. + */ @Nullable @Override @JsonProperty diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/FirehoseSampler.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/FirehoseSampler.java deleted file mode 100644 index 133b92b2b15e..000000000000 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/FirehoseSampler.java +++ /dev/null @@ -1,299 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.indexing.overlord.sampler; - -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Preconditions; -import com.google.common.collect.ImmutableList; -import com.google.common.io.Files; -import com.google.inject.Inject; -import org.apache.commons.io.FileUtils; -import org.apache.commons.lang3.ArrayUtils; -import org.apache.druid.common.utils.UUIDUtils; -import org.apache.druid.data.input.Firehose; -import org.apache.druid.data.input.FirehoseFactory; -import org.apache.druid.data.input.InputRow; -import org.apache.druid.data.input.InputRowListPlusJson; -import org.apache.druid.data.input.Row; -import org.apache.druid.data.input.impl.AbstractTextFilesFirehoseFactory; -import org.apache.druid.data.input.impl.DimensionsSpec; -import org.apache.druid.data.input.impl.InputRowParser; -import org.apache.druid.data.input.impl.ParseSpec; -import org.apache.druid.data.input.impl.StringInputRowParser; -import org.apache.druid.data.input.impl.TimestampSpec; -import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.java.util.common.Intervals; -import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.java.util.common.parsers.ParseException; -import org.apache.druid.java.util.common.parsers.Parser; -import org.apache.druid.java.util.emitter.EmittingLogger; -import org.apache.druid.query.aggregation.AggregatorFactory; -import org.apache.druid.query.aggregation.LongMinAggregatorFactory; -import org.apache.druid.segment.column.ColumnHolder; -import org.apache.druid.segment.incremental.IncrementalIndex; -import org.apache.druid.segment.incremental.IncrementalIndexAddResult; -import org.apache.druid.segment.incremental.IncrementalIndexSchema; -import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.realtime.firehose.TimedShutoffFirehoseFactory; - -import javax.annotation.Nullable; -import java.io.File; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Objects; -import java.util.stream.Collectors; - -public class FirehoseSampler -{ - private static final EmittingLogger log = new EmittingLogger(FirehoseSampler.class); - - // These are convenience shims to allow the data loader to not need to provide a dummy parseSpec during the early - // stages when the parameters for the parseSpec are still unknown and they are only interested in the unparsed rows. - // We need two of these because firehose factories based on AbstractTextFilesFirehoseFactory expect to be used with - // StringInputRowParser, while all the others expect InputRowParser. - // --------------------------- - private static final InputRowParser EMPTY_STRING_PARSER_SHIM = new StringInputRowParser( - new ParseSpec(new TimestampSpec(null, null, DateTimes.EPOCH), new DimensionsSpec(null)) - { - @Override - public Parser makeParser() - { - return new Parser() - { - @Nullable - @Override - public Map parseToMap(String input) - { - throw new ParseException(null); - } - - @Override - public void setFieldNames(Iterable fieldNames) - { - } - - @Override - public List getFieldNames() - { - return ImmutableList.of(); - } - }; - } - }, null); - - private static final InputRowParser EMPTY_PARSER_SHIM = new InputRowParser() - { - @Override - public ParseSpec getParseSpec() - { - return null; - } - - @Override - public InputRowParser withParseSpec(ParseSpec parseSpec) - { - return null; - } - - @Override - public List parseBatch(Object input) - { - throw new ParseException(null); - } - - @Override - public InputRow parse(Object input) - { - throw new ParseException(null); - } - }; - // --------------------------- - - // We want to be able to sort the list of processed results back into the same order that we read them from the - // firehose so that the rows in the data loader are not always changing. To do this, we add a temporary column to the - // InputRow (in SamplerInputRow) and tag each row with a sortKey. We use an aggregator so that it will not affect - // rollup, and we use a longMin aggregator so that as rows get rolled up, the earlier rows stay stable and later - // rows may get rolled into these rows. After getting the results back from the IncrementalIndex, we sort by this - // column and then exclude it from the response. - private static final AggregatorFactory INTERNAL_ORDERING_AGGREGATOR = new LongMinAggregatorFactory( - SamplerInputRow.SAMPLER_ORDERING_COLUMN, - SamplerInputRow.SAMPLER_ORDERING_COLUMN - ); - - private final ObjectMapper objectMapper; - private final SamplerCache samplerCache; - - @Inject - public FirehoseSampler(ObjectMapper objectMapper, SamplerCache samplerCache) - { - this.objectMapper = objectMapper; - this.samplerCache = samplerCache; - } - - public SamplerResponse sample(FirehoseFactory firehoseFactory, DataSchema dataSchema, SamplerConfig samplerConfig) - { - Preconditions.checkNotNull(firehoseFactory, "firehoseFactory required"); - - if (dataSchema == null) { - dataSchema = new DataSchema("sampler", null, null, null, null, objectMapper); - } - - if (samplerConfig == null) { - samplerConfig = SamplerConfig.empty(); - } - - final InputRowParser parser = dataSchema.getParser() != null - ? dataSchema.getParser() - : (firehoseFactory instanceof AbstractTextFilesFirehoseFactory - ? EMPTY_STRING_PARSER_SHIM - : EMPTY_PARSER_SHIM); - - final IncrementalIndexSchema indexSchema = new IncrementalIndexSchema.Builder() - .withTimestampSpec(parser) - .withQueryGranularity(dataSchema.getGranularitySpec().getQueryGranularity()) - .withDimensionsSpec(parser) - .withMetrics(ArrayUtils.addAll(dataSchema.getAggregators(), INTERNAL_ORDERING_AGGREGATOR)) - .withRollup(dataSchema.getGranularitySpec().isRollup()) - .build(); - - FirehoseFactory myFirehoseFactory = null; - boolean usingCachedData = true; - if (!samplerConfig.isSkipCache() && samplerConfig.getCacheKey() != null) { - myFirehoseFactory = samplerCache.getAsFirehoseFactory(samplerConfig.getCacheKey(), parser); - } - if (myFirehoseFactory == null) { - myFirehoseFactory = firehoseFactory; - usingCachedData = false; - } - - if (samplerConfig.getTimeoutMs() > 0) { - myFirehoseFactory = new TimedShutoffFirehoseFactory( - myFirehoseFactory, - DateTimes.nowUtc().plusMillis(samplerConfig.getTimeoutMs()) - ); - } - - final File tempDir = Files.createTempDir(); - try (final Firehose firehose = myFirehoseFactory.connectForSampler(parser, tempDir); - final IncrementalIndex index = new IncrementalIndex.Builder().setIndexSchema(indexSchema) - .setMaxRowCount(samplerConfig.getNumRows()) - .buildOnheap()) { - - List dataToCache = new ArrayList<>(); - SamplerResponse.SamplerResponseRow responseRows[] = new SamplerResponse.SamplerResponseRow[samplerConfig.getNumRows()]; - int counter = 0, numRowsIndexed = 0; - - while (counter < responseRows.length && firehose.hasMore()) { - String raw = null; - try { - final InputRowListPlusJson row = firehose.nextRowWithRaw(); - - if (row == null || row.isEmpty()) { - continue; - } - - if (row.getRaw() != null) { - raw = StringUtils.fromUtf8(row.getRaw()); - - if (!usingCachedData) { - dataToCache.add(row.getRaw()); - } - } - - if (row.getParseException() != null) { - throw row.getParseException(); - } - - if (row.getInputRow() == null) { - continue; - } - - if (!Intervals.ETERNITY.contains(row.getInputRow().getTimestamp())) { - throw new ParseException("Timestamp cannot be represented as a long: [%s]", row.getInputRow()); - } - - IncrementalIndexAddResult result = index.add(new SamplerInputRow(row.getInputRow(), counter), true); - if (result.getParseException() != null) { - throw result.getParseException(); - } else { - // store the raw value; will be merged with the data from the IncrementalIndex later - responseRows[counter] = new SamplerResponse.SamplerResponseRow(raw, null, null, null); - counter++; - numRowsIndexed++; - } - } - catch (ParseException e) { - responseRows[counter] = new SamplerResponse.SamplerResponseRow(raw, null, true, e.getMessage()); - counter++; - } - } - - final List columnNames = index.getColumnNames(); - columnNames.remove(SamplerInputRow.SAMPLER_ORDERING_COLUMN); - - for (Row row : (Iterable) index) { - Map parsed = new HashMap<>(); - - columnNames.forEach(k -> { - if (row.getRaw(k) != null) { - parsed.put(k, row.getRaw(k)); - } - }); - parsed.put(ColumnHolder.TIME_COLUMN_NAME, row.getTimestampFromEpoch()); - - Number sortKey = row.getMetric(SamplerInputRow.SAMPLER_ORDERING_COLUMN); - if (sortKey != null) { - responseRows[sortKey.intValue()] = responseRows[sortKey.intValue()].withParsed(parsed); - } - } - - // cache raw data if available - String cacheKey = usingCachedData ? samplerConfig.getCacheKey() : null; - if (!samplerConfig.isSkipCache() && !dataToCache.isEmpty()) { - cacheKey = samplerCache.put(UUIDUtils.generateUuid(), dataToCache); - } - - return new SamplerResponse( - cacheKey, - counter, - numRowsIndexed, - Arrays.stream(responseRows) - .filter(Objects::nonNull) - .filter(x -> x.getParsed() != null || x.isUnparseable() != null) - .collect(Collectors.toList()) - ); - } - catch (Exception e) { - throw new SamplerException(e, "Failed to sample data: %s", e.getMessage()); - } - finally { - try { - FileUtils.deleteDirectory(tempDir); - } - catch (IOException e) { - log.warn(e, "Failed to cleanup temporary directory"); - } - } - } -} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/IndexTaskSamplerSpec.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/IndexTaskSamplerSpec.java index 241651cfe473..be73cc7b5509 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/IndexTaskSamplerSpec.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/IndexTaskSamplerSpec.java @@ -23,40 +23,74 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; +import org.apache.druid.data.input.FiniteFirehoseFactory; import org.apache.druid.data.input.FirehoseFactory; +import org.apache.druid.data.input.FirehoseFactoryToInputSourceAdaptor; +import org.apache.druid.data.input.InputFormat; +import org.apache.druid.data.input.InputSource; import org.apache.druid.indexing.common.task.IndexTask; +import org.apache.druid.java.util.common.IAE; import org.apache.druid.segment.indexing.DataSchema; +import javax.annotation.Nullable; + public class IndexTaskSamplerSpec implements SamplerSpec { + @Nullable private final DataSchema dataSchema; - private final FirehoseFactory firehoseFactory; + private final InputSource inputSource; + /** + * InputFormat can be null if {@link InputSource#needsFormat()} = false. + */ + @Nullable + private final InputFormat inputFormat; + @Nullable private final SamplerConfig samplerConfig; - private final FirehoseSampler firehoseSampler; + private final InputSourceSampler inputSourceSampler; @JsonCreator public IndexTaskSamplerSpec( @JsonProperty("spec") final IndexTask.IndexIngestionSpec ingestionSpec, - @JsonProperty("samplerConfig") final SamplerConfig samplerConfig, - @JacksonInject FirehoseSampler firehoseSampler + @JsonProperty("samplerConfig") @Nullable final SamplerConfig samplerConfig, + @JacksonInject InputSourceSampler inputSourceSampler ) { this.dataSchema = Preconditions.checkNotNull(ingestionSpec, "[spec] is required").getDataSchema(); Preconditions.checkNotNull(ingestionSpec.getIOConfig(), "[spec.ioConfig] is required"); - this.firehoseFactory = Preconditions.checkNotNull( - ingestionSpec.getIOConfig().getFirehoseFactory(), - "[spec.ioConfig.firehose] is required" - ); + if (ingestionSpec.getIOConfig().getInputSource() != null) { + this.inputSource = ingestionSpec.getIOConfig().getInputSource(); + if (ingestionSpec.getIOConfig().getInputSource().needsFormat()) { + this.inputFormat = Preconditions.checkNotNull( + ingestionSpec.getIOConfig().getInputFormat(), + "[spec.ioConfig.inputFormat] is required" + ); + } else { + this.inputFormat = null; + } + } else { + final FirehoseFactory firehoseFactory = Preconditions.checkNotNull( + ingestionSpec.getIOConfig().getFirehoseFactory(), + "[spec.ioConfig.firehose] is required" + ); + if (!(firehoseFactory instanceof FiniteFirehoseFactory)) { + throw new IAE("firehose should be an instance of FiniteFirehoseFactory"); + } + this.inputSource = new FirehoseFactoryToInputSourceAdaptor( + (FiniteFirehoseFactory) firehoseFactory, + ingestionSpec.getDataSchema().getParser() + ); + this.inputFormat = null; + } this.samplerConfig = samplerConfig; - this.firehoseSampler = firehoseSampler; + this.inputSourceSampler = inputSourceSampler; } @Override public SamplerResponse sample() { - return firehoseSampler.sample(firehoseFactory, dataSchema, samplerConfig); + return inputSourceSampler.sample(inputSource, inputFormat, dataSchema, samplerConfig); } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/InputSourceSampler.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/InputSourceSampler.java new file mode 100644 index 000000000000..6c44a3726157 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/InputSourceSampler.java @@ -0,0 +1,230 @@ +/* + * 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.druid.indexing.overlord.sampler; + +import com.google.common.base.Preconditions; +import com.google.common.io.Files; +import org.apache.commons.io.FileUtils; +import org.apache.commons.lang3.ArrayUtils; +import org.apache.druid.data.input.InputFormat; +import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.InputRowListPlusJson; +import org.apache.druid.data.input.InputRowSchema; +import org.apache.druid.data.input.InputSource; +import org.apache.druid.data.input.InputSourceReader; +import org.apache.druid.data.input.Row; +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.TimedShutoffInputSourceReader; +import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.indexing.overlord.sampler.SamplerResponse.SamplerResponseRow; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.io.Closer; +import org.apache.druid.java.util.common.parsers.CloseableIterator; +import org.apache.druid.java.util.common.parsers.ParseException; +import org.apache.druid.query.aggregation.Aggregator; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.aggregation.LongMinAggregatorFactory; +import org.apache.druid.segment.column.ColumnHolder; +import org.apache.druid.segment.incremental.IncrementalIndex; +import org.apache.druid.segment.incremental.IncrementalIndexAddResult; +import org.apache.druid.segment.incremental.IncrementalIndexSchema; +import org.apache.druid.segment.indexing.DataSchema; + +import javax.annotation.Nullable; +import java.io.File; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.stream.Collectors; + +public class InputSourceSampler +{ + private static final String SAMPLER_DATA_SOURCE = "sampler"; + + private static final DataSchema DEFAULT_DATA_SCHEMA = new DataSchema( + SAMPLER_DATA_SOURCE, + new TimestampSpec(null, null, null), + new DimensionsSpec(null), + null, + null, + null + ); + + // We want to be able to sort the list of processed results back into the same order that we read them from the + // firehose so that the rows in the data loader are not always changing. To do this, we add a temporary column to the + // InputRow (in SamplerInputRow) and tag each row with a sortKey. We use an aggregator so that it will not affect + // rollup, and we use a longMin aggregator so that as rows get rolled up, the earlier rows stay stable and later + // rows may get rolled into these rows. After getting the results back from the IncrementalIndex, we sort by this + // column and then exclude it from the response. + private static final AggregatorFactory INTERNAL_ORDERING_AGGREGATOR = new LongMinAggregatorFactory( + SamplerInputRow.SAMPLER_ORDERING_COLUMN, + SamplerInputRow.SAMPLER_ORDERING_COLUMN + ); + + public SamplerResponse sample( + final InputSource inputSource, + @Nullable final InputFormat inputFormat, // can be null only if inputSource.needsFormat() = false + @Nullable final DataSchema dataSchema, + @Nullable final SamplerConfig samplerConfig + ) + { + Preconditions.checkNotNull(inputSource, "inputSource required"); + if (inputSource.needsFormat()) { + Preconditions.checkNotNull(inputFormat, "inputFormat required"); + } + final DataSchema nonNullDataSchema = dataSchema == null + ? DEFAULT_DATA_SCHEMA + : dataSchema; + final SamplerConfig nonNullSamplerConfig = samplerConfig == null + ? SamplerConfig.empty() + : samplerConfig; + + final Closer closer = Closer.create(); + final File tempDir = Files.createTempDir(); + closer.register(() -> FileUtils.deleteDirectory(tempDir)); + + final InputSourceReader reader = buildReader( + nonNullSamplerConfig, + nonNullDataSchema, + inputSource, + inputFormat, + tempDir + ); + try (final CloseableIterator iterator = reader.sample(); + final IncrementalIndex index = buildIncrementalIndex(nonNullSamplerConfig, nonNullDataSchema); + final Closer closer1 = closer) { + SamplerResponseRow[] responseRows = new SamplerResponseRow[nonNullSamplerConfig.getNumRows()]; + int counter = 0, numRowsIndexed = 0; + + while (counter < responseRows.length && iterator.hasNext()) { + Map rawColumns = null; + try { + final InputRowListPlusJson inputRowListPlusJson = iterator.next(); + + if (inputRowListPlusJson.getRawValues() != null) { + rawColumns = inputRowListPlusJson.getRawValues(); + } + + if (inputRowListPlusJson.getParseException() != null) { + throw inputRowListPlusJson.getParseException(); + } + + if (inputRowListPlusJson.getInputRows() == null) { + continue; + } + + for (InputRow row : inputRowListPlusJson.getInputRows()) { + if (!Intervals.ETERNITY.contains(row.getTimestamp())) { + throw new ParseException("Timestamp cannot be represented as a long: [%s]", row); + } + IncrementalIndexAddResult result = index.add(new SamplerInputRow(row, counter), true); + if (result.getParseException() != null) { + throw result.getParseException(); + } else { + // store the raw value; will be merged with the data from the IncrementalIndex later + responseRows[counter] = new SamplerResponseRow(rawColumns, null, null, null); + counter++; + numRowsIndexed++; + } + } + } + catch (ParseException e) { + responseRows[counter] = new SamplerResponseRow(rawColumns, null, true, e.getMessage()); + counter++; + } + } + + final List columnNames = index.getColumnNames(); + columnNames.remove(SamplerInputRow.SAMPLER_ORDERING_COLUMN); + + for (Row row : index) { + Map parsed = new HashMap<>(); + + columnNames.forEach(k -> { + if (row.getRaw(k) != null) { + parsed.put(k, row.getRaw(k)); + } + }); + parsed.put(ColumnHolder.TIME_COLUMN_NAME, row.getTimestampFromEpoch()); + + Number sortKey = row.getMetric(SamplerInputRow.SAMPLER_ORDERING_COLUMN); + if (sortKey != null) { + responseRows[sortKey.intValue()] = responseRows[sortKey.intValue()].withParsed(parsed); + } + } + + return new SamplerResponse( + counter, + numRowsIndexed, + Arrays.stream(responseRows) + .filter(Objects::nonNull) + .filter(x -> x.getParsed() != null || x.isUnparseable() != null) + .collect(Collectors.toList()) + ); + } + catch (Exception e) { + throw new SamplerException(e, "Failed to sample data: %s", e.getMessage()); + } + } + + private InputSourceReader buildReader( + SamplerConfig samplerConfig, + DataSchema dataSchema, + InputSource inputSource, + InputFormat inputFormat, + File tempDir + ) + { + final List metricsNames = Arrays.stream(dataSchema.getAggregators()) + .map(AggregatorFactory::getName) + .collect(Collectors.toList()); + final InputRowSchema inputRowSchema = new InputRowSchema( + dataSchema.getTimestampSpec(), + dataSchema.getDimensionsSpec(), + metricsNames + ); + + InputSourceReader reader = inputSource.reader(inputRowSchema, inputFormat, tempDir); + + if (samplerConfig.getTimeoutMs() > 0) { + reader = new TimedShutoffInputSourceReader(reader, DateTimes.nowUtc().plusMillis(samplerConfig.getTimeoutMs())); + } + + return dataSchema.getTransformSpec().decorate(reader); + } + + private IncrementalIndex buildIncrementalIndex(SamplerConfig samplerConfig, DataSchema dataSchema) + { + final IncrementalIndexSchema schema = new IncrementalIndexSchema.Builder() + .withTimestampSpec(dataSchema.getTimestampSpec()) + .withQueryGranularity(dataSchema.getGranularitySpec().getQueryGranularity()) + .withDimensionsSpec(dataSchema.getDimensionsSpec()) + .withMetrics(ArrayUtils.addAll(dataSchema.getAggregators(), INTERNAL_ORDERING_AGGREGATOR)) + .withRollup(dataSchema.getGranularitySpec().isRollup()) + .build(); + + return new IncrementalIndex.Builder().setIndexSchema(schema) + .setMaxRowCount(samplerConfig.getNumRows()) + .buildOnheap(); + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/SamplerCache.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/SamplerCache.java deleted file mode 100644 index f3e4becefb5c..000000000000 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/SamplerCache.java +++ /dev/null @@ -1,179 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.indexing.overlord.sampler; - -import org.apache.druid.client.cache.Cache; -import org.apache.druid.data.input.ByteBufferInputRowParser; -import org.apache.druid.data.input.Firehose; -import org.apache.druid.data.input.FirehoseFactory; -import org.apache.druid.data.input.InputRow; -import org.apache.druid.data.input.InputRowListPlusJson; -import org.apache.druid.data.input.impl.InputRowParser; -import org.apache.druid.data.input.impl.StringInputRowParser; -import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.java.util.common.parsers.ParseException; -import org.apache.druid.java.util.emitter.EmittingLogger; - -import javax.annotation.Nullable; -import javax.inject.Inject; -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.File; -import java.io.IOException; -import java.io.ObjectInputStream; -import java.io.ObjectOutputStream; -import java.nio.ByteBuffer; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.Iterator; -import java.util.List; -import java.util.NoSuchElementException; - -public class SamplerCache -{ - private static final EmittingLogger log = new EmittingLogger(SamplerCache.class); - private static final String NAMESPACE = "sampler"; - - private final Cache cache; - - @Inject - public SamplerCache(Cache cache) - { - this.cache = cache; - } - - @Nullable - public String put(String key, Collection values) - { - if (values == null) { - return null; - } - - try (ByteArrayOutputStream baos = new ByteArrayOutputStream(); - ObjectOutputStream oos = new ObjectOutputStream(baos)) { - oos.writeObject(new ArrayList<>(values)); - cache.put(new Cache.NamedKey(NAMESPACE, StringUtils.toUtf8(key)), baos.toByteArray()); - return key; - } - catch (IOException e) { - log.warn(e, "Exception while serializing to sampler cache"); - return null; - } - } - - @Nullable - public FirehoseFactory getAsFirehoseFactory(String key, InputRowParser parser) - { - if (!(parser instanceof ByteBufferInputRowParser)) { - log.warn("SamplerCache expects a ByteBufferInputRowParser"); - return null; - } - - Collection data = get(key); - if (data == null) { - return null; - } - - return new FirehoseFactory() - { - @Override - public Firehose connect(ByteBufferInputRowParser parser, @Nullable File temporaryDirectory) - { - return new SamplerCacheFirehose(parser, data); - } - }; - } - - @Nullable - private Collection get(String key) - { - byte[] data = cache.get(new Cache.NamedKey(NAMESPACE, StringUtils.toUtf8(key))); - if (data == null) { - return null; - } - - try (ByteArrayInputStream bais = new ByteArrayInputStream(data); - ObjectInputStream ois = new ObjectInputStream(bais)) { - return (ArrayList) ois.readObject(); - } - catch (Exception e) { - log.warn(e, "Exception while deserializing from sampler cache"); - return null; - } - } - - public static class SamplerCacheFirehose implements Firehose - { - private final ByteBufferInputRowParser parser; - private final Iterator it; - - public SamplerCacheFirehose(ByteBufferInputRowParser parser, Collection data) - { - this.parser = parser; - this.it = data != null ? data.iterator() : Collections.emptyIterator(); - - if (parser instanceof StringInputRowParser) { - ((StringInputRowParser) parser).startFileFromBeginning(); - } - } - - @Override - public boolean hasMore() - { - return it.hasNext(); - } - - @Nullable - @Override - public InputRow nextRow() - { - if (!hasMore()) { - throw new NoSuchElementException(); - } - - List rows = parser.parseBatch(ByteBuffer.wrap(it.next())); - return rows.isEmpty() ? null : rows.get(0); - } - - @Override - public InputRowListPlusJson nextRowWithRaw() - { - if (!hasMore()) { - throw new NoSuchElementException(); - } - - byte[] raw = it.next(); - - try { - List rows = parser.parseBatch(ByteBuffer.wrap(raw)); - return InputRowListPlusJson.of(rows.isEmpty() ? null : rows.get(0), raw); - } - catch (ParseException e) { - return InputRowListPlusJson.of(raw, e); - } - } - - @Override - public void close() - { - } - } -} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/SamplerConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/SamplerConfig.java index cde5fb8f55ec..3a907c026622 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/SamplerConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/SamplerConfig.java @@ -22,31 +22,23 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; -import org.apache.druid.data.input.InputRowListPlusJson; public class SamplerConfig { private static final int DEFAULT_NUM_ROWS = 200; private static final int MAX_NUM_ROWS = 5000; - private static final boolean DEFAULT_SKIP_CACHE = false; private static final int DEFAULT_TIMEOUT_MS = 10000; private final int numRows; - private final String cacheKey; - private final boolean skipCache; private final int timeoutMs; @JsonCreator public SamplerConfig( @JsonProperty("numRows") Integer numRows, - @JsonProperty("cacheKey") String cacheKey, - @JsonProperty("skipCache") Boolean skipCache, @JsonProperty("timeoutMs") Integer timeoutMs ) { this.numRows = numRows != null ? numRows : DEFAULT_NUM_ROWS; - this.cacheKey = cacheKey; - this.skipCache = skipCache != null ? skipCache : DEFAULT_SKIP_CACHE; this.timeoutMs = timeoutMs != null ? timeoutMs : DEFAULT_TIMEOUT_MS; Preconditions.checkArgument(this.numRows <= MAX_NUM_ROWS, "numRows must be <= %s", MAX_NUM_ROWS); @@ -68,37 +60,6 @@ public int getNumRows() return numRows; } - /** - * The sampler uses a best-effort system to attempt to cache the raw data so that future requests to the sampler - * can be answered without reading again from the source. In addition to responsiveness benefits, this also provides a - * better user experience for sources such as streams, where repeated calls to the sampler (which would happen as the - * user tweaks data schema configurations) would otherwise return a different set of sampled data every time. For the - * caching system to work, 1) the sampler must have access to the raw data (e.g. for {@link FirehoseSampler}, - * {@link InputRowListPlusJson#getRaw()} must be non-null) and 2) the parser must be an - * implementation of {@link org.apache.druid.data.input.ByteBufferInputRowParser} since the data is cached as a byte - * array. If these conditions are not satisfied, the cache returns a miss and the sampler would read from source. - *

- * {@link SamplerResponse} returns a {@link SamplerResponse#cacheKey} which should be supplied here in - * {@link SamplerConfig} for future requests to prefer the cache if available. This field is ignored if - * {@link SamplerConfig#skipCache} is true. - * - * @return key to use for locating previously cached raw data - */ - public String getCacheKey() - { - return cacheKey; - } - - /** - * Whether to read/write to the cache. See cache description in {@link SamplerConfig#getCacheKey()}. - * - * @return true if cache reads and writes should be skipped - */ - public boolean isSkipCache() - { - return skipCache; - } - /** * Time to wait in milliseconds before closing the sampler and returning the data which has already been read. * Particularly useful for handling streaming input sources where the rate of data is unknown, to prevent the sampler @@ -113,6 +74,6 @@ public int getTimeoutMs() public static SamplerConfig empty() { - return new SamplerConfig(null, null, null, null); + return new SamplerConfig(null, null); } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/SamplerModule.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/SamplerModule.java index a4f23c55f19f..07561fcaa1e5 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/SamplerModule.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/SamplerModule.java @@ -51,8 +51,7 @@ public void configure(Binder binder) binder.install(new CacheModule()); - binder.bind(FirehoseSampler.class).in(Singleton.class); + binder.bind(InputSourceSampler.class).in(Singleton.class); binder.bind(SamplerExceptionMapper.class).in(Singleton.class); - binder.bind(SamplerCache.class).in(Singleton.class); } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/SamplerResponse.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/SamplerResponse.java index 2320a3754b4f..efa4b2f8c37f 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/SamplerResponse.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/SamplerResponse.java @@ -29,33 +29,25 @@ @JsonInclude(JsonInclude.Include.NON_NULL) public class SamplerResponse { - private final String cacheKey; - private final Integer numRowsRead; - private final Integer numRowsIndexed; + private final int numRowsRead; + private final int numRowsIndexed; private final List data; - public SamplerResponse(String cacheKey, Integer numRowsRead, Integer numRowsIndexed, List data) + public SamplerResponse(int numRowsRead, int numRowsIndexed, List data) { - this.cacheKey = cacheKey; this.numRowsRead = numRowsRead; this.numRowsIndexed = numRowsIndexed; this.data = data; } @JsonProperty - public String getCacheKey() - { - return cacheKey; - } - - @JsonProperty - public Integer getNumRowsRead() + public int getNumRowsRead() { return numRowsRead; } @JsonProperty - public Integer getNumRowsIndexed() + public int getNumRowsIndexed() { return numRowsIndexed; } @@ -69,28 +61,28 @@ public List getData() @JsonInclude(JsonInclude.Include.NON_NULL) public static class SamplerResponseRow { - private final String raw; + private final Map input; private final Map parsed; private final Boolean unparseable; private final String error; public SamplerResponseRow( - String raw, + Map input, Map parsed, Boolean unparseable, String error ) { - this.raw = raw; + this.input = input; this.parsed = parsed; this.unparseable = unparseable; this.error = error; } @JsonProperty - public String getRaw() + public Map getInput() { - return raw; + return input; } @JsonProperty @@ -113,7 +105,7 @@ public String getError() public SamplerResponseRow withParsed(Map parsed) { - return new SamplerResponseRow(raw, parsed, unparseable, error); + return new SamplerResponseRow(input, parsed, unparseable, error); } @Override @@ -126,7 +118,7 @@ public boolean equals(Object o) return false; } SamplerResponseRow that = (SamplerResponseRow) o; - return Objects.equals(raw, that.raw) && + return Objects.equals(input, that.input) && Objects.equals(parsed, that.parsed) && Objects.equals(unparseable, that.unparseable) && Objects.equals(error, that.error); @@ -135,14 +127,14 @@ public boolean equals(Object o) @Override public int hashCode() { - return Objects.hash(raw, parsed, unparseable, error); + return Objects.hash(input, parsed, unparseable, error); } @Override public String toString() { return "SamplerResponseRow{" + - "raw='" + raw + '\'' + + "rawInput=" + input + ", parsed=" + parsed + ", unparseable=" + unparseable + ", error='" + error + '\'' + diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/RecordSupplierInputSource.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/RecordSupplierInputSource.java new file mode 100644 index 000000000000..f3d1b6122577 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/RecordSupplierInputSource.java @@ -0,0 +1,150 @@ +/* + * 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.druid.indexing.seekablestream; + +import org.apache.druid.data.input.AbstractInputSource; +import org.apache.druid.data.input.InputEntity; +import org.apache.druid.data.input.InputFormat; +import org.apache.druid.data.input.InputRowSchema; +import org.apache.druid.data.input.InputSourceReader; +import org.apache.druid.data.input.impl.ByteEntity; +import org.apache.druid.data.input.impl.InputEntityIteratingReader; +import org.apache.druid.indexing.overlord.sampler.SamplerException; +import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; +import org.apache.druid.indexing.seekablestream.common.RecordSupplier; +import org.apache.druid.indexing.seekablestream.common.StreamPartition; +import org.apache.druid.java.util.common.parsers.CloseableIterator; + +import javax.annotation.Nullable; +import java.io.File; +import java.util.Iterator; +import java.util.Set; +import java.util.stream.Collectors; + +public class RecordSupplierInputSource extends AbstractInputSource +{ + private final String topic; + private final RecordSupplier recordSupplier; + private final boolean useEarliestOffset; + + public RecordSupplierInputSource( + String topic, + RecordSupplier recordSupplier, + boolean useEarliestOffset + ) + { + this.topic = topic; + this.recordSupplier = recordSupplier; + this.useEarliestOffset = useEarliestOffset; + try { + assignAndSeek(recordSupplier); + } + catch (InterruptedException e) { + throw new SamplerException(e, "Exception while seeking to partitions"); + } + } + + private void assignAndSeek(RecordSupplier recordSupplier) + throws InterruptedException + { + final Set> partitions = recordSupplier + .getPartitionIds(topic) + .stream() + .map(partitionId -> StreamPartition.of(topic, partitionId)) + .collect(Collectors.toSet()); + + recordSupplier.assign(partitions); + + if (useEarliestOffset) { + recordSupplier.seekToEarliest(partitions); + } else { + recordSupplier.seekToLatest(partitions); + } + } + + @Override + public boolean isSplittable() + { + return false; + } + + @Override + public boolean needsFormat() + { + return true; + } + + @Override + protected InputSourceReader formattableReader( + InputRowSchema inputRowSchema, + InputFormat inputFormat, + @Nullable File temporaryDirectory + ) + { + return new InputEntityIteratingReader( + inputRowSchema, + inputFormat, + createEntityIterator(), + temporaryDirectory + ); + } + + CloseableIterator createEntityIterator() + { + return new CloseableIterator() + { + private Iterator> recordIterator; + private Iterator bytesIterator; + private volatile boolean closed; + + private void waitNextIteratorIfNecessary() + { + while (!closed && (bytesIterator == null || !bytesIterator.hasNext())) { + while (!closed && (recordIterator == null || !recordIterator.hasNext())) { + recordIterator = recordSupplier.poll(SeekableStreamSamplerSpec.POLL_TIMEOUT_MS).iterator(); + } + if (!closed) { + bytesIterator = recordIterator.next().getData().iterator(); + } + } + } + + @Override + public boolean hasNext() + { + waitNextIteratorIfNecessary(); + return bytesIterator != null && bytesIterator.hasNext(); + } + + @Override + public InputEntity next() + { + return new ByteEntity(bytesIterator.next()); + } + + @Override + public void close() + { + closed = true; + recordSupplier.close(); + } + }; + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskIOConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskIOConfig.java index 1ecb7c268597..9ac35a9fab9e 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskIOConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskIOConfig.java @@ -22,6 +22,8 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Optional; import com.google.common.base.Preconditions; +import org.apache.druid.data.input.InputFormat; +import org.apache.druid.data.input.impl.ParseSpec; import org.apache.druid.segment.indexing.IOConfig; import org.joda.time.DateTime; @@ -39,15 +41,17 @@ public abstract class SeekableStreamIndexTaskIOConfig minimumMessageTime; private final Optional maximumMessageTime; + private final InputFormat inputFormat; public SeekableStreamIndexTaskIOConfig( - final @Nullable Integer taskGroupId, // can be null for backward compabitility + @Nullable final Integer taskGroupId, // can be null for backward compabitility final String baseSequenceName, final SeekableStreamStartSequenceNumbers startSequenceNumbers, final SeekableStreamEndSequenceNumbers endSequenceNumbers, final Boolean useTransaction, final DateTime minimumMessageTime, - final DateTime maximumMessageTime + final DateTime maximumMessageTime, + @Nullable final InputFormat inputFormat ) { this.taskGroupId = taskGroupId; @@ -57,6 +61,7 @@ public SeekableStreamIndexTaskIOConfig( this.useTransaction = useTransaction != null ? useTransaction : DEFAULT_USE_TRANSACTION; this.minimumMessageTime = Optional.fromNullable(minimumMessageTime); this.maximumMessageTime = Optional.fromNullable(maximumMessageTime); + this.inputFormat = inputFormat; Preconditions.checkArgument( startSequenceNumbers.getStream().equals(endSequenceNumbers.getStream()), @@ -114,4 +119,15 @@ public Optional getMinimumMessageTime() return minimumMessageTime; } + @Nullable + @JsonProperty("inputFormat") + private InputFormat getGivenInputFormat() + { + return inputFormat; + } + + public InputFormat getInputFormat(ParseSpec parseSpec) + { + return inputFormat == null ? Preconditions.checkNotNull(parseSpec, "parseSpec").toInputFormat() : inputFormat; + } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java index 3ac7139cc4c3..3591056bdc45 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java @@ -37,7 +37,11 @@ import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.SettableFuture; import org.apache.druid.data.input.Committer; +import org.apache.druid.data.input.InputEntityReader; +import org.apache.druid.data.input.InputFormat; import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.InputRowSchema; +import org.apache.druid.data.input.impl.ByteEntity; import org.apache.druid.data.input.impl.InputRowParser; import org.apache.druid.discovery.DiscoveryDruidNode; import org.apache.druid.discovery.LookupNodeService; @@ -69,8 +73,10 @@ import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.collect.Utils; +import org.apache.druid.java.util.common.parsers.CloseableIterator; import org.apache.druid.java.util.common.parsers.ParseException; import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.segment.indexing.RealtimeIOConfig; import org.apache.druid.segment.realtime.FireDepartment; import org.apache.druid.segment.realtime.FireDepartmentMetrics; @@ -106,6 +112,7 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -196,6 +203,8 @@ public enum Status private final SeekableStreamIndexTask task; private final SeekableStreamIndexTaskIOConfig ioConfig; private final SeekableStreamIndexTaskTuningConfig tuningConfig; + private final InputRowSchema inputRowSchema; + private final InputFormat inputFormat; private final InputRowParser parser; private final AuthorizerMapper authorizerMapper; private final Optional chatHandlerProvider; @@ -226,7 +235,7 @@ public enum Status public SeekableStreamIndexTaskRunner( final SeekableStreamIndexTask task, - final InputRowParser parser, + @Nullable final InputRowParser parser, final AuthorizerMapper authorizerMapper, final Optional chatHandlerProvider, final CircularBuffer savedParseExceptions, @@ -239,6 +248,14 @@ public SeekableStreamIndexTaskRunner( this.task = task; this.ioConfig = task.getIOConfig(); this.tuningConfig = task.getTuningConfig(); + this.inputRowSchema = new InputRowSchema( + task.getDataSchema().getTimestampSpec(), + task.getDataSchema().getDimensionsSpec(), + Arrays.stream(task.getDataSchema().getAggregators()) + .map(AggregatorFactory::getName) + .collect(Collectors.toList()) + ); + this.inputFormat = ioConfig.getInputFormat(parser == null ? null : parser.getParseSpec()); this.parser = parser; this.authorizerMapper = authorizerMapper; this.chatHandlerProvider = chatHandlerProvider; @@ -346,6 +363,42 @@ public void initializeSequences() throws IOException log.info("Starting with sequences: %s", sequences); } + private List parseBytes(List valueBytess) throws IOException + { + if (parser != null) { + return parseWithParser(valueBytess); + } else { + return parseWithInputFormat(valueBytess); + } + } + + private List parseWithParser(List valueBytess) + { + final List rows = new ArrayList<>(); + for (byte[] valueBytes : valueBytess) { + rows.addAll(parser.parseBatch(ByteBuffer.wrap(valueBytes))); + } + return rows; + } + + private List parseWithInputFormat(List valueBytess) throws IOException + { + final List rows = new ArrayList<>(); + for (byte[] valueBytes : valueBytess) { + final InputEntityReader reader = task.getDataSchema().getTransformSpec().decorate( + Preconditions.checkNotNull(inputFormat, "inputFormat").createReader( + inputRowSchema, + new ByteEntity(valueBytes), + toolbox.getIndexingTmpDir() + ) + ); + try (CloseableIterator rowIterator = reader.read()) { + rowIterator.forEachRemaining(rows::add); + } + } + return rows; + } + private TaskStatus runInternal(TaskToolbox toolbox) throws Exception { log.info("SeekableStream indexing task starting up!"); @@ -600,10 +653,7 @@ public void run() if (valueBytess == null || valueBytess.isEmpty()) { rows = Utils.nullableListOf((InputRow) null); } else { - rows = new ArrayList<>(); - for (byte[] valueBytes : valueBytess) { - rows.addAll(parser.parseBatch(ByteBuffer.wrap(valueBytes))); - } + rows = parseBytes(valueBytess); } boolean isPersistRequired = false; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamSamplerSpec.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamSamplerSpec.java index b219d4baaf50..dadeee506666 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamSamplerSpec.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamSamplerSpec.java @@ -20,181 +20,195 @@ package org.apache.druid.indexing.seekablestream; import com.google.common.base.Preconditions; +import org.apache.druid.data.input.ByteBufferInputRowParser; +import org.apache.druid.data.input.FiniteFirehoseFactory; import org.apache.druid.data.input.Firehose; -import org.apache.druid.data.input.FirehoseFactory; +import org.apache.druid.data.input.FirehoseFactoryToInputSourceAdaptor; +import org.apache.druid.data.input.InputEntity; +import org.apache.druid.data.input.InputFormat; import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.InputRowListPlusJson; +import org.apache.druid.data.input.InputSource; +import org.apache.druid.data.input.InputSplit; +import org.apache.druid.data.input.SplitHintSpec; +import org.apache.druid.data.input.impl.ByteEntity; import org.apache.druid.data.input.impl.InputRowParser; import org.apache.druid.data.input.impl.StringInputRowParser; -import org.apache.druid.indexing.overlord.sampler.FirehoseSampler; +import org.apache.druid.indexing.overlord.sampler.InputSourceSampler; import org.apache.druid.indexing.overlord.sampler.SamplerConfig; -import org.apache.druid.indexing.overlord.sampler.SamplerException; import org.apache.druid.indexing.overlord.sampler.SamplerResponse; import org.apache.druid.indexing.overlord.sampler.SamplerSpec; -import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; import org.apache.druid.indexing.seekablestream.common.RecordSupplier; -import org.apache.druid.indexing.seekablestream.common.StreamPartition; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorIOConfig; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorSpec; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorTuningConfig; +import org.apache.druid.java.util.common.parsers.CloseableIterator; import org.apache.druid.java.util.common.parsers.ParseException; import org.apache.druid.segment.indexing.DataSchema; import javax.annotation.Nullable; import java.io.File; +import java.io.IOException; import java.nio.ByteBuffer; -import java.util.Iterator; import java.util.List; -import java.util.Set; -import java.util.stream.Collectors; +import java.util.Map; +import java.util.stream.Stream; public abstract class SeekableStreamSamplerSpec implements SamplerSpec { - private static final int POLL_TIMEOUT_MS = 100; + static final long POLL_TIMEOUT_MS = 100; + @Nullable private final DataSchema dataSchema; - private final FirehoseSampler firehoseSampler; + private final InputSourceSampler inputSourceSampler; protected final SeekableStreamSupervisorIOConfig ioConfig; + @Nullable protected final SeekableStreamSupervisorTuningConfig tuningConfig; protected final SamplerConfig samplerConfig; public SeekableStreamSamplerSpec( final SeekableStreamSupervisorSpec ingestionSpec, - final SamplerConfig samplerConfig, - final FirehoseSampler firehoseSampler + @Nullable final SamplerConfig samplerConfig, + final InputSourceSampler inputSourceSampler ) { this.dataSchema = Preconditions.checkNotNull(ingestionSpec, "[spec] is required").getDataSchema(); this.ioConfig = Preconditions.checkNotNull(ingestionSpec.getIoConfig(), "[spec.ioConfig] is required"); this.tuningConfig = ingestionSpec.getTuningConfig(); - this.samplerConfig = samplerConfig; - this.firehoseSampler = firehoseSampler; + this.samplerConfig = samplerConfig == null ? SamplerConfig.empty() : samplerConfig; + this.inputSourceSampler = inputSourceSampler; } @Override public SamplerResponse sample() { - return firehoseSampler.sample( - new FirehoseFactory() - { - @Override - public Firehose connect(InputRowParser parser, @Nullable File temporaryDirectory) - { - return getFirehose(parser); - } - }, - dataSchema, - samplerConfig - ); + final InputSource inputSource; + final InputFormat inputFormat; + if (dataSchema.getParser() != null) { + inputSource = new FirehoseFactoryToInputSourceAdaptor( + new SeekableStreamSamplerFirehoseFactory(), + dataSchema.getParser() + ); + inputFormat = null; + } else { + inputSource = new RecordSupplierInputSource<>( + ioConfig.getStream(), + createRecordSupplier(), + ioConfig.isUseEarliestSequenceNumber() + ); + inputFormat = Preconditions.checkNotNull( + ioConfig.getInputFormat(null), + "[spec.ioConfig.inputFormat] is required" + ); + } + + return inputSourceSampler.sample(inputSource, inputFormat, dataSchema, samplerConfig); } - protected abstract Firehose getFirehose(InputRowParser parser); + protected abstract RecordSupplier createRecordSupplier(); - protected abstract class SeekableStreamSamplerFirehose implements Firehose + private class SeekableStreamSamplerFirehoseFactory implements FiniteFirehoseFactory { - private final InputRowParser parser; - private final RecordSupplier recordSupplier; + @Override + public Firehose connect(ByteBufferInputRowParser parser, @Nullable File temporaryDirectory) + { + throw new UnsupportedOperationException(); + } - private Iterator> recordIterator; - private Iterator recordDataIterator; + @Override + public Firehose connectForSampler(ByteBufferInputRowParser parser, @Nullable File temporaryDirectory) + { + return new SeekableStreamSamplerFirehose(parser); + } + + @Override + public boolean isSplittable() + { + return false; + } - private volatile boolean closed = false; + @Override + public Stream> getSplits(@Nullable SplitHintSpec splitHintSpec) + { + throw new UnsupportedOperationException(); + } + + @Override + public int getNumSplits(@Nullable SplitHintSpec splitHintSpec) + { + throw new UnsupportedOperationException(); + } + + @Override + public FiniteFirehoseFactory withSplit(InputSplit split) + { + throw new UnsupportedOperationException(); + } + } + + private class SeekableStreamSamplerFirehose implements Firehose + { + private final InputRowParser parser; + private final CloseableIterator entityIterator; protected SeekableStreamSamplerFirehose(InputRowParser parser) { this.parser = parser; - if (parser instanceof StringInputRowParser) { ((StringInputRowParser) parser).startFileFromBeginning(); } - this.recordSupplier = getRecordSupplier(); - - try { - assignAndSeek(); - } - catch (InterruptedException e) { - throw new SamplerException(e, "Exception while seeking to partitions"); - } + RecordSupplierInputSource inputSource = new RecordSupplierInputSource<>( + ioConfig.getStream(), + createRecordSupplier(), + ioConfig.isUseEarliestSequenceNumber() + ); + this.entityIterator = inputSource.createEntityIterator(); } @Override public boolean hasMore() { - return !closed; + return entityIterator.hasNext(); } - @Nullable @Override public InputRow nextRow() { - InputRowListPlusJson row = nextRowWithRaw(); - if (row.getParseException() != null) { - throw row.getParseException(); - } - - return row.getInputRow(); + throw new UnsupportedOperationException(); } @Override public InputRowListPlusJson nextRowWithRaw() { - if (recordDataIterator == null || !recordDataIterator.hasNext()) { - if (recordIterator == null || !recordIterator.hasNext()) { - recordIterator = recordSupplier.poll(POLL_TIMEOUT_MS).iterator(); - - if (!recordIterator.hasNext()) { - return InputRowListPlusJson.of((InputRow) null, null); - } - } + final ByteBuffer bb = ((ByteEntity) entityIterator.next()).getBuffer(); - recordDataIterator = recordIterator.next().getData().iterator(); - - if (!recordDataIterator.hasNext()) { - return InputRowListPlusJson.of((InputRow) null, null); + final Map rawColumns; + try { + if (parser instanceof StringInputRowParser) { + rawColumns = ((StringInputRowParser) parser).buildStringKeyMap(bb); + } else { + rawColumns = null; } } - - byte[] raw = recordDataIterator.next(); + catch (ParseException e) { + return InputRowListPlusJson.of(null, e); + } try { - List rows = parser.parseBatch(ByteBuffer.wrap(raw)); - return InputRowListPlusJson.of(rows.isEmpty() ? null : rows.get(0), raw); + final List rows = parser.parseBatch(bb); + return InputRowListPlusJson.of(rows.isEmpty() ? null : rows, rawColumns); } catch (ParseException e) { - return InputRowListPlusJson.of(raw, e); + return InputRowListPlusJson.of(rawColumns, e); } } @Override - public void close() - { - if (closed) { - return; - } - - closed = true; - recordSupplier.close(); - } - - private void assignAndSeek() throws InterruptedException + public void close() throws IOException { - final Set> partitions = recordSupplier - .getPartitionIds(ioConfig.getStream()) - .stream() - .map(x -> StreamPartition.of(ioConfig.getStream(), x)) - .collect(Collectors.toSet()); - - recordSupplier.assign(partitions); - - if (ioConfig.isUseEarliestSequenceNumber()) { - recordSupplier.seekToEarliest(partitions); - } else { - recordSupplier.seekToLatest(partitions); - } + entityIterator.close(); } - - protected abstract RecordSupplier getRecordSupplier(); } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorIOConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorIOConfig.java index 453a730a956e..caa0abfd41e2 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorIOConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorIOConfig.java @@ -22,15 +22,21 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Optional; import com.google.common.base.Preconditions; +import org.apache.druid.data.input.InputFormat; +import org.apache.druid.data.input.impl.ParseSpec; import org.apache.druid.java.util.common.IAE; import org.joda.time.DateTime; import org.joda.time.Duration; import org.joda.time.Period; +import javax.annotation.Nullable; + public abstract class SeekableStreamSupervisorIOConfig { private final String stream; + @Nullable + private final InputFormat inputFormat; // nullable for backward compatibility private final Integer replicas; private final Integer taskCount; private final Duration taskDuration; @@ -44,6 +50,7 @@ public abstract class SeekableStreamSupervisorIOConfig public SeekableStreamSupervisorIOConfig( String stream, + @Nullable InputFormat inputFormat, Integer replicas, Integer taskCount, Period taskDuration, @@ -57,6 +64,7 @@ public SeekableStreamSupervisorIOConfig( ) { this.stream = Preconditions.checkNotNull(stream, "stream cannot be null"); + this.inputFormat = inputFormat; this.replicas = replicas != null ? replicas : 1; this.taskCount = taskCount != null ? taskCount : 1; this.taskDuration = defaultDuration(taskDuration, "PT1H"); @@ -93,6 +101,23 @@ public String getStream() return stream; } + @Nullable + @JsonProperty + private InputFormat getGivenInputFormat() + { + return inputFormat; + } + + @Nullable + public InputFormat getInputFormat(@Nullable ParseSpec parseSpec) + { + if (inputFormat == null) { + return Preconditions.checkNotNull(parseSpec, "parseSpec").toInputFormat(); + } else { + return inputFormat; + } + } + @JsonProperty public Integer getReplicas() { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/TestFirehose.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/TestFirehose.java index fea30f23a29a..eb9db06747f5 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/TestFirehose.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/TestFirehose.java @@ -19,22 +19,16 @@ package org.apache.druid.indexing.common; -import org.apache.druid.data.input.FiniteFirehoseFactory; import org.apache.druid.data.input.Firehose; import org.apache.druid.data.input.FirehoseFactory; import org.apache.druid.data.input.InputRow; -import org.apache.druid.data.input.InputRowListPlusJson; -import org.apache.druid.data.input.InputSplit; -import org.apache.druid.data.input.impl.AbstractTextFilesFirehoseFactory; import org.apache.druid.data.input.impl.InputRowParser; import org.apache.druid.data.input.impl.StringInputRowParser; -import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.parsers.ParseException; import java.io.File; -import java.io.InputStream; import java.util.ArrayDeque; -import java.util.Collection; +import java.util.Collections; import java.util.Deque; import java.util.List; import java.util.Optional; @@ -44,66 +38,12 @@ public class TestFirehose implements Firehose public static class TestFirehoseFactory implements FirehoseFactory { private boolean waitForClose = true; - private List seedRows; - - public TestFirehoseFactory() - { - } - - public TestFirehoseFactory(boolean waitForClose, List seedRows) - { - this.waitForClose = waitForClose; - this.seedRows = seedRows; - } @Override @SuppressWarnings("unchecked") public Firehose connect(InputRowParser parser, File temporaryDirectory) throws ParseException { - return new TestFirehose(parser, waitForClose, seedRows); - } - } - - public static class TestAbstractTextFilesFirehoseFactory extends AbstractTextFilesFirehoseFactory - { - private boolean waitForClose; - private List seedRows; - - public TestAbstractTextFilesFirehoseFactory(boolean waitForClose, List seedRows) - { - this.waitForClose = waitForClose; - this.seedRows = seedRows; - } - - @Override - @SuppressWarnings("unchecked") - public Firehose connect(StringInputRowParser parser, File temporaryDirectory) throws ParseException - { - return new TestFirehose(parser, waitForClose, seedRows); - } - - @Override - protected Collection initObjects() - { - return null; - } - - @Override - protected InputStream openObjectStream(Object object) - { - return null; - } - - @Override - protected InputStream wrapObjectStream(Object object, InputStream stream) - { - return null; - } - - @Override - public FiniteFirehoseFactory withSplit(InputSplit split) - { - return null; + return new TestFirehose(parser, waitForClose, Collections.emptyList()); } } @@ -167,28 +107,6 @@ public InputRow nextRow() } } - @Override - public InputRowListPlusJson nextRowWithRaw() - { - Object next = queue.removeFirst().orElse(null); - - synchronized (this) { - try { - final InputRow row = parser instanceof StringInputRowParser - ? ((StringInputRowParser) parser).parse((String) next) - : (InputRow) parser.parseBatch(next).get(0); - - if (row != null && row.getRaw(FAIL_DIM) != null) { - throw new ParseException(FAIL_DIM); - } - return InputRowListPlusJson.of(row, next != null ? StringUtils.toUtf8(next.toString()) : null); - } - catch (ParseException e) { - return InputRowListPlusJson.of(next != null ? StringUtils.toUtf8(next.toString()) : null, e); - } - } - } - @Override public void close() { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/FirehoseSamplerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/FirehoseSamplerTest.java deleted file mode 100644 index 06ab5dc1b58e..000000000000 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/FirehoseSamplerTest.java +++ /dev/null @@ -1,839 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.indexing.overlord.sampler; - -import com.fasterxml.jackson.core.type.TypeReference; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import org.apache.druid.client.cache.MapCache; -import org.apache.druid.common.config.NullHandling; -import org.apache.druid.data.input.FirehoseFactory; -import org.apache.druid.data.input.impl.DelimitedParseSpec; -import org.apache.druid.data.input.impl.DimensionsSpec; -import org.apache.druid.data.input.impl.InputRowParser; -import org.apache.druid.data.input.impl.JSONParseSpec; -import org.apache.druid.data.input.impl.MapInputRowParser; -import org.apache.druid.data.input.impl.ParseSpec; -import org.apache.druid.data.input.impl.StringDimensionSchema; -import org.apache.druid.data.input.impl.StringInputRowParser; -import org.apache.druid.data.input.impl.TimestampSpec; -import org.apache.druid.indexing.common.TestFirehose; -import org.apache.druid.indexing.overlord.sampler.SamplerResponse.SamplerResponseRow; -import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.query.aggregation.AggregatorFactory; -import org.apache.druid.query.aggregation.LongSumAggregatorFactory; -import org.apache.druid.query.expression.TestExprMacroTable; -import org.apache.druid.query.filter.SelectorDimFilter; -import org.apache.druid.segment.TestHelper; -import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.granularity.GranularitySpec; -import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; -import org.apache.druid.segment.transform.ExpressionTransform; -import org.apache.druid.segment.transform.TransformSpec; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.ExpectedException; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -import javax.annotation.Nullable; -import java.nio.charset.StandardCharsets; -import java.util.List; -import java.util.Map; -import java.util.stream.Collectors; - -@RunWith(Parameterized.class) -public class FirehoseSamplerTest -{ - private enum ParserType - { - MAP, STR_JSON, STR_CSV - } - - private static final ObjectMapper OBJECT_MAPPER = TestHelper.makeJsonMapper(); - private static final boolean USE_DEFAULT_VALUE_FOR_NULL = Boolean.valueOf(System.getProperty( - NullHandling.NULL_HANDLING_CONFIG_STRING, - "true" - )); - - private static final List MAP_ROWS = ImmutableList.of( - ImmutableMap.of("t", "2019-04-22T12:00", "dim1", "foo", "met1", "1"), - ImmutableMap.of("t", "2019-04-22T12:00", "dim1", "foo", "met1", "2"), - ImmutableMap.of("t", "2019-04-22T12:01", "dim1", "foo", "met1", "3"), - ImmutableMap.of("t", "2019-04-22T12:00", "dim1", "foo2", "met1", "4"), - ImmutableMap.of("t", "2019-04-22T12:00", "dim1", "foo", "dim2", "bar", "met1", "5"), - ImmutableMap.of("t", "bad_timestamp", "dim1", "foo", "met1", "6") - ); - - private static final List STR_JSON_ROWS = ImmutableList.of( - "{ \"t\": \"2019-04-22T12:00\", \"dim1\": \"foo\", \"met1\": 1 }", - "{ \"t\": \"2019-04-22T12:00\", \"dim1\": \"foo\", \"met1\": 2 }", - "{ \"t\": \"2019-04-22T12:01\", \"dim1\": \"foo\", \"met1\": 3 }", - "{ \"t\": \"2019-04-22T12:00\", \"dim1\": \"foo2\", \"met1\": 4 }", - "{ \"t\": \"2019-04-22T12:00\", \"dim1\": \"foo\", \"dim2\": \"bar\", \"met1\": 5 }", - "{ \"t\": \"bad_timestamp\", \"dim1\": \"foo\", \"met1\": 6 }" - ); - - private static final List STR_CSV_ROWS = ImmutableList.of( - "2019-04-22T12:00,foo,,1", - "2019-04-22T12:00,foo,,2", - "2019-04-22T12:01,foo,,3", - "2019-04-22T12:00,foo2,,4", - "2019-04-22T12:00,foo,bar,5", - "bad_timestamp,foo,,6" - ); - - private SamplerCache samplerCache; - private FirehoseSampler firehoseSampler; - private ParserType parserType; - - @Rule - public ExpectedException expectedException = ExpectedException.none(); - - @Parameterized.Parameters(name = "parserType = {0}") - public static Iterable constructorFeeder() - { - return ImmutableList.of( - new Object[]{ParserType.MAP}, - new Object[]{ParserType.STR_JSON}, - new Object[]{ParserType.STR_CSV} - ); - } - - public FirehoseSamplerTest(ParserType parserType) - { - this.parserType = parserType; - } - - @Before - public void setupTest() - { - samplerCache = new SamplerCache(MapCache.create(100000)); - firehoseSampler = new FirehoseSampler(OBJECT_MAPPER, samplerCache); - } - - @Test - public void testNoParams() - { - expectedException.expect(NullPointerException.class); - expectedException.expectMessage("firehoseFactory required"); - - firehoseSampler.sample(null, null, null); - } - - @Test - public void testNoDataSchema() - { - FirehoseFactory firehoseFactory = getFirehoseFactory(getTestRows()); - - SamplerResponse response = firehoseSampler.sample(firehoseFactory, null, null); - - Assert.assertEquals(6, (int) response.getNumRowsRead()); - Assert.assertEquals(0, (int) response.getNumRowsIndexed()); - Assert.assertEquals(6, response.getData().size()); - - List data = response.getData(); - - Assert.assertEquals(new SamplerResponseRow(getTestRows().get(0).toString(), null, true, null), data.get(0)); - Assert.assertEquals(new SamplerResponseRow(getTestRows().get(1).toString(), null, true, null), data.get(1)); - Assert.assertEquals(new SamplerResponseRow(getTestRows().get(2).toString(), null, true, null), data.get(2)); - Assert.assertEquals(new SamplerResponseRow(getTestRows().get(3).toString(), null, true, null), data.get(3)); - Assert.assertEquals(new SamplerResponseRow(getTestRows().get(4).toString(), null, true, null), data.get(4)); - Assert.assertEquals(new SamplerResponseRow(getTestRows().get(5).toString(), null, true, null), data.get(5)); - } - - @Test - public void testNoDataSchemaNumRows() - { - FirehoseFactory firehoseFactory = getFirehoseFactory(getTestRows()); - - SamplerResponse response = firehoseSampler.sample(firehoseFactory, null, new SamplerConfig(3, null, true, null)); - - Assert.assertNull(response.getCacheKey()); - Assert.assertEquals(3, (int) response.getNumRowsRead()); - Assert.assertEquals(0, (int) response.getNumRowsIndexed()); - Assert.assertEquals(3, response.getData().size()); - - List data = response.getData(); - - Assert.assertEquals(new SamplerResponseRow(getTestRows().get(0).toString(), null, true, null), data.get(0)); - Assert.assertEquals(new SamplerResponseRow(getTestRows().get(1).toString(), null, true, null), data.get(1)); - Assert.assertEquals(new SamplerResponseRow(getTestRows().get(2).toString(), null, true, null), data.get(2)); - } - - @Test - public void testNoDataSchemaNumRowsCacheReplay() - { - FirehoseFactory firehoseFactory = getFirehoseFactory(getTestRows()); - - SamplerResponse response = firehoseSampler.sample(firehoseFactory, null, new SamplerConfig(3, null, false, null)); - String cacheKey = response.getCacheKey(); - - Assert.assertNotNull(cacheKey); - Assert.assertEquals(3, (int) response.getNumRowsRead()); - Assert.assertEquals(0, (int) response.getNumRowsIndexed()); - Assert.assertEquals(3, response.getData().size()); - - List data = response.getData(); - - Assert.assertEquals(new SamplerResponseRow(getTestRows().get(0).toString(), null, true, null), data.get(0)); - Assert.assertEquals(new SamplerResponseRow(getTestRows().get(1).toString(), null, true, null), data.get(1)); - Assert.assertEquals(new SamplerResponseRow(getTestRows().get(2).toString(), null, true, null), data.get(2)); - - response = firehoseSampler.sample(firehoseFactory, null, new SamplerConfig(3, cacheKey, false, null)); - - Assert.assertTrue(!isCacheable() || cacheKey.equals(response.getCacheKey())); - Assert.assertEquals(3, (int) response.getNumRowsRead()); - Assert.assertEquals(0, (int) response.getNumRowsIndexed()); - Assert.assertEquals(3, response.getData().size()); - Assert.assertEquals(data, response.getData()); - } - - @Test - public void testMissingValueTimestampSpec() - { - FirehoseFactory firehoseFactory = getFirehoseFactory(getTestRows()); - - ParseSpec parseSpec = getParseSpec(new TimestampSpec(null, null, DateTimes.of("1970")), new DimensionsSpec(null)); - DataSchema dataSchema = new DataSchema("sampler", getParser(parseSpec), null, null, null, OBJECT_MAPPER); - - SamplerResponse response = firehoseSampler.sample(firehoseFactory, dataSchema, null); - - Assert.assertEquals(6, (int) response.getNumRowsRead()); - Assert.assertEquals(6, (int) response.getNumRowsIndexed()); - Assert.assertEquals(6, response.getData().size()); - - List data = removeEmptyColumns(response.getData()); - - Assert.assertEquals(new SamplerResponseRow( - getTestRows().get(0).toString(), - ImmutableMap.of("__time", 0L, "t", "2019-04-22T12:00", "dim1", "foo", "met1", "1"), - null, - null - ), data.get(0)); - Assert.assertEquals(new SamplerResponseRow( - getTestRows().get(1).toString(), - ImmutableMap.of("__time", 0L, "t", "2019-04-22T12:00", "dim1", "foo", "met1", "2"), - null, - null - ), data.get(1)); - Assert.assertEquals(new SamplerResponseRow( - getTestRows().get(2).toString(), - ImmutableMap.of("__time", 0L, "t", "2019-04-22T12:01", "dim1", "foo", "met1", "3"), - null, - null - ), data.get(2)); - Assert.assertEquals(new SamplerResponseRow( - getTestRows().get(3).toString(), - ImmutableMap.of("__time", 0L, "t", "2019-04-22T12:00", "dim1", "foo2", "met1", "4"), - null, - null - ), data.get(3)); - Assert.assertEquals(new SamplerResponseRow( - getTestRows().get(4).toString(), - ImmutableMap.of("__time", 0L, "t", "2019-04-22T12:00", "dim1", "foo", "dim2", "bar", "met1", "5"), - null, - null - ), data.get(4)); - Assert.assertEquals(new SamplerResponseRow( - getTestRows().get(5).toString(), - ImmutableMap.of("__time", 0L, "t", "bad_timestamp", "dim1", "foo", "met1", "6"), - null, - null - ), data.get(5)); - } - - @Test - public void testWithTimestampSpec() - { - FirehoseFactory firehoseFactory = getFirehoseFactory(getTestRows()); - - ParseSpec parseSpec = getParseSpec(new TimestampSpec("t", null, null), new DimensionsSpec(null)); - DataSchema dataSchema = new DataSchema("sampler", getParser(parseSpec), null, null, null, OBJECT_MAPPER); - - SamplerResponse response = firehoseSampler.sample(firehoseFactory, dataSchema, null); - - Assert.assertEquals(6, (int) response.getNumRowsRead()); - Assert.assertEquals(5, (int) response.getNumRowsIndexed()); - Assert.assertEquals(6, response.getData().size()); - - List data = removeEmptyColumns(response.getData()); - - Assert.assertEquals(new SamplerResponseRow( - getTestRows().get(0).toString(), - ImmutableMap.of("__time", 1555934400000L, "dim1", "foo", "met1", "1"), - null, - null - ), data.get(0)); - Assert.assertEquals(new SamplerResponseRow( - getTestRows().get(1).toString(), - ImmutableMap.of("__time", 1555934400000L, "dim1", "foo", "met1", "2"), - null, - null - ), data.get(1)); - Assert.assertEquals(new SamplerResponseRow( - getTestRows().get(2).toString(), - ImmutableMap.of("__time", 1555934460000L, "dim1", "foo", "met1", "3"), - null, - null - ), data.get(2)); - Assert.assertEquals(new SamplerResponseRow( - getTestRows().get(3).toString(), - ImmutableMap.of("__time", 1555934400000L, "dim1", "foo2", "met1", "4"), - null, - null - ), data.get(3)); - Assert.assertEquals(new SamplerResponseRow( - getTestRows().get(4).toString(), - ImmutableMap.of("__time", 1555934400000L, "dim1", "foo", "dim2", "bar", "met1", "5"), - null, - null - ), data.get(4)); - Assert.assertEquals(new SamplerResponseRow( - getTestRows().get(5).toString(), - null, - true, - getUnparseableTimestampString() - ), data.get(5)); - } - - @Test - public void testWithDimensionSpec() - { - FirehoseFactory firehoseFactory = getFirehoseFactory(getTestRows()); - - ParseSpec parseSpec = getParseSpec( - new TimestampSpec("t", null, null), - new DimensionsSpec(ImmutableList.of( - StringDimensionSchema.create("dim1"), - StringDimensionSchema.create("met1") - )) - ); - DataSchema dataSchema = new DataSchema("sampler", getParser(parseSpec), null, null, null, OBJECT_MAPPER); - - SamplerResponse response = firehoseSampler.sample(firehoseFactory, dataSchema, null); - - Assert.assertEquals(6, (int) response.getNumRowsRead()); - Assert.assertEquals(5, (int) response.getNumRowsIndexed()); - Assert.assertEquals(6, response.getData().size()); - - List data = response.getData(); - - Assert.assertEquals(new SamplerResponseRow( - getTestRows().get(0).toString(), - ImmutableMap.of("__time", 1555934400000L, "dim1", "foo", "met1", "1"), - null, - null - ), data.get(0)); - Assert.assertEquals(new SamplerResponseRow( - getTestRows().get(1).toString(), - ImmutableMap.of("__time", 1555934400000L, "dim1", "foo", "met1", "2"), - null, - null - ), data.get(1)); - Assert.assertEquals(new SamplerResponseRow( - getTestRows().get(2).toString(), - ImmutableMap.of("__time", 1555934460000L, "dim1", "foo", "met1", "3"), - null, - null - ), data.get(2)); - Assert.assertEquals(new SamplerResponseRow( - getTestRows().get(3).toString(), - ImmutableMap.of("__time", 1555934400000L, "dim1", "foo2", "met1", "4"), - null, - null - ), data.get(3)); - Assert.assertEquals(new SamplerResponseRow( - getTestRows().get(4).toString(), - ImmutableMap.of("__time", 1555934400000L, "dim1", "foo", "met1", "5"), - null, - null - ), data.get(4)); - Assert.assertEquals(new SamplerResponseRow( - getTestRows().get(5).toString(), - null, - true, - getUnparseableTimestampString() - ), data.get(5)); - } - - @Test - public void testWithNoRollup() - { - FirehoseFactory firehoseFactory = getFirehoseFactory(getTestRows()); - - ParseSpec parseSpec = getParseSpec(new TimestampSpec("t", null, null), new DimensionsSpec(null)); - AggregatorFactory[] aggregatorFactories = {new LongSumAggregatorFactory("met1", "met1")}; - GranularitySpec granularitySpec = new UniformGranularitySpec(Granularities.DAY, Granularities.HOUR, false, null); - DataSchema dataSchema = new DataSchema( - "sampler", - getParser(parseSpec), - aggregatorFactories, - granularitySpec, - null, - OBJECT_MAPPER - ); - - SamplerResponse response = firehoseSampler.sample(firehoseFactory, dataSchema, null); - - Assert.assertEquals(6, (int) response.getNumRowsRead()); - Assert.assertEquals(5, (int) response.getNumRowsIndexed()); - Assert.assertEquals(6, response.getData().size()); - - List data = removeEmptyColumns(response.getData()); - - Assert.assertEquals(new SamplerResponseRow( - getTestRows().get(0).toString(), - ImmutableMap.of("__time", 1555934400000L, "dim1", "foo", "met1", 1L), - null, - null - ), data.get(0)); - Assert.assertEquals(new SamplerResponseRow( - getTestRows().get(1).toString(), - ImmutableMap.of("__time", 1555934400000L, "dim1", "foo", "met1", 2L), - null, - null - ), data.get(1)); - Assert.assertEquals(new SamplerResponseRow( - getTestRows().get(2).toString(), - ImmutableMap.of("__time", 1555934400000L, "dim1", "foo", "met1", 3L), - null, - null - ), data.get(2)); - Assert.assertEquals(new SamplerResponseRow( - getTestRows().get(3).toString(), - ImmutableMap.of("__time", 1555934400000L, "dim1", "foo2", "met1", 4L), - null, - null - ), data.get(3)); - Assert.assertEquals(new SamplerResponseRow( - getTestRows().get(4).toString(), - ImmutableMap.of("__time", 1555934400000L, "dim1", "foo", "dim2", "bar", "met1", 5L), - null, - null - ), data.get(4)); - Assert.assertEquals(new SamplerResponseRow( - getTestRows().get(5).toString(), - null, - true, - getUnparseableTimestampString() - ), data.get(5)); - } - - @Test - public void testWithRollup() - { - FirehoseFactory firehoseFactory = getFirehoseFactory(getTestRows()); - - ParseSpec parseSpec = getParseSpec(new TimestampSpec("t", null, null), new DimensionsSpec(null)); - AggregatorFactory[] aggregatorFactories = {new LongSumAggregatorFactory("met1", "met1")}; - GranularitySpec granularitySpec = new UniformGranularitySpec(Granularities.DAY, Granularities.HOUR, true, null); - DataSchema dataSchema = new DataSchema( - "sampler", - getParser(parseSpec), - aggregatorFactories, - granularitySpec, - null, - OBJECT_MAPPER - ); - - SamplerResponse response = firehoseSampler.sample(firehoseFactory, dataSchema, null); - - Assert.assertEquals(6, (int) response.getNumRowsRead()); - Assert.assertEquals(5, (int) response.getNumRowsIndexed()); - Assert.assertEquals(4, response.getData().size()); - - List data = removeEmptyColumns(response.getData()); - - Assert.assertEquals(new SamplerResponseRow( - getTestRows().get(0).toString(), - ImmutableMap.of("__time", 1555934400000L, "dim1", "foo", "met1", 6L), - null, - null - ), data.get(0)); - Assert.assertEquals(new SamplerResponseRow( - getTestRows().get(3).toString(), - ImmutableMap.of("__time", 1555934400000L, "dim1", "foo2", "met1", 4L), - null, - null - ), data.get(1)); - Assert.assertEquals(new SamplerResponseRow( - getTestRows().get(4).toString(), - ImmutableMap.of("__time", 1555934400000L, "dim1", "foo", "dim2", "bar", "met1", 5L), - null, - null - ), data.get(2)); - Assert.assertEquals(new SamplerResponseRow( - getTestRows().get(5).toString(), - null, - true, - getUnparseableTimestampString() - ), data.get(3)); - } - - @Test - public void testWithMoreRollup() - { - FirehoseFactory firehoseFactory = getFirehoseFactory(getTestRows()); - - ParseSpec parseSpec = getParseSpec( - new TimestampSpec("t", null, null), - new DimensionsSpec(ImmutableList.of(StringDimensionSchema.create("dim1"))) - ); - AggregatorFactory[] aggregatorFactories = {new LongSumAggregatorFactory("met1", "met1")}; - GranularitySpec granularitySpec = new UniformGranularitySpec(Granularities.DAY, Granularities.HOUR, true, null); - DataSchema dataSchema = new DataSchema( - "sampler", - getParser(parseSpec), - aggregatorFactories, - granularitySpec, - null, - OBJECT_MAPPER - ); - - SamplerResponse response = firehoseSampler.sample(firehoseFactory, dataSchema, null); - - Assert.assertEquals(6, (int) response.getNumRowsRead()); - Assert.assertEquals(5, (int) response.getNumRowsIndexed()); - Assert.assertEquals(3, response.getData().size()); - - List data = response.getData(); - - Assert.assertEquals(new SamplerResponseRow( - getTestRows().get(0).toString(), - ImmutableMap.of("__time", 1555934400000L, "dim1", "foo", "met1", 11L), - null, - null - ), data.get(0)); - Assert.assertEquals(new SamplerResponseRow( - getTestRows().get(3).toString(), - ImmutableMap.of("__time", 1555934400000L, "dim1", "foo2", "met1", 4L), - null, - null - ), data.get(1)); - Assert.assertEquals(new SamplerResponseRow( - getTestRows().get(5).toString(), - null, - true, - getUnparseableTimestampString() - ), data.get(2)); - } - - @Test - public void testWithMoreRollupCacheReplay() - { - FirehoseFactory firehoseFactory = getFirehoseFactory(getTestRows()); - - ParseSpec parseSpec = getParseSpec( - new TimestampSpec("t", null, null), - new DimensionsSpec(ImmutableList.of(StringDimensionSchema.create("dim1"))) - ); - AggregatorFactory[] aggregatorFactories = {new LongSumAggregatorFactory("met1", "met1")}; - GranularitySpec granularitySpec = new UniformGranularitySpec(Granularities.DAY, Granularities.HOUR, true, null); - DataSchema dataSchema = new DataSchema( - "sampler", - getParser(parseSpec), - aggregatorFactories, - granularitySpec, - null, - OBJECT_MAPPER - ); - - SamplerResponse response = firehoseSampler.sample(firehoseFactory, dataSchema, null); - String cacheKey = response.getCacheKey(); - - response = firehoseSampler.sample(firehoseFactory, dataSchema, new SamplerConfig(null, cacheKey, false, null)); - - Assert.assertTrue(!isCacheable() || cacheKey.equals(response.getCacheKey())); - - Assert.assertEquals(6, (int) response.getNumRowsRead()); - Assert.assertEquals(5, (int) response.getNumRowsIndexed()); - Assert.assertEquals(3, response.getData().size()); - - List data = response.getData(); - - Assert.assertEquals(new SamplerResponseRow( - getTestRows().get(0).toString(), - ImmutableMap.of("__time", 1555934400000L, "dim1", "foo", "met1", 11L), - null, - null - ), data.get(0)); - Assert.assertEquals(new SamplerResponseRow( - getTestRows().get(3).toString(), - ImmutableMap.of("__time", 1555934400000L, "dim1", "foo2", "met1", 4L), - null, - null - ), data.get(1)); - Assert.assertEquals(new SamplerResponseRow( - getTestRows().get(5).toString(), - null, - true, - getUnparseableTimestampString() - ), data.get(2)); - } - - @Test - public void testWithTransformsAutoDimensions() - { - FirehoseFactory firehoseFactory = getFirehoseFactory(getTestRows()); - - ParseSpec parseSpec = getParseSpec( - new TimestampSpec("t", null, null), - new DimensionsSpec(null) - ); - AggregatorFactory[] aggregatorFactories = {new LongSumAggregatorFactory("met1", "met1")}; - GranularitySpec granularitySpec = new UniformGranularitySpec(Granularities.DAY, Granularities.HOUR, true, null); - TransformSpec transformSpec = new TransformSpec( - null, - ImmutableList.of(new ExpressionTransform("dim1PlusBar", "concat(dim1, 'bar')", TestExprMacroTable.INSTANCE)) - ); - - DataSchema dataSchema = new DataSchema( - "sampler", - getParser(parseSpec), - aggregatorFactories, - granularitySpec, - transformSpec, - OBJECT_MAPPER - ); - - SamplerResponse response = firehoseSampler.sample(firehoseFactory, dataSchema, null); - - Assert.assertEquals(6, (int) response.getNumRowsRead()); - Assert.assertEquals(5, (int) response.getNumRowsIndexed()); - Assert.assertEquals(4, response.getData().size()); - - List data = removeEmptyColumns(response.getData()); - - Assert.assertEquals(new SamplerResponseRow( - getTestRows().get(0).toString(), - ImmutableMap.of("__time", 1555934400000L, "dim1", "foo", "met1", 6L), - null, - null - ), data.get(0)); - Assert.assertEquals(new SamplerResponseRow( - getTestRows().get(3).toString(), - ImmutableMap.of("__time", 1555934400000L, "dim1", "foo2", "met1", 4L), - null, - null - ), data.get(1)); - Assert.assertEquals(new SamplerResponseRow( - getTestRows().get(4).toString(), - ImmutableMap.of("__time", 1555934400000L, "dim1", "foo", "dim2", "bar", "met1", 5L), - null, - null - ), data.get(2)); - Assert.assertEquals(new SamplerResponseRow( - getTestRows().get(5).toString(), - null, - true, - getUnparseableTimestampString() - ), data.get(3)); - } - - @Test - public void testWithTransformsDimensionsSpec() - { - // There's a bug in the CSV parser that does not allow a column added by a transform to be put in the dimensions - // list if the 'columns' field is specified (it will complain that the dimensionName is not a valid column). - if (ParserType.STR_CSV.equals(parserType)) { - return; - } - - FirehoseFactory firehoseFactory = getFirehoseFactory(getTestRows()); - - ParseSpec parseSpec = getParseSpec( - new TimestampSpec("t", null, null), - new DimensionsSpec(ImmutableList.of(StringDimensionSchema.create("dim1PlusBar"))) - ); - AggregatorFactory[] aggregatorFactories = {new LongSumAggregatorFactory("met1", "met1")}; - GranularitySpec granularitySpec = new UniformGranularitySpec(Granularities.DAY, Granularities.HOUR, true, null); - TransformSpec transformSpec = new TransformSpec( - null, - ImmutableList.of(new ExpressionTransform("dim1PlusBar", "concat(dim1 + 'bar')", TestExprMacroTable.INSTANCE)) - ); - - DataSchema dataSchema = new DataSchema( - "sampler", - getParser(parseSpec), - aggregatorFactories, - granularitySpec, - transformSpec, - OBJECT_MAPPER - ); - - SamplerResponse response = firehoseSampler.sample(firehoseFactory, dataSchema, null); - - Assert.assertEquals(6, (int) response.getNumRowsRead()); - Assert.assertEquals(5, (int) response.getNumRowsIndexed()); - Assert.assertEquals(3, response.getData().size()); - - List data = response.getData(); - - Assert.assertEquals(new SamplerResponseRow( - getTestRows().get(0).toString(), - ImmutableMap.of("__time", 1555934400000L, "dim1PlusBar", "foobar", "met1", 11L), - null, - null - ), data.get(0)); - Assert.assertEquals(new SamplerResponseRow( - getTestRows().get(3).toString(), - ImmutableMap.of("__time", 1555934400000L, "dim1PlusBar", "foo2bar", "met1", 4L), - null, - null - ), data.get(1)); - Assert.assertEquals(new SamplerResponseRow( - getTestRows().get(5).toString(), - null, - true, - getUnparseableTimestampString() - ), data.get(2)); - } - - @Test - public void testWithFilter() - { - FirehoseFactory firehoseFactory = getFirehoseFactory(getTestRows()); - - ParseSpec parseSpec = getParseSpec( - new TimestampSpec("t", null, null), - new DimensionsSpec(null) - ); - AggregatorFactory[] aggregatorFactories = {new LongSumAggregatorFactory("met1", "met1")}; - GranularitySpec granularitySpec = new UniformGranularitySpec(Granularities.DAY, Granularities.HOUR, true, null); - TransformSpec transformSpec = new TransformSpec(new SelectorDimFilter("dim1", "foo", null), null); - DataSchema dataSchema = new DataSchema( - "sampler", - getParser(parseSpec), - aggregatorFactories, - granularitySpec, - transformSpec, - OBJECT_MAPPER - ); - - SamplerResponse response = firehoseSampler.sample(firehoseFactory, dataSchema, null); - - Assert.assertEquals(5, (int) response.getNumRowsRead()); - Assert.assertEquals(4, (int) response.getNumRowsIndexed()); - Assert.assertEquals(3, response.getData().size()); - - List data = removeEmptyColumns(response.getData()); - - Assert.assertEquals(new SamplerResponseRow( - getTestRows().get(0).toString(), - ImmutableMap.of("__time", 1555934400000L, "dim1", "foo", "met1", 6L), - null, - null - ), data.get(0)); - Assert.assertEquals(new SamplerResponseRow( - getTestRows().get(4).toString(), - ImmutableMap.of("__time", 1555934400000L, "dim1", "foo", "dim2", "bar", "met1", 5L), - null, - null - ), data.get(1)); - Assert.assertEquals(new SamplerResponseRow( - getTestRows().get(5).toString(), - null, - true, - getUnparseableTimestampString() - ), data.get(2)); - } - - private Map getParser(ParseSpec parseSpec) - { - return OBJECT_MAPPER.convertValue( - ParserType.MAP.equals(parserType) - ? new MapInputRowParser(parseSpec) - : new StringInputRowParser(parseSpec, StandardCharsets.UTF_8.name()), - new TypeReference>() - { - } - ); - } - - private List getTestRows() - { - switch (parserType) { - case MAP: - return MAP_ROWS; - case STR_JSON: - return STR_JSON_ROWS; - case STR_CSV: - return STR_CSV_ROWS; - default: - throw new UnsupportedOperationException(); - } - } - - private FirehoseFactory getFirehoseFactory(List seedRows) - { - return ParserType.MAP.equals(parserType) - ? new TestFirehose.TestFirehoseFactory(false, seedRows) - : new TestFirehose.TestAbstractTextFilesFirehoseFactory(false, seedRows); - } - - private boolean isCacheable() - { - return !ParserType.MAP.equals(parserType); - } - - private ParseSpec getParseSpec(TimestampSpec timestampSpec, DimensionsSpec dimensionsSpec) - { - return ParserType.STR_CSV.equals(parserType) ? new DelimitedParseSpec( - timestampSpec, - dimensionsSpec, - ",", - null, - ImmutableList.of("t", "dim1", "dim2", "met1"), - false, - 0 - ) : new JSONParseSpec(timestampSpec, dimensionsSpec, null, null); - } - - private String getUnparseableTimestampString() - { - return ParserType.STR_CSV.equals(parserType) - ? (USE_DEFAULT_VALUE_FOR_NULL - ? "Unparseable timestamp found! Event: {t=bad_timestamp, dim1=foo, dim2=null, met1=6}" - : "Unparseable timestamp found! Event: {t=bad_timestamp, dim1=foo, dim2=, met1=6}") - : "Unparseable timestamp found! Event: {t=bad_timestamp, dim1=foo, met1=6}"; - } - - private List removeEmptyColumns(List rows) - { - return USE_DEFAULT_VALUE_FOR_NULL - ? rows - : rows.stream().map(x -> x.withParsed(removeEmptyValues(x.getParsed()))).collect(Collectors.toList()); - } - - @Nullable - private Map removeEmptyValues(Map data) - { - return data == null - ? null : data.entrySet() - .stream() - .filter(x -> !(x.getValue() instanceof String) || !((String) x.getValue()).isEmpty()) - .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); - } -} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/IndexTaskSamplerSpecTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/IndexTaskSamplerSpecTest.java index 43b71b180fb5..cf07c4bd5396 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/IndexTaskSamplerSpecTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/IndexTaskSamplerSpecTest.java @@ -22,11 +22,13 @@ import com.fasterxml.jackson.databind.InjectableValues; import com.fasterxml.jackson.databind.Module; import com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.druid.data.input.FirehoseFactory; +import org.apache.druid.data.input.InputFormat; +import org.apache.druid.data.input.InputSource; +import org.apache.druid.data.input.impl.JsonInputFormat; +import org.apache.druid.data.input.impl.LocalInputSource; import org.apache.druid.guice.FirehoseModule; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.realtime.firehose.LocalFirehoseFactory; import org.easymock.Capture; import org.easymock.EasyMock; import org.easymock.EasyMockSupport; @@ -37,13 +39,12 @@ import java.io.File; import java.io.IOException; -import java.util.Map; public class IndexTaskSamplerSpecTest extends EasyMockSupport { private static final ObjectMapper MAPPER = TestHelper.makeJsonMapper(); - private final FirehoseSampler firehoseSampler = createMock(FirehoseSampler.class); + private final InputSourceSampler inputSourceSampler = createMock(InputSourceSampler.class); @Rule public ExpectedException expectedException = ExpectedException.none(); @@ -52,7 +53,7 @@ public IndexTaskSamplerSpecTest() { MAPPER.setInjectableValues( new InjectableValues.Std() - .addValue(FirehoseSampler.class, firehoseSampler) + .addValue(InputSourceSampler.class, inputSourceSampler) .addValue(ObjectMapper.class, MAPPER) ); MAPPER.registerModules((Iterable) new SamplerModule().getJacksonModules()); @@ -66,64 +67,59 @@ public void testSerde() throws IOException + " \"type\": \"index\",\n" + " \"samplerConfig\": {\n" + " \"numRows\": 123,\n" - + " \"cacheKey\": \"eaebbfd87ec34bc6a9f8c03ecee4dd7a\",\n" - + " \"skipCache\": false,\n" + " \"timeoutMs\": 2345\n" + " },\n" + " \"spec\": {\n" + " \"dataSchema\": {\n" + " \"dataSource\": \"sampler\",\n" - + " \"parser\": {\n" - + " \"type\": \"string\",\n" - + " \"parseSpec\": {\n" - + " \"format\": \"json\",\n" - + " \"dimensionsSpec\": {},\n" - + " \"timestampSpec\": {\n" - + " \"missingValue\": \"1970\"\n" - + " }\n" - + " }\n" + + " \"dimensionsSpec\": {},\n" + + " \"timestampSpec\": {\n" + + " \"missingValue\": \"1970\"\n" + " }\n" + " },\n" + " \"ioConfig\": {\n" + " \"type\": \"index\",\n" - + " \"firehose\": {\n" + + " \"inputSource\": {\n" + " \"type\": \"local\",\n" + " \"baseDir\": \"/tmp\",\n" + " \"filter\": \"wikiticker-2015-09-12-sampled.json\"\n" + + " },\n" + + " \"inputFormat\": {\n" + + " \"type\": \"json\"\n" + " }\n" + " }\n" + " }\n" + "}"; - Capture capturedFirehoseFactory = EasyMock.newCapture(); + Capture capturedInputSource = EasyMock.newCapture(); + Capture capturedInputFormat = EasyMock.newCapture(); Capture capturedDataSchema = EasyMock.newCapture(); Capture capturedSamplerConfig = EasyMock.newCapture(); IndexTaskSamplerSpec spec = MAPPER.readValue(json, IndexTaskSamplerSpec.class); - EasyMock.expect(firehoseSampler.sample( - EasyMock.capture(capturedFirehoseFactory), + EasyMock.expect(inputSourceSampler.sample( + EasyMock.capture(capturedInputSource), + EasyMock.capture(capturedInputFormat), EasyMock.capture(capturedDataSchema), EasyMock.capture(capturedSamplerConfig) - )).andReturn(new SamplerResponse(null, null, null, null)); + )).andReturn(new SamplerResponse(0, 0, null)); replayAll(); spec.sample(); verifyAll(); - FirehoseFactory firehoseFactory = capturedFirehoseFactory.getValue(); - Assert.assertEquals(new File("/tmp"), ((LocalFirehoseFactory) firehoseFactory).getBaseDir()); - Assert.assertEquals("wikiticker-2015-09-12-sampled.json", ((LocalFirehoseFactory) firehoseFactory).getFilter()); + InputSource inputSource = capturedInputSource.getValue(); + Assert.assertEquals(new File("/tmp"), ((LocalInputSource) inputSource).getBaseDir()); + Assert.assertEquals("wikiticker-2015-09-12-sampled.json", ((LocalInputSource) inputSource).getFilter()); DataSchema dataSchema = capturedDataSchema.getValue(); Assert.assertEquals("sampler", dataSchema.getDataSource()); - Assert.assertEquals("json", ((Map) dataSchema.getParserMap().get("parseSpec")).get("format")); + Assert.assertEquals(JsonInputFormat.class, capturedInputFormat.getValue().getClass()); SamplerConfig samplerConfig = capturedSamplerConfig.getValue(); Assert.assertEquals(123, samplerConfig.getNumRows()); - Assert.assertEquals("eaebbfd87ec34bc6a9f8c03ecee4dd7a", samplerConfig.getCacheKey()); - Assert.assertFalse(samplerConfig.isSkipCache()); Assert.assertEquals(2345, samplerConfig.getTimeoutMs()); } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/InputSourceSamplerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/InputSourceSamplerTest.java new file mode 100644 index 000000000000..f4e834a9d8ec --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/InputSourceSamplerTest.java @@ -0,0 +1,1100 @@ +/* + * 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.druid.indexing.overlord.sampler; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import org.apache.druid.common.config.NullHandling; +import org.apache.druid.data.input.FirehoseFactoryToInputSourceAdaptor; +import org.apache.druid.data.input.InputFormat; +import org.apache.druid.data.input.InputSource; +import org.apache.druid.data.input.impl.CsvInputFormat; +import org.apache.druid.data.input.impl.DelimitedParseSpec; +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.InlineInputSource; +import org.apache.druid.data.input.impl.InputRowParser; +import org.apache.druid.data.input.impl.JSONParseSpec; +import org.apache.druid.data.input.impl.JsonInputFormat; +import org.apache.druid.data.input.impl.StringDimensionSchema; +import org.apache.druid.data.input.impl.StringInputRowParser; +import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.indexing.overlord.sampler.SamplerResponse.SamplerResponseRow; +import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.collect.Utils; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.aggregation.LongSumAggregatorFactory; +import org.apache.druid.query.expression.TestExprMacroTable; +import org.apache.druid.query.filter.SelectorDimFilter; +import org.apache.druid.segment.indexing.DataSchema; +import org.apache.druid.segment.indexing.granularity.GranularitySpec; +import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; +import org.apache.druid.segment.realtime.firehose.InlineFirehoseFactory; +import org.apache.druid.segment.transform.ExpressionTransform; +import org.apache.druid.segment.transform.TransformSpec; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Objects; +import java.util.stream.Collectors; + +@RunWith(Parameterized.class) +public class InputSourceSamplerTest +{ + private enum ParserType + { + STR_JSON, STR_CSV + } + + private static final ObjectMapper OBJECT_MAPPER = new DefaultObjectMapper(); + private static final boolean USE_DEFAULT_VALUE_FOR_NULL = Boolean.parseBoolean( + System.getProperty(NullHandling.NULL_HANDLING_CONFIG_STRING, "true") + ); + + private static final List STR_JSON_ROWS = ImmutableList.of( + "{ \"t\": \"2019-04-22T12:00\", \"dim1\": \"foo\", \"met1\": 1 }", + "{ \"t\": \"2019-04-22T12:00\", \"dim1\": \"foo\", \"met1\": 2 }", + "{ \"t\": \"2019-04-22T12:01\", \"dim1\": \"foo\", \"met1\": 3 }", + "{ \"t\": \"2019-04-22T12:00\", \"dim1\": \"foo2\", \"met1\": 4 }", + "{ \"t\": \"2019-04-22T12:00\", \"dim1\": \"foo\", \"dim2\": \"bar\", \"met1\": 5 }", + "{ \"t\": \"bad_timestamp\", \"dim1\": \"foo\", \"met1\": 6 }" + ); + + private static final List STR_CSV_ROWS = ImmutableList.of( + "2019-04-22T12:00,foo,,1", + "2019-04-22T12:00,foo,,2", + "2019-04-22T12:01,foo,,3", + "2019-04-22T12:00,foo2,,4", + "2019-04-22T12:00,foo,bar,5", + "bad_timestamp,foo,,6" + ); + + + private List> mapOfRows; + private InputSourceSampler inputSourceSampler; + private ParserType parserType; + private boolean useInputFormatApi; + + @Rule + public ExpectedException expectedException = ExpectedException.none(); + + @Parameterized.Parameters(name = "parserType = {0}, useInputFormatApi={1}") + public static Iterable constructorFeeder() + { + return ImmutableList.of( + new Object[]{ParserType.STR_JSON, false}, + new Object[]{ParserType.STR_JSON, true}, + new Object[]{ParserType.STR_CSV, false}, + new Object[]{ParserType.STR_CSV, true} + ); + } + + public InputSourceSamplerTest(ParserType parserType, boolean useInputFormatApi) + { + this.parserType = parserType; + this.useInputFormatApi = useInputFormatApi; + } + + @Before + public void setupTest() + { + inputSourceSampler = new InputSourceSampler(); + + mapOfRows = new ArrayList<>(); + final List columns = ImmutableList.of("t", "dim1", "dim2", "met1"); + for (String row : STR_CSV_ROWS) { + final List values = new ArrayList<>(); + final String[] tokens = row.split(","); + for (int i = 0; i < tokens.length; i++) { + if (i < tokens.length - 1) { + values.add("".equals(tokens[i]) ? null : tokens[i]); + } else { + values.add(Integer.parseInt(tokens[i])); + } + } + mapOfRows.add(Utils.zipMapPartial(columns, values)); + } + } + + @Test + public void testNoParams() + { + expectedException.expect(NullPointerException.class); + expectedException.expectMessage("inputSource required"); + + inputSourceSampler.sample(null, null, null, null); + } + + @Test + public void testNoDataSchema() + { + final InputSource inputSource = createInputSource(getTestRows(), null); + final SamplerResponse response = inputSourceSampler.sample(inputSource, createInputFormat(), null, null); + + Assert.assertEquals(6, response.getNumRowsRead()); + Assert.assertEquals(0, response.getNumRowsIndexed()); + Assert.assertEquals(6, response.getData().size()); + + List data = response.getData(); + + assertEqualsSamplerResponseRow( + new SamplerResponseRow( + getRawColumns().get(0), + null, + true, + unparseableTimestampErrorString(data.get(0).getInput()) + ), + data.get(0) + ); + assertEqualsSamplerResponseRow( + new SamplerResponseRow( + getRawColumns().get(1), + null, + true, + unparseableTimestampErrorString(data.get(1).getInput()) + ), + data.get(1) + ); + assertEqualsSamplerResponseRow( + new SamplerResponseRow( + getRawColumns().get(2), + null, + true, + unparseableTimestampErrorString(data.get(2).getInput()) + ), + data.get(2) + ); + assertEqualsSamplerResponseRow( + new SamplerResponseRow( + getRawColumns().get(3), + null, + true, + unparseableTimestampErrorString(data.get(3).getInput()) + ), + data.get(3) + ); + assertEqualsSamplerResponseRow( + new SamplerResponseRow( + getRawColumns().get(4), + null, + true, + unparseableTimestampErrorString(data.get(4).getInput()) + ), + data.get(4) + ); + assertEqualsSamplerResponseRow( + new SamplerResponseRow( + getRawColumns().get(5), + null, + true, + unparseableTimestampErrorString(data.get(5).getInput()) + ), + data.get(5) + ); + } + + @Test + public void testNoDataSchemaNumRows() + { + final InputSource inputSource = createInputSource(getTestRows(), null); + final SamplerResponse response = inputSourceSampler.sample( + inputSource, + createInputFormat(), + null, + new SamplerConfig(3, null) + ); + + Assert.assertEquals(3, response.getNumRowsRead()); + Assert.assertEquals(0, response.getNumRowsIndexed()); + Assert.assertEquals(3, response.getData().size()); + + List data = response.getData(); + + assertEqualsSamplerResponseRow( + new SamplerResponseRow( + getRawColumns().get(0), + null, + true, + unparseableTimestampErrorString(data.get(0).getInput()) + ), + data.get(0) + ); + assertEqualsSamplerResponseRow( + new SamplerResponseRow( + getRawColumns().get(1), + null, + true, + unparseableTimestampErrorString(data.get(1).getInput()) + ), + data.get(1) + ); + assertEqualsSamplerResponseRow( + new SamplerResponseRow( + getRawColumns().get(2), + null, + true, + unparseableTimestampErrorString(data.get(2).getInput()) + ), + data.get(2) + ); + } + + @Test + public void testMissingValueTimestampSpec() throws IOException + { + final TimestampSpec timestampSpec = new TimestampSpec(null, null, DateTimes.of("1970")); + final DimensionsSpec dimensionsSpec = new DimensionsSpec(null); + final DataSchema dataSchema = createDataSchema(timestampSpec, dimensionsSpec, null, null, null); + final InputSource inputSource = createInputSource(getTestRows(), dataSchema); + final InputFormat inputFormat = createInputFormat(); + + SamplerResponse response = inputSourceSampler.sample(inputSource, inputFormat, dataSchema, null); + + Assert.assertEquals(6, response.getNumRowsRead()); + Assert.assertEquals(6, response.getNumRowsIndexed()); + Assert.assertEquals(6, response.getData().size()); + + List data = removeEmptyColumns(response.getData()); + + assertEqualsSamplerResponseRow( + new SamplerResponseRow( + getRawColumns().get(0), + ImmutableMap.of("__time", 0L, "t", "2019-04-22T12:00", "dim1", "foo", "met1", "1"), + null, + null + ), + data.get(0) + ); + assertEqualsSamplerResponseRow( + new SamplerResponseRow( + getRawColumns().get(1), + ImmutableMap.of("__time", 0L, "t", "2019-04-22T12:00", "dim1", "foo", "met1", "2"), + null, + null + ), + data.get(1) + ); + assertEqualsSamplerResponseRow( + new SamplerResponseRow( + getRawColumns().get(2), + ImmutableMap.of("__time", 0L, "t", "2019-04-22T12:01", "dim1", "foo", "met1", "3"), + null, + null + ), + data.get(2) + ); + assertEqualsSamplerResponseRow( + new SamplerResponseRow( + getRawColumns().get(3), + ImmutableMap.of("__time", 0L, "t", "2019-04-22T12:00", "dim1", "foo2", "met1", "4"), + null, + null + ), + data.get(3) + ); + assertEqualsSamplerResponseRow( + new SamplerResponseRow( + getRawColumns().get(4), + ImmutableMap.of("__time", 0L, "t", "2019-04-22T12:00", "dim1", "foo", "dim2", "bar", "met1", "5"), + null, + null + ), + data.get(4) + ); + assertEqualsSamplerResponseRow( + new SamplerResponseRow( + getRawColumns().get(5), + ImmutableMap.of("__time", 0L, "t", "bad_timestamp", "dim1", "foo", "met1", "6"), + null, + null + ), + data.get(5) + ); + } + + @Test + public void testWithTimestampSpec() throws IOException + { + final TimestampSpec timestampSpec = new TimestampSpec("t", null, null); + final DimensionsSpec dimensionsSpec = new DimensionsSpec(null); + final DataSchema dataSchema = createDataSchema(timestampSpec, dimensionsSpec, null, null, null); + final InputSource inputSource = createInputSource(getTestRows(), dataSchema); + final InputFormat inputFormat = createInputFormat(); + + SamplerResponse response = inputSourceSampler.sample(inputSource, inputFormat, dataSchema, null); + + Assert.assertEquals(6, response.getNumRowsRead()); + Assert.assertEquals(5, response.getNumRowsIndexed()); + Assert.assertEquals(6, response.getData().size()); + + List data = removeEmptyColumns(response.getData()); + + assertEqualsSamplerResponseRow( + new SamplerResponseRow( + getRawColumns().get(0), + ImmutableMap.of("__time", 1555934400000L, "dim1", "foo", "met1", "1"), + null, + null + ), + data.get(0) + ); + assertEqualsSamplerResponseRow( + new SamplerResponseRow( + getRawColumns().get(1), + ImmutableMap.of("__time", 1555934400000L, "dim1", "foo", "met1", "2"), + null, + null + ), + data.get(1) + ); + assertEqualsSamplerResponseRow( + new SamplerResponseRow( + getRawColumns().get(2), + ImmutableMap.of("__time", 1555934460000L, "dim1", "foo", "met1", "3"), + null, + null + ), + data.get(2) + ); + assertEqualsSamplerResponseRow( + new SamplerResponseRow( + getRawColumns().get(3), + ImmutableMap.of("__time", 1555934400000L, "dim1", "foo2", "met1", "4"), + null, + null + ), + data.get(3) + ); + assertEqualsSamplerResponseRow( + new SamplerResponseRow( + getRawColumns().get(4), + ImmutableMap.of("__time", 1555934400000L, "dim1", "foo", "dim2", "bar", "met1", "5"), + null, + null + ), + data.get(4) + ); + assertEqualsSamplerResponseRow( + new SamplerResponseRow( + getRawColumns().get(5), + null, + true, + getUnparseableTimestampString() + ), + data.get(5) + ); + } + + @Test + public void testWithDimensionSpec() throws IOException + { + final TimestampSpec timestampSpec = new TimestampSpec("t", null, null); + final DimensionsSpec dimensionsSpec = new DimensionsSpec( + ImmutableList.of(StringDimensionSchema.create("dim1"), StringDimensionSchema.create("met1")) + ); + final DataSchema dataSchema = createDataSchema(timestampSpec, dimensionsSpec, null, null, null); + final InputSource inputSource = createInputSource(getTestRows(), dataSchema); + final InputFormat inputFormat = createInputFormat(); + + SamplerResponse response = inputSourceSampler.sample(inputSource, inputFormat, dataSchema, null); + + Assert.assertEquals(6, response.getNumRowsRead()); + Assert.assertEquals(5, response.getNumRowsIndexed()); + Assert.assertEquals(6, response.getData().size()); + + List data = response.getData(); + + assertEqualsSamplerResponseRow( + new SamplerResponseRow( + getRawColumns().get(0), + ImmutableMap.of("__time", 1555934400000L, "dim1", "foo", "met1", "1"), + null, + null + ), + data.get(0) + ); + assertEqualsSamplerResponseRow( + new SamplerResponseRow( + getRawColumns().get(1), + ImmutableMap.of("__time", 1555934400000L, "dim1", "foo", "met1", "2"), + null, + null + ), + data.get(1) + ); + assertEqualsSamplerResponseRow( + new SamplerResponseRow( + getRawColumns().get(2), + ImmutableMap.of("__time", 1555934460000L, "dim1", "foo", "met1", "3"), + null, + null + ), + data.get(2) + ); + assertEqualsSamplerResponseRow( + new SamplerResponseRow( + getRawColumns().get(3), + ImmutableMap.of("__time", 1555934400000L, "dim1", "foo2", "met1", "4"), + null, + null + ), + data.get(3) + ); + assertEqualsSamplerResponseRow( + new SamplerResponseRow( + getRawColumns().get(4), + ImmutableMap.of("__time", 1555934400000L, "dim1", "foo", "met1", "5"), + null, + null + ), + data.get(4) + ); + assertEqualsSamplerResponseRow( + new SamplerResponseRow( + getRawColumns().get(5), + null, + true, + getUnparseableTimestampString() + ), + data.get(5) + ); + } + + @Test + public void testWithNoRollup() throws IOException + { + final TimestampSpec timestampSpec = new TimestampSpec("t", null, null); + final DimensionsSpec dimensionsSpec = new DimensionsSpec(null); + final AggregatorFactory[] aggregatorFactories = {new LongSumAggregatorFactory("met1", "met1")}; + final GranularitySpec granularitySpec = new UniformGranularitySpec( + Granularities.DAY, + Granularities.HOUR, + false, + null + ); + final DataSchema dataSchema = createDataSchema( + timestampSpec, + dimensionsSpec, + aggregatorFactories, + granularitySpec, + null + ); + final InputSource inputSource = createInputSource(getTestRows(), dataSchema); + final InputFormat inputFormat = createInputFormat(); + + SamplerResponse response = inputSourceSampler.sample(inputSource, inputFormat, dataSchema, null); + + Assert.assertEquals(6, response.getNumRowsRead()); + Assert.assertEquals(5, response.getNumRowsIndexed()); + Assert.assertEquals(6, response.getData().size()); + + List data = removeEmptyColumns(response.getData()); + + assertEqualsSamplerResponseRow( + new SamplerResponseRow( + getRawColumns().get(0), + ImmutableMap.of("__time", 1555934400000L, "dim1", "foo", "met1", 1L), + null, + null + ), + data.get(0) + ); + assertEqualsSamplerResponseRow( + new SamplerResponseRow( + getRawColumns().get(1), + ImmutableMap.of("__time", 1555934400000L, "dim1", "foo", "met1", 2L), + null, + null + ), + data.get(1) + ); + assertEqualsSamplerResponseRow( + new SamplerResponseRow( + getRawColumns().get(2), + ImmutableMap.of("__time", 1555934400000L, "dim1", "foo", "met1", 3L), + null, + null + ), + data.get(2) + ); + assertEqualsSamplerResponseRow( + new SamplerResponseRow( + getRawColumns().get(3), + ImmutableMap.of("__time", 1555934400000L, "dim1", "foo2", "met1", 4L), + null, + null + ), + data.get(3) + ); + assertEqualsSamplerResponseRow( + new SamplerResponseRow( + getRawColumns().get(4), + ImmutableMap.of("__time", 1555934400000L, "dim1", "foo", "dim2", "bar", "met1", 5L), + null, + null + ), + data.get(4) + ); + assertEqualsSamplerResponseRow( + new SamplerResponseRow( + getRawColumns().get(5), + null, + true, + getUnparseableTimestampString() + ), + data.get(5) + ); + } + + @Test + public void testWithRollup() throws IOException + { + final TimestampSpec timestampSpec = new TimestampSpec("t", null, null); + final DimensionsSpec dimensionsSpec = new DimensionsSpec(null); + final AggregatorFactory[] aggregatorFactories = {new LongSumAggregatorFactory("met1", "met1")}; + final GranularitySpec granularitySpec = new UniformGranularitySpec( + Granularities.DAY, + Granularities.HOUR, + true, + null + ); + final DataSchema dataSchema = createDataSchema( + timestampSpec, + dimensionsSpec, + aggregatorFactories, + granularitySpec, + null + ); + final InputSource inputSource = createInputSource(getTestRows(), dataSchema); + final InputFormat inputFormat = createInputFormat(); + + SamplerResponse response = inputSourceSampler.sample(inputSource, inputFormat, dataSchema, null); + + Assert.assertEquals(6, response.getNumRowsRead()); + Assert.assertEquals(5, response.getNumRowsIndexed()); + Assert.assertEquals(4, response.getData().size()); + + List data = removeEmptyColumns(response.getData()); + + assertEqualsSamplerResponseRow( + new SamplerResponseRow( + getRawColumns().get(0), + ImmutableMap.of("__time", 1555934400000L, "dim1", "foo", "met1", 6L), + null, + null + ), + data.get(0) + ); + assertEqualsSamplerResponseRow( + new SamplerResponseRow( + getRawColumns().get(3), + ImmutableMap.of("__time", 1555934400000L, "dim1", "foo2", "met1", 4L), + null, + null + ), + data.get(1) + ); + assertEqualsSamplerResponseRow( + new SamplerResponseRow( + getRawColumns().get(4), + ImmutableMap.of("__time", 1555934400000L, "dim1", "foo", "dim2", "bar", "met1", 5L), + null, + null + ), + data.get(2) + ); + assertEqualsSamplerResponseRow( + new SamplerResponseRow( + getRawColumns().get(5), + null, + true, + getUnparseableTimestampString() + ), + data.get(3) + ); + } + + @Test + public void testWithMoreRollup() throws IOException + { + final TimestampSpec timestampSpec = new TimestampSpec("t", null, null); + final DimensionsSpec dimensionsSpec = new DimensionsSpec(ImmutableList.of(StringDimensionSchema.create("dim1"))); + final AggregatorFactory[] aggregatorFactories = {new LongSumAggregatorFactory("met1", "met1")}; + final GranularitySpec granularitySpec = new UniformGranularitySpec( + Granularities.DAY, + Granularities.HOUR, + true, + null + ); + final DataSchema dataSchema = createDataSchema( + timestampSpec, + dimensionsSpec, + aggregatorFactories, + granularitySpec, + null + ); + final InputSource inputSource = createInputSource(getTestRows(), dataSchema); + final InputFormat inputFormat = createInputFormat(); + + SamplerResponse response = inputSourceSampler.sample(inputSource, inputFormat, dataSchema, null); + + Assert.assertEquals(6, response.getNumRowsRead()); + Assert.assertEquals(5, response.getNumRowsIndexed()); + Assert.assertEquals(3, response.getData().size()); + + List data = response.getData(); + + assertEqualsSamplerResponseRow( + new SamplerResponseRow( + getRawColumns().get(0), + ImmutableMap.of("__time", 1555934400000L, "dim1", "foo", "met1", 11L), + null, + null + ), + data.get(0) + ); + assertEqualsSamplerResponseRow( + new SamplerResponseRow( + getRawColumns().get(3), + ImmutableMap.of("__time", 1555934400000L, "dim1", "foo2", "met1", 4L), + null, + null + ), + data.get(1) + ); + assertEqualsSamplerResponseRow( + new SamplerResponseRow( + getRawColumns().get(5), + null, + true, + getUnparseableTimestampString() + ), + data.get(2) + ); + } + + @Test + public void testWithTransformsAutoDimensions() throws IOException + { + final TimestampSpec timestampSpec = new TimestampSpec("t", null, null); + final DimensionsSpec dimensionsSpec = new DimensionsSpec(null); + final TransformSpec transformSpec = new TransformSpec( + null, + ImmutableList.of(new ExpressionTransform("dim1PlusBar", "concat(dim1, 'bar')", TestExprMacroTable.INSTANCE)) + ); + final AggregatorFactory[] aggregatorFactories = {new LongSumAggregatorFactory("met1", "met1")}; + final GranularitySpec granularitySpec = new UniformGranularitySpec( + Granularities.DAY, + Granularities.HOUR, + true, + null + ); + final DataSchema dataSchema = createDataSchema( + timestampSpec, + dimensionsSpec, + aggregatorFactories, + granularitySpec, + transformSpec + ); + final InputSource inputSource = createInputSource(getTestRows(), dataSchema); + final InputFormat inputFormat = createInputFormat(); + + SamplerResponse response = inputSourceSampler.sample(inputSource, inputFormat, dataSchema, null); + + Assert.assertEquals(6, response.getNumRowsRead()); + Assert.assertEquals(5, response.getNumRowsIndexed()); + Assert.assertEquals(4, response.getData().size()); + + List data = removeEmptyColumns(response.getData()); + + assertEqualsSamplerResponseRow( + new SamplerResponseRow( + getRawColumns().get(0), + ImmutableMap.of("__time", 1555934400000L, "dim1", "foo", "met1", 6L), + null, + null + ), + data.get(0) + ); + assertEqualsSamplerResponseRow( + new SamplerResponseRow( + getRawColumns().get(3), + ImmutableMap.of("__time", 1555934400000L, "dim1", "foo2", "met1", 4L), + null, + null + ), + data.get(1) + ); + assertEqualsSamplerResponseRow( + new SamplerResponseRow( + getRawColumns().get(4), + ImmutableMap.of("__time", 1555934400000L, "dim1", "foo", "dim2", "bar", "met1", 5L), + null, + null + ), + data.get(2) + ); + assertEqualsSamplerResponseRow( + new SamplerResponseRow( + getRawColumns().get(5), + null, + true, + getUnparseableTimestampString() + ), + data.get(3) + ); + } + + @Test + public void testWithTransformsDimensionsSpec() throws IOException + { + final TimestampSpec timestampSpec = new TimestampSpec("t", null, null); + final DimensionsSpec dimensionsSpec = new DimensionsSpec( + ImmutableList.of(StringDimensionSchema.create("dim1PlusBar")) + ); + final TransformSpec transformSpec = new TransformSpec( + null, + ImmutableList.of(new ExpressionTransform("dim1PlusBar", "concat(dim1 + 'bar')", TestExprMacroTable.INSTANCE)) + ); + final AggregatorFactory[] aggregatorFactories = {new LongSumAggregatorFactory("met1", "met1")}; + final GranularitySpec granularitySpec = new UniformGranularitySpec( + Granularities.DAY, + Granularities.HOUR, + true, + null + ); + final DataSchema dataSchema = createDataSchema( + timestampSpec, + dimensionsSpec, + aggregatorFactories, + granularitySpec, + transformSpec + ); + final InputSource inputSource = createInputSource(getTestRows(), dataSchema); + final InputFormat inputFormat = createInputFormat(); + + SamplerResponse response = inputSourceSampler.sample(inputSource, inputFormat, dataSchema, null); + + Assert.assertEquals(6, response.getNumRowsRead()); + Assert.assertEquals(5, response.getNumRowsIndexed()); + Assert.assertEquals(3, response.getData().size()); + + List data = response.getData(); + + assertEqualsSamplerResponseRow( + new SamplerResponseRow( + getRawColumns().get(0), + ImmutableMap.of("__time", 1555934400000L, "dim1PlusBar", "foobar", "met1", 11L), + null, + null + ), + data.get(0) + ); + assertEqualsSamplerResponseRow( + new SamplerResponseRow( + getRawColumns().get(3), + ImmutableMap.of("__time", 1555934400000L, "dim1PlusBar", "foo2bar", "met1", 4L), + null, + null + ), + data.get(1) + ); + assertEqualsSamplerResponseRow( + new SamplerResponseRow( + getRawColumns().get(5), + null, + true, + getUnparseableTimestampString() + ), + data.get(2) + ); + } + + @Test + public void testWithFilter() throws IOException + { + final TimestampSpec timestampSpec = new TimestampSpec("t", null, null); + final DimensionsSpec dimensionsSpec = new DimensionsSpec(null); + final TransformSpec transformSpec = new TransformSpec(new SelectorDimFilter("dim1", "foo", null), null); + final AggregatorFactory[] aggregatorFactories = {new LongSumAggregatorFactory("met1", "met1")}; + final GranularitySpec granularitySpec = new UniformGranularitySpec( + Granularities.DAY, + Granularities.HOUR, + true, + null + ); + final DataSchema dataSchema = createDataSchema( + timestampSpec, + dimensionsSpec, + aggregatorFactories, + granularitySpec, + transformSpec + ); + final InputSource inputSource = createInputSource(getTestRows(), dataSchema); + final InputFormat inputFormat = createInputFormat(); + + SamplerResponse response = inputSourceSampler.sample(inputSource, inputFormat, dataSchema, null); + + Assert.assertEquals(5, response.getNumRowsRead()); + Assert.assertEquals(4, response.getNumRowsIndexed()); + Assert.assertEquals(3, response.getData().size()); + + List data = removeEmptyColumns(response.getData()); + + assertEqualsSamplerResponseRow( + new SamplerResponseRow( + getRawColumns().get(0), + ImmutableMap.of("__time", 1555934400000L, "dim1", "foo", "met1", 6L), + null, + null + ), + data.get(0) + ); + assertEqualsSamplerResponseRow( + new SamplerResponseRow( + getRawColumns().get(4), + ImmutableMap.of("__time", 1555934400000L, "dim1", "foo", "dim2", "bar", "met1", 5L), + null, + null + ), + data.get(1) + ); + assertEqualsSamplerResponseRow( + new SamplerResponseRow( + getRawColumns().get(5), + null, + true, + getUnparseableTimestampString() + ), + data.get(2) + ); + } + + private List getTestRows() + { + switch (parserType) { + case STR_JSON: + return STR_JSON_ROWS; + case STR_CSV: + return STR_CSV_ROWS; + default: + throw new IAE("Unknown parser type: %s", parserType); + } + } + + private List> getRawColumns() + { + switch (parserType) { + case STR_JSON: + return mapOfRows.stream().map(this::removeEmptyValues).collect(Collectors.toList()); + case STR_CSV: + return mapOfRows; + default: + throw new IAE("Unknown parser type: %s", parserType); + } + } + + private InputFormat createInputFormat() + { + switch (parserType) { + case STR_JSON: + return new JsonInputFormat(null, null); + case STR_CSV: + return new CsvInputFormat(ImmutableList.of("t", "dim1", "dim2", "met1"), null, false, 0); + default: + throw new IAE("Unknown parser type: %s", parserType); + } + } + + private InputRowParser createInputRowParser(TimestampSpec timestampSpec, DimensionsSpec dimensionsSpec) + { + switch (parserType) { + case STR_JSON: + return new StringInputRowParser(new JSONParseSpec(timestampSpec, dimensionsSpec, null, null)); + case STR_CSV: + return new StringInputRowParser( + new DelimitedParseSpec( + timestampSpec, + dimensionsSpec, + ",", + null, + ImmutableList.of("t", "dim1", "dim2", "met1"), + false, + 0 + ) + ); + default: + throw new IAE("Unknown parser type: %s", parserType); + } + } + + private DataSchema createDataSchema( + @Nullable TimestampSpec timestampSpec, + @Nullable DimensionsSpec dimensionsSpec, + @Nullable AggregatorFactory[] aggregators, + @Nullable GranularitySpec granularitySpec, + @Nullable TransformSpec transformSpec + ) throws IOException + { + if (useInputFormatApi) { + return new DataSchema( + "sampler", + timestampSpec, + dimensionsSpec, + aggregators, + granularitySpec, + transformSpec + ); + } else { + final Map parserMap = getParserMap(createInputRowParser(timestampSpec, dimensionsSpec)); + return new DataSchema( + "sampler", + parserMap, + aggregators, + granularitySpec, + transformSpec, + OBJECT_MAPPER + ); + } + } + + private Map getParserMap(InputRowParser parser) throws IOException + { + if (useInputFormatApi) { + throw new RuntimeException("Don't call this if useInputFormatApi = true"); + } + return OBJECT_MAPPER.readValue(OBJECT_MAPPER.writeValueAsBytes(parser), Map.class); + } + + private InputSource createInputSource(List rows, DataSchema dataSchema) + { + final String data = String.join("\n", rows); + if (useInputFormatApi) { + return new InlineInputSource(data); + } else { + return new FirehoseFactoryToInputSourceAdaptor( + new InlineFirehoseFactory(data), + createInputRowParser( + dataSchema == null ? new TimestampSpec(null, null, null) : dataSchema.getTimestampSpec(), + dataSchema == null ? new DimensionsSpec(null) : dataSchema.getDimensionsSpec() + ) + ); + } + } + + private String getUnparseableTimestampString() + { + return ParserType.STR_CSV.equals(parserType) + ? (USE_DEFAULT_VALUE_FOR_NULL + ? "Unparseable timestamp found! Event: {t=bad_timestamp, dim1=foo, dim2=null, met1=6}" + : "Unparseable timestamp found! Event: {t=bad_timestamp, dim1=foo, dim2=, met1=6}") + : "Unparseable timestamp found! Event: {t=bad_timestamp, dim1=foo, met1=6}"; + } + + private String unparseableTimestampErrorString(Map rawColumns) + { + return StringUtils.format("Unparseable timestamp found! Event: %s", rawColumns); + } + + private List removeEmptyColumns(List rows) + { + return USE_DEFAULT_VALUE_FOR_NULL + ? rows + : rows.stream().map(x -> x.withParsed(removeEmptyValues(x.getParsed()))).collect(Collectors.toList()); + } + + @Nullable + private Map removeEmptyValues(Map data) + { + return data == null + ? null : data.entrySet() + .stream() + .filter(x -> x.getValue() != null) + .filter(x -> !(x.getValue() instanceof String) || !((String) x.getValue()).isEmpty()) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + } + + private static void assertEqualsSamplerResponseRow(SamplerResponseRow row1, SamplerResponseRow row2) + { + Assert.assertTrue(equalsIgnoringType(row1.getInput(), row2.getInput())); + Assert.assertEquals(row1.getParsed(), row2.getParsed()); + Assert.assertEquals(row1.getError(), row2.getError()); + Assert.assertEquals(row1.isUnparseable(), row2.isUnparseable()); + } + + private static boolean equalsIgnoringType(Map map1, Map map2) + { + for (Entry entry1 : map1.entrySet()) { + final Object val1 = entry1.getValue(); + final Object val2 = map2.get(entry1.getKey()); + if (!equalsStringOrInteger(val1, val2)) { + return false; + } + } + return true; + } + + private static boolean equalsStringOrInteger(Object val1, Object val2) + { + if (val1 == null || val2 == null) { + return val1 == val2; + } else if (val1.equals(val2)) { + return true; + } else { + if (val1 instanceof Number || val2 instanceof Number) { + final Integer int1, int2; + if (val1 instanceof String) { + int1 = Integer.parseInt((String) val1); + } else if (val1 instanceof Number) { + int1 = ((Number) val1).intValue(); + } else { + int1 = null; + } + + if (val2 instanceof String) { + int2 = Integer.parseInt((String) val2); + } else if (val2 instanceof Number) { + int2 = ((Number) val2).intValue(); + } else { + int2 = null; + } + + return Objects.equals(int1, int2); + } + } + + return false; + } +} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/SamplerCacheTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/SamplerCacheTest.java deleted file mode 100644 index c1d82c98c6d3..000000000000 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/SamplerCacheTest.java +++ /dev/null @@ -1,177 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.indexing.overlord.sampler; - -import com.google.common.collect.ImmutableList; -import com.google.common.io.Files; -import org.apache.commons.io.FileUtils; -import org.apache.druid.client.cache.MapCache; -import org.apache.druid.data.input.Firehose; -import org.apache.druid.data.input.FirehoseFactory; -import org.apache.druid.data.input.InputRow; -import org.apache.druid.data.input.InputRowListPlusJson; -import org.apache.druid.data.input.impl.CSVParseSpec; -import org.apache.druid.data.input.impl.DimensionsSpec; -import org.apache.druid.data.input.impl.StringInputRowParser; -import org.apache.druid.data.input.impl.TimestampSpec; -import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.java.util.common.StringUtils; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - -import java.io.File; -import java.io.IOException; -import java.nio.charset.StandardCharsets; - -public class SamplerCacheTest -{ - private static final String KEY_1 = "abcdefghijklmnopqrstuvwxyz"; - private static final String KEY_2 = "1234567890!@#$%^&*()"; - - private static final byte[] VALUE_1_1 = StringUtils.toUtf8("The quick"); - private static final byte[] VALUE_1_2 = StringUtils.toUtf8("brown fox"); - private static final byte[] VALUE_1_3 = StringUtils.toUtf8("jumps over"); - private static final byte[] VALUE_2_1 = StringUtils.toUtf8("the lazy"); - private static final byte[] VALUE_2_2 = StringUtils.toUtf8("Druid"); - - private static final StringInputRowParser PARSER = new StringInputRowParser( - new CSVParseSpec( - new TimestampSpec(null, null, DateTimes.of("1970")), - new DimensionsSpec(null), - null, - ImmutableList.of("col"), - false, - 0 - ), - StandardCharsets.UTF_8.name() - ); - - private SamplerCache cache; - private File tempDir; - - @Before - public void setupTest() - { - cache = new SamplerCache(MapCache.create(100000)); - tempDir = Files.createTempDir(); - } - - @After - public void teardownTest() throws IOException - { - FileUtils.deleteDirectory(tempDir); - } - - @Test - public void testOneEntryNextRowWithRaw() throws IOException - { - cache.put(KEY_1, ImmutableList.of(VALUE_1_1, VALUE_1_2, VALUE_1_3)); - - for (int i = 0; i < 4; i++) { - Firehose firehose1 = cache.getAsFirehoseFactory(KEY_1, PARSER).connectForSampler(PARSER, tempDir); - - Assert.assertTrue(firehose1.hasMore()); - - InputRowListPlusJson row = firehose1.nextRowWithRaw(); - Assert.assertArrayEquals(VALUE_1_1, row.getRaw()); - Assert.assertEquals("The quick", row.getInputRow().getDimension("col").get(0)); - row = firehose1.nextRowWithRaw(); - Assert.assertArrayEquals(VALUE_1_2, row.getRaw()); - Assert.assertEquals("brown fox", row.getInputRow().getDimension("col").get(0)); - row = firehose1.nextRowWithRaw(); - Assert.assertArrayEquals(VALUE_1_3, row.getRaw()); - Assert.assertEquals("jumps over", row.getInputRow().getDimension("col").get(0)); - - Assert.assertFalse(firehose1.hasMore()); - - firehose1.close(); - - if (i % 2 == 1) { - FirehoseFactory firehoseFactory2 = cache.getAsFirehoseFactory(KEY_2, PARSER); - Assert.assertNull(firehoseFactory2); - } - } - } - - @Test - public void testOneEntryNextRow() throws IOException - { - cache.put(KEY_1, ImmutableList.of(VALUE_1_1, VALUE_1_2, VALUE_1_3)); - - Firehose firehose = cache.getAsFirehoseFactory(KEY_1, PARSER).connectForSampler(PARSER, tempDir); - - Assert.assertTrue(firehose.hasMore()); - - InputRow row = firehose.nextRow(); - Assert.assertEquals("The quick", row.getDimension("col").get(0)); - row = firehose.nextRow(); - Assert.assertEquals("brown fox", row.getDimension("col").get(0)); - row = firehose.nextRow(); - Assert.assertEquals("jumps over", row.getDimension("col").get(0)); - - Assert.assertFalse(firehose.hasMore()); - - firehose.close(); - } - - @Test - public void testTwoEntriesNextRowWithRaw() throws IOException - { - cache.put(KEY_1, ImmutableList.of(VALUE_1_1, VALUE_1_2, VALUE_1_3)); - cache.put(KEY_2, ImmutableList.of(VALUE_2_1, VALUE_2_2)); - - for (int i = 0; i < 4; i++) { - Firehose firehose1 = cache.getAsFirehoseFactory(KEY_1, PARSER).connectForSampler(PARSER, tempDir); - - Assert.assertTrue(firehose1.hasMore()); - - InputRowListPlusJson row = firehose1.nextRowWithRaw(); - Assert.assertArrayEquals(VALUE_1_1, row.getRaw()); - Assert.assertEquals("The quick", row.getInputRow().getDimension("col").get(0)); - row = firehose1.nextRowWithRaw(); - Assert.assertArrayEquals(VALUE_1_2, row.getRaw()); - Assert.assertEquals("brown fox", row.getInputRow().getDimension("col").get(0)); - row = firehose1.nextRowWithRaw(); - Assert.assertArrayEquals(VALUE_1_3, row.getRaw()); - Assert.assertEquals("jumps over", row.getInputRow().getDimension("col").get(0)); - - Assert.assertFalse(firehose1.hasMore()); - - firehose1.close(); - - Firehose firehose2 = cache.getAsFirehoseFactory(KEY_2, PARSER).connectForSampler(PARSER, tempDir); - - Assert.assertTrue(firehose2.hasMore()); - - row = firehose2.nextRowWithRaw(); - Assert.assertArrayEquals(VALUE_2_1, row.getRaw()); - Assert.assertEquals("the lazy", row.getInputRow().getDimension("col").get(0)); - row = firehose2.nextRowWithRaw(); - Assert.assertArrayEquals(VALUE_2_2, row.getRaw()); - Assert.assertEquals("Druid", row.getInputRow().getDimension("col").get(0)); - - Assert.assertFalse(firehose2.hasMore()); - - firehose2.close(); - } - } -} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/SamplerResponseTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/SamplerResponseTest.java index e0dca58497a4..7cdcf42b3544 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/SamplerResponseTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/SamplerResponseTest.java @@ -38,22 +38,22 @@ public void testSerde() throws IOException { List data = ImmutableList.of( new SamplerResponse.SamplerResponseRow( - "parsed1", + ImmutableMap.of("row1", "val1"), ImmutableMap.of("t", 123456, "dim1", "foo", "met1", 6), null, null ), new SamplerResponse.SamplerResponseRow( - "parsed2", + ImmutableMap.of("row2", "val2"), ImmutableMap.of("t", 123457, "dim1", "foo2", "met1", 7), null, null ), - new SamplerResponse.SamplerResponseRow("unparsed", null, true, "Could not parse") + new SamplerResponse.SamplerResponseRow(ImmutableMap.of("row3", "val3"), null, true, "Could not parse") ); - String out = MAPPER.writeValueAsString(new SamplerResponse("eaebbfd87ec34bc6a9f8c03ecee4dd7a", 1123, 1112, data)); - String expected = "{\"cacheKey\":\"eaebbfd87ec34bc6a9f8c03ecee4dd7a\",\"numRowsRead\":1123,\"numRowsIndexed\":1112,\"data\":[{\"raw\":\"parsed1\",\"parsed\":{\"t\":123456,\"dim1\":\"foo\",\"met1\":6}},{\"raw\":\"parsed2\",\"parsed\":{\"t\":123457,\"dim1\":\"foo2\",\"met1\":7}},{\"raw\":\"unparsed\",\"unparseable\":true,\"error\":\"Could not parse\"}]}"; + String out = MAPPER.writeValueAsString(new SamplerResponse(1123, 1112, data)); + String expected = "{\"numRowsRead\":1123,\"numRowsIndexed\":1112,\"data\":[{\"input\":{\"row1\":\"val1\"},\"parsed\":{\"t\":123456,\"dim1\":\"foo\",\"met1\":6}},{\"input\":{\"row2\":\"val2\"},\"parsed\":{\"t\":123457,\"dim1\":\"foo2\",\"met1\":7}},{\"input\":{\"row3\":\"val3\"},\"unparseable\":true,\"error\":\"Could not parse\"}]}"; Assert.assertEquals(expected, out); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/RecordSupplierInputSourceTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/RecordSupplierInputSourceTest.java new file mode 100644 index 000000000000..fbd6a91bb872 --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/RecordSupplierInputSourceTest.java @@ -0,0 +1,219 @@ +/* + * 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.druid.indexing.seekablestream; + +import org.apache.commons.lang3.RandomStringUtils; +import org.apache.druid.data.input.InputFormat; +import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.InputRowSchema; +import org.apache.druid.data.input.InputSource; +import org.apache.druid.data.input.InputSourceReader; +import org.apache.druid.data.input.impl.CsvInputFormat; +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; +import org.apache.druid.indexing.seekablestream.common.RecordSupplier; +import org.apache.druid.indexing.seekablestream.common.StreamPartition; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.parsers.CloseableIterator; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import javax.annotation.Nullable; +import javax.validation.constraints.NotNull; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Random; +import java.util.Set; +import java.util.concurrent.ThreadLocalRandom; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +public class RecordSupplierInputSourceTest +{ + private static final int NUM_COLS = 16; + private static final int NUM_ROWS = 128; + private static final String TIMESTAMP_STRING = "2019-01-01"; + + @Rule + public TemporaryFolder temporaryFolder = new TemporaryFolder(); + + @Test + public void testRead() throws IOException + { + final RandomCsvSupplier supplier = new RandomCsvSupplier(); + final InputSource inputSource = new RecordSupplierInputSource<>("topic", supplier, false); + final List colNames = IntStream.range(0, NUM_COLS) + .mapToObj(i -> StringUtils.format("col_%d", i)) + .collect(Collectors.toList()); + final InputFormat inputFormat = new CsvInputFormat(colNames, null, false, 0); + final InputSourceReader reader = inputSource.reader( + new InputRowSchema( + new TimestampSpec("col_0", "auto", null), + new DimensionsSpec(DimensionsSpec.getDefaultSchemas(colNames.subList(1, colNames.size()))), + Collections.emptyList() + ), + inputFormat, + temporaryFolder.newFolder() + ); + + int read = 0; + try (CloseableIterator iterator = reader.read()) { + for (; read < NUM_ROWS && iterator.hasNext(); read++) { + final InputRow inputRow = iterator.next(); + Assert.assertEquals(DateTimes.of(TIMESTAMP_STRING), inputRow.getTimestamp()); + Assert.assertEquals(NUM_COLS - 1, inputRow.getDimensions().size()); + } + } + + Assert.assertEquals(NUM_ROWS, read); + Assert.assertTrue(supplier.isClosed()); + } + + private static class RandomCsvSupplier implements RecordSupplier + { + private static final int STR_LEN = 8; + + private final Random random = ThreadLocalRandom.current(); + private final Map partitionToOffset; + + private volatile boolean closed = false; + + private RandomCsvSupplier() + { + partitionToOffset = new HashMap<>(3); + for (int i = 0; i < 3; i++) { + partitionToOffset.put(i, 0); + } + } + + @Override + public void assign(Set> streamPartitions) + { + // do nothing + } + + @Override + public void seekToEarliest(Set> streamPartitions) + { + // do nothing + } + + @Override + public void seekToLatest(Set> streamPartitions) + { + // do nothing + } + + @NotNull + @Override + public List> poll(long timeout) + { + final long sleepTime = random.nextInt((int) timeout); + try { + Thread.sleep(sleepTime); + } + catch (InterruptedException e) { + throw new RuntimeException(e); + } + if (sleepTime == timeout) { + return Collections.emptyList(); + } else { + final int numRecords = random.nextInt(8); // can be 0 + final List> records = new ArrayList<>(numRecords); + for (int i = 0; i < numRecords; i++) { + final int partitionId = random.nextInt(partitionToOffset.size()); + final int offset = partitionToOffset.get(partitionId); + final int numBytes = random.nextInt(3); // can be 0 + final List bytes = IntStream + .range(0, numBytes) + .mapToObj(j -> { + final List columns = new ArrayList<>(NUM_COLS); + columns.add(TIMESTAMP_STRING); // timestamp + for (int k = 0; k < NUM_COLS - 1; k++) { + columns.add(RandomStringUtils.random(STR_LEN, true, false)); + } + return StringUtils.toUtf8(String.join(",", columns)); + }) + .collect(Collectors.toList()); + records.add(new OrderedPartitionableRecord<>("topic", partitionId, offset, bytes)); + } + return records; + } + } + + @Override + public Set getPartitionIds(String stream) + { + return partitionToOffset.keySet(); + } + + @Override + public void close() + { + closed = true; + } + + boolean isClosed() + { + return closed; + } + + @Override + public void seek(StreamPartition partition, Integer sequenceNumber) + { + throw new UnsupportedOperationException(); + } + + @Override + public Collection> getAssignment() + { + throw new UnsupportedOperationException(); + } + + @Nullable + @Override + public Integer getLatestSequenceNumber(StreamPartition partition) + { + throw new UnsupportedOperationException(); + } + + @Nullable + @Override + public Integer getEarliestSequenceNumber(StreamPartition partition) + { + throw new UnsupportedOperationException(); + } + + @Override + public Integer getPosition(StreamPartition partition) + { + throw new UnsupportedOperationException(); + } + } +} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java index b8c2715b1384..2d1b9a4078dc 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java @@ -19,8 +19,11 @@ package org.apache.druid.indexing.seekablestream; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.jsontype.NamedType; import com.google.common.base.Predicates; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; @@ -29,9 +32,11 @@ import com.google.common.io.Files; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListeningExecutorService; +import org.apache.druid.data.input.InputFormat; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.FloatDimensionSchema; import org.apache.druid.data.input.impl.JSONParseSpec; +import org.apache.druid.data.input.impl.JsonInputFormat; import org.apache.druid.data.input.impl.LongDimensionSchema; import org.apache.druid.data.input.impl.StringDimensionSchema; import org.apache.druid.data.input.impl.StringInputRowParser; @@ -81,6 +86,7 @@ import org.joda.time.Interval; import org.junit.Assert; +import javax.annotation.Nullable; import java.io.File; import java.io.IOException; import java.lang.reflect.InvocationTargetException; @@ -97,38 +103,32 @@ public class SeekableStreamIndexTaskTestBase extends EasyMockSupport { - protected static final ObjectMapper OBJECT_MAPPER = new TestUtils().getTestObjectMapper(); - protected static final DataSchema DATA_SCHEMA = new DataSchema( + protected static final ObjectMapper OBJECT_MAPPER; + protected static final DataSchema OLD_DATA_SCHEMA; + protected static final DataSchema NEW_DATA_SCHEMA = new DataSchema( "test_ds", - OBJECT_MAPPER.convertValue( - new StringInputRowParser( - new JSONParseSpec( - new TimestampSpec("timestamp", "iso", null), - new DimensionsSpec( - Arrays.asList( - new StringDimensionSchema("dim1"), - new StringDimensionSchema("dim1t"), - new StringDimensionSchema("dim2"), - new LongDimensionSchema("dimLong"), - new FloatDimensionSchema("dimFloat") - ), - null, - null - ), - new JSONPathSpec(true, ImmutableList.of()), - ImmutableMap.of() - ), - StandardCharsets.UTF_8.name() + new TimestampSpec("timestamp", "iso", null), + new DimensionsSpec( + Arrays.asList( + new StringDimensionSchema("dim1"), + new StringDimensionSchema("dim1t"), + new StringDimensionSchema("dim2"), + new LongDimensionSchema("dimLong"), + new FloatDimensionSchema("dimFloat") ), - Map.class + null, + null ), new AggregatorFactory[]{ new DoubleSumAggregatorFactory("met1sum", "met1"), new CountAggregatorFactory("rows") }, new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null), - null, - OBJECT_MAPPER + null + ); + protected static final InputFormat INPUT_FORMAT = new JsonInputFormat( + new JSONPathSpec(true, ImmutableList.of()), + ImmutableMap.of() ); protected static final Logger LOG = new Logger(SeekableStreamIndexTaskTestBase.class); protected static ListeningExecutorService taskExec; @@ -142,6 +142,43 @@ public class SeekableStreamIndexTaskTestBase extends EasyMockSupport protected TaskLockbox taskLockbox; protected IndexerMetadataStorageCoordinator metadataStorageCoordinator; + static { + OBJECT_MAPPER = new TestUtils().getTestObjectMapper(); + OBJECT_MAPPER.registerSubtypes(new NamedType(UnimplementedInputFormatJsonParseSpec.class, "json")); + OLD_DATA_SCHEMA = new DataSchema( + "test_ds", + OBJECT_MAPPER.convertValue( + new StringInputRowParser( + new UnimplementedInputFormatJsonParseSpec( + new TimestampSpec("timestamp", "iso", null), + new DimensionsSpec( + Arrays.asList( + new StringDimensionSchema("dim1"), + new StringDimensionSchema("dim1t"), + new StringDimensionSchema("dim2"), + new LongDimensionSchema("dimLong"), + new FloatDimensionSchema("dimFloat") + ), + null, + null + ), + new JSONPathSpec(true, ImmutableList.of()), + ImmutableMap.of() + ), + StandardCharsets.UTF_8.name() + ), + Map.class + ), + new AggregatorFactory[]{ + new DoubleSumAggregatorFactory("met1sum", "met1"), + new CountAggregatorFactory("rows") + }, + new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null), + null, + OBJECT_MAPPER + ); + } + public SeekableStreamIndexTaskTestBase( LockGranularity lockGranularity ) @@ -198,7 +235,7 @@ protected List readSegmentColumn(final String column, final SegmentDescr StringUtils.format( "%s/%s/%s_%s/%s/%d", getSegmentDirectory(), - DATA_SCHEMA.getDataSource(), + OLD_DATA_SCHEMA.getDataSource(), descriptor.getInterval().getStart(), descriptor.getInterval().getEnd(), descriptor.getVersion(), @@ -340,7 +377,7 @@ protected long countEvents(final Task task) { // Do a query. TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() - .dataSource(DATA_SCHEMA.getDataSource()) + .dataSource(OLD_DATA_SCHEMA.getDataSource()) .aggregators( ImmutableList.of( new LongSumAggregatorFactory("rows", "rows") @@ -367,7 +404,7 @@ protected void unlockAppenderatorBasePersistDirForTask(SeekableStreamIndexTask t protected List publishedDescriptors() { return metadataStorageCoordinator - .getUsedSegmentsForInterval(DATA_SCHEMA.getDataSource(), Intervals.of("0000/3000"), Segments.ONLY_VISIBLE) + .getUsedSegmentsForInterval(OLD_DATA_SCHEMA.getDataSource(), Intervals.of("0000/3000"), Segments.ONLY_VISIBLE) .stream() .map(DataSegment::toDescriptor) .collect(Collectors.toList()); @@ -401,4 +438,25 @@ public SegmentDescriptor getSegmentDescriptor() return segmentDescriptor; } } + + private static class UnimplementedInputFormatJsonParseSpec extends JSONParseSpec + { + @JsonCreator + private UnimplementedInputFormatJsonParseSpec( + @JsonProperty("timestampSpec") TimestampSpec timestampSpec, + @JsonProperty("dimensionsSpec") DimensionsSpec dimensionsSpec, + @JsonProperty("flattenSpec") JSONPathSpec flattenSpec, + @JsonProperty("featureSpec") Map featureSpec + ) + { + super(timestampSpec, dimensionsSpec, flattenSpec, featureSpec); + } + + @Nullable + @Override + public InputFormat toInputFormat() + { + return null; + } + } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateTest.java index 9d22d37178cd..3dcd6d9f9a03 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateTest.java @@ -26,9 +26,8 @@ import com.google.common.collect.ImmutableSet; import org.apache.druid.data.input.impl.DimensionSchema; import org.apache.druid.data.input.impl.DimensionsSpec; -import org.apache.druid.data.input.impl.JSONParseSpec; +import org.apache.druid.data.input.impl.JsonInputFormat; import org.apache.druid.data.input.impl.StringDimensionSchema; -import org.apache.druid.data.input.impl.StringInputRowParser; import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.indexing.common.TestUtils; import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; @@ -81,7 +80,6 @@ import javax.annotation.Nullable; import java.io.File; import java.math.BigInteger; -import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.List; import java.util.Map; @@ -561,21 +559,11 @@ private static DataSchema getDataSchema() return new DataSchema( DATASOURCE, - OBJECT_MAPPER.convertValue( - new StringInputRowParser( - new JSONParseSpec( - new TimestampSpec("timestamp", "iso", null), - new DimensionsSpec( - dimensions, - null, - null - ), - new JSONPathSpec(true, ImmutableList.of()), - ImmutableMap.of() - ), - StandardCharsets.UTF_8.name() - ), - Map.class + new TimestampSpec("timestamp", "iso", null), + new DimensionsSpec( + dimensions, + null, + null ), new AggregatorFactory[]{new CountAggregatorFactory("rows")}, new UniformGranularitySpec( @@ -583,8 +571,7 @@ private static DataSchema getDataSchema() Granularities.NONE, ImmutableList.of() ), - null, - OBJECT_MAPPER + null ); } @@ -592,6 +579,7 @@ private static SeekableStreamSupervisorIOConfig getIOConfig() { return new SeekableStreamSupervisorIOConfig( "stream", + new JsonInputFormat(new JSONPathSpec(true, ImmutableList.of()), ImmutableMap.of()), 1, 1, new Period("PT1H"), @@ -786,7 +774,10 @@ protected SeekableStreamIndexTaskIOConfig createTaskIoConfig( new SeekableStreamEndSequenceNumbers<>(STREAM, endPartitions), true, minimumMessageTime, - maximumMessageTime + maximumMessageTime, + ioConfig.getInputFormat( + getDataSchema().getParser() == null ? null : getDataSchema().getParser().getParseSpec() + ) ) { }; diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractKafkaIndexerTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractKafkaIndexerTest.java index 58a91becc523..04b3a9f74996 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractKafkaIndexerTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractKafkaIndexerTest.java @@ -55,7 +55,8 @@ abstract class AbstractKafkaIndexerTest extends AbstractIndexerTest { private static final Logger LOG = new Logger(AbstractKafkaIndexerTest.class); - private static final String INDEXER_FILE = "/indexer/kafka_supervisor_spec.json"; + protected static final String INDEXER_FILE_LEGACY_PARSER = "/indexer/kafka_supervisor_spec_legacy_parser.json"; + protected static final String INDEXER_FILE_INPUT_FORMAT = "/indexer/kafka_supervisor_spec_input_format.json"; private static final String QUERIES_FILE = "/indexer/kafka_index_queries.json"; private static final String TOPIC_NAME = "kafka_indexing_service_topic"; @@ -98,7 +99,7 @@ abstract class AbstractKafkaIndexerTest extends AbstractIndexerTest private String fullDatasourceName; - void doKafkaIndexTest(String dataSourceName, boolean txnEnabled) + void doKafkaIndexTest(String dataSourceName, String supervisorSpecPath, boolean txnEnabled) { fullDatasourceName = dataSourceName + config.getExtraDatasourceNameSuffix(); // create topic @@ -128,21 +129,21 @@ void doKafkaIndexTest(String dataSourceName, boolean txnEnabled) String spec; try { - LOG.info("supervisorSpec name: [%s]", INDEXER_FILE); + LOG.info("supervisorSpec name: [%s]", supervisorSpecPath); final Map consumerConfigs = KafkaConsumerConfigs.getConsumerProperties(); final Properties consumerProperties = new Properties(); consumerProperties.putAll(consumerConfigs); consumerProperties.setProperty("bootstrap.servers", config.getKafkaInternalHost()); - spec = getResourceAsString(INDEXER_FILE); + spec = getResourceAsString(supervisorSpecPath); spec = StringUtils.replace(spec, "%%DATASOURCE%%", fullDatasourceName); spec = StringUtils.replace(spec, "%%TOPIC%%", TOPIC_NAME); spec = StringUtils.replace(spec, "%%CONSUMER_PROPERTIES%%", jsonMapper.writeValueAsString(consumerProperties)); LOG.info("supervisorSpec: [%s]\n", spec); } catch (Exception e) { - LOG.error("could not read file [%s]", INDEXER_FILE); - throw new ISE(e, "could not read file [%s]", INDEXER_FILE); + LOG.error("could not read file [%s]", supervisorSpecPath); + throw new ISE(e, "could not read file [%s]", supervisorSpecPath); } // start supervisor diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKafkaIndexingServiceTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKafkaIndexingServiceTest.java index bf74eea41490..30e4bab23e1c 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKafkaIndexingServiceTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKafkaIndexingServiceTest.java @@ -19,10 +19,12 @@ package org.apache.druid.tests.indexer; +import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.testing.guice.DruidTestModuleFactory; import org.apache.druid.tests.TestNGGroup; -import org.testng.annotations.AfterClass; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.DataProvider; import org.testng.annotations.Guice; import org.testng.annotations.Test; @@ -33,14 +35,26 @@ public class ITKafkaIndexingServiceTest extends AbstractKafkaIndexerTest private static final Logger LOG = new Logger(ITKafkaIndexingServiceTest.class); private static final String DATASOURCE = "kafka_indexing_service_test"; - @Test - public void testKafka() + @DataProvider + public static Object[][] testParams() { + return new Object[][]{ + {"legacy_parser"}, + {"input_format"} + }; + } + + @Test(dataProvider = "testParams") + public void testKafka(String param) + { + final String supervisorSpecPath = "legacy_parser".equals(param) + ? INDEXER_FILE_LEGACY_PARSER + : INDEXER_FILE_INPUT_FORMAT; LOG.info("Starting test: ITKafkaIndexingServiceTest"); - doKafkaIndexTest(DATASOURCE, false); + doKafkaIndexTest(StringUtils.format("%s_%s", DATASOURCE, param), supervisorSpecPath, false); } - @AfterClass + @AfterMethod public void afterClass() { LOG.info("teardown"); diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKafkaIndexingServiceTransactionalTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKafkaIndexingServiceTransactionalTest.java index 7424561db081..04c52b2f97d7 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKafkaIndexingServiceTransactionalTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITKafkaIndexingServiceTransactionalTest.java @@ -19,10 +19,12 @@ package org.apache.druid.tests.indexer; +import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.testing.guice.DruidTestModuleFactory; import org.apache.druid.tests.TestNGGroup; -import org.testng.annotations.AfterClass; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.DataProvider; import org.testng.annotations.Guice; import org.testng.annotations.Test; @@ -36,14 +38,26 @@ public class ITKafkaIndexingServiceTransactionalTest extends AbstractKafkaIndexe private static final Logger LOG = new Logger(ITKafkaIndexingServiceTransactionalTest.class); private static final String DATASOURCE = "kafka_indexing_service_txn_test"; - @Test - public void testKafka() + @DataProvider + public static Object[][] testParams() { + return new Object[][]{ + {"legacy_parser"}, + {"input_format"} + }; + } + + @Test(dataProvider = "testParams") + public void testKafka(String param) + { + final String supervisorSpecPath = "legacy_parser".equals(param) + ? INDEXER_FILE_LEGACY_PARSER + : INDEXER_FILE_INPUT_FORMAT; LOG.info("Starting test: ITKafkaIndexingServiceTransactionalTest"); - doKafkaIndexTest(DATASOURCE, true); + doKafkaIndexTest(StringUtils.format("%s_%s", DATASOURCE, param), supervisorSpecPath, false); } - @AfterClass + @AfterMethod public void afterClass() { LOG.info("teardown"); diff --git a/integration-tests/src/test/resources/indexer/kafka_supervisor_spec_input_format.json b/integration-tests/src/test/resources/indexer/kafka_supervisor_spec_input_format.json new file mode 100644 index 000000000000..4ba59afdcac7 --- /dev/null +++ b/integration-tests/src/test/resources/indexer/kafka_supervisor_spec_input_format.json @@ -0,0 +1,58 @@ +{ + "type": "kafka", + "dataSchema": { + "dataSource": "%%DATASOURCE%%", + "timestampSpec": { + "column": "timestamp", + "format": "auto" + }, + "dimensionsSpec": { + "dimensions": ["page", "language", "user", "unpatrolled", "newPage", "robot", "anonymous", "namespace", "continent", "country", "region", "city"], + "dimensionExclusions": [], + "spatialDimensions": [] + }, + "metricsSpec": [ + { + "type": "count", + "name": "count" + }, + { + "type": "doubleSum", + "name": "added", + "fieldName": "added" + }, + { + "type": "doubleSum", + "name": "deleted", + "fieldName": "deleted" + }, + { + "type": "doubleSum", + "name": "delta", + "fieldName": "delta" + } + ], + "granularitySpec": { + "type": "uniform", + "segmentGranularity": "MINUTE", + "queryGranularity": "NONE" + } + }, + "tuningConfig": { + "type": "kafka", + "intermediatePersistPeriod": "PT30S", + "maxRowsPerSegment": 5000000, + "maxRowsInMemory": 500000 + }, + "ioConfig": { + "topic": "%%TOPIC%%", + "consumerProperties": %%CONSUMER_PROPERTIES%%, + "taskCount": 2, + "replicas": 1, + "taskDuration": "PT2M", + "useEarliestOffset": true, + "inputFormat" : { + "type" : "json" + } + } +} diff --git a/integration-tests/src/test/resources/indexer/kafka_supervisor_spec.json b/integration-tests/src/test/resources/indexer/kafka_supervisor_spec_legacy_parser.json similarity index 100% rename from integration-tests/src/test/resources/indexer/kafka_supervisor_spec.json rename to integration-tests/src/test/resources/indexer/kafka_supervisor_spec_legacy_parser.json diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndex.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndex.java index 5f6c8fce07cb..6d07cf2c88a7 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndex.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndex.java @@ -429,7 +429,7 @@ public Builder setMaxBytesInMemory(final long maxBytesInMemory) return this; } - public IncrementalIndex buildOnheap() + public OnheapIncrementalIndex buildOnheap() { if (maxRowCount <= 0) { throw new IllegalArgumentException("Invalid max row count: " + maxRowCount); diff --git a/processing/src/main/java/org/apache/druid/segment/transform/TransformSpec.java b/processing/src/main/java/org/apache/druid/segment/transform/TransformSpec.java index c5418609edca..3eae123d7469 100644 --- a/processing/src/main/java/org/apache/druid/segment/transform/TransformSpec.java +++ b/processing/src/main/java/org/apache/druid/segment/transform/TransformSpec.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.ImmutableList; +import org.apache.druid.data.input.InputEntityReader; import org.apache.druid.data.input.InputSourceReader; import org.apache.druid.data.input.impl.InputRowParser; import org.apache.druid.data.input.impl.StringInputRowParser; @@ -114,7 +115,12 @@ public InputRowParser decorate(final InputRowParser parser) public InputSourceReader decorate(InputSourceReader reader) { - return new TransformingReader(reader, toTransformer()); + return new TransformingInputSourceReader(reader, toTransformer()); + } + + public InputEntityReader decorate(InputEntityReader reader) + { + return new TransformingInputEntityReader(reader, toTransformer()); } /** diff --git a/processing/src/main/java/org/apache/druid/segment/transform/Transformer.java b/processing/src/main/java/org/apache/druid/segment/transform/Transformer.java index ce34da044879..cbde1d0e9c6b 100644 --- a/processing/src/main/java/org/apache/druid/segment/transform/Transformer.java +++ b/processing/src/main/java/org/apache/druid/segment/transform/Transformer.java @@ -19,7 +19,6 @@ package org.apache.druid.segment.transform; -import com.google.common.base.Preconditions; import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.InputRowListPlusJson; import org.apache.druid.data.input.Row; @@ -102,27 +101,33 @@ public InputRowListPlusJson transform(@Nullable final InputRowListPlusJson row) return null; } - final InputRowListPlusJson transformedRow; + final InputRowListPlusJson inputRowListPlusJson; - if (transforms.isEmpty()) { - transformedRow = row; + if (transforms.isEmpty() || row.getInputRows() == null) { + inputRowListPlusJson = row; } else { - final List originalRows = Preconditions.checkNotNull(row.getInputRows(), "rows before transform"); + final List originalRows = row.getInputRows(); final List transformedRows = new ArrayList<>(originalRows.size()); for (InputRow originalRow : originalRows) { transformedRows.add(new TransformedInputRow(originalRow, transforms)); } - transformedRow = InputRowListPlusJson.of(transformedRows, row.getRawJson()); + inputRowListPlusJson = InputRowListPlusJson.of(transformedRows, row.getRawValues()); } if (valueMatcher != null) { - rowSupplierForValueMatcher.set(transformedRow.getInputRow()); - if (!valueMatcher.matches()) { - return null; + if (inputRowListPlusJson.getInputRows() != null) { + final List filteredRows = new ArrayList<>(inputRowListPlusJson.getInputRows().size()); + for (InputRow inputRow : inputRowListPlusJson.getInputRows()) { + rowSupplierForValueMatcher.set(inputRow); + if (valueMatcher.matches()) { + filteredRows.add(inputRow); + } + } + return InputRowListPlusJson.of(filteredRows, row.getRawValues()); } } - return transformedRow; + return inputRowListPlusJson; } public static class TransformedInputRow implements InputRow diff --git a/processing/src/main/java/org/apache/druid/segment/transform/TransformingInputEntityReader.java b/processing/src/main/java/org/apache/druid/segment/transform/TransformingInputEntityReader.java new file mode 100644 index 000000000000..7fd0101df9dc --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/transform/TransformingInputEntityReader.java @@ -0,0 +1,51 @@ +/* + * 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.druid.segment.transform; + +import org.apache.druid.data.input.InputEntityReader; +import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.InputRowListPlusJson; +import org.apache.druid.java.util.common.parsers.CloseableIterator; + +import java.io.IOException; + +public class TransformingInputEntityReader implements InputEntityReader +{ + private final InputEntityReader delegate; + private final Transformer transformer; + + public TransformingInputEntityReader(InputEntityReader delegate, Transformer transformer) + { + this.delegate = delegate; + this.transformer = transformer; + } + + @Override + public CloseableIterator read() throws IOException + { + return delegate.read().map(transformer::transform); + } + + @Override + public CloseableIterator sample() throws IOException + { + return delegate.sample().map(transformer::transform); + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/transform/TransformingReader.java b/processing/src/main/java/org/apache/druid/segment/transform/TransformingInputSourceReader.java similarity index 90% rename from processing/src/main/java/org/apache/druid/segment/transform/TransformingReader.java rename to processing/src/main/java/org/apache/druid/segment/transform/TransformingInputSourceReader.java index 25944f6b6c27..5f9cfc0a885e 100644 --- a/processing/src/main/java/org/apache/druid/segment/transform/TransformingReader.java +++ b/processing/src/main/java/org/apache/druid/segment/transform/TransformingInputSourceReader.java @@ -26,12 +26,12 @@ import java.io.IOException; -public class TransformingReader implements InputSourceReader +public class TransformingInputSourceReader implements InputSourceReader { private final InputSourceReader delegate; private final Transformer transformer; - TransformingReader(InputSourceReader delegate, Transformer transformer) + TransformingInputSourceReader(InputSourceReader delegate, Transformer transformer) { this.delegate = delegate; this.transformer = transformer; diff --git a/server/src/main/java/org/apache/druid/segment/indexing/DataSchema.java b/server/src/main/java/org/apache/druid/segment/indexing/DataSchema.java index f0c7bffa9885..37009da2bdae 100644 --- a/server/src/main/java/org/apache/druid/segment/indexing/DataSchema.java +++ b/server/src/main/java/org/apache/druid/segment/indexing/DataSchema.java @@ -87,15 +87,15 @@ public DataSchema( this.dataSource = dataSource; this.timestampSpec = timestampSpec; + this.aggregators = aggregators == null ? new AggregatorFactory[]{} : aggregators; this.dimensionsSpec = dimensionsSpec == null ? null : computeDimensionsSpec( Preconditions.checkNotNull(timestampSpec, "timestampSpec"), dimensionsSpec, - aggregators + this.aggregators ); - this.aggregators = aggregators == null ? new AggregatorFactory[]{} : aggregators; if (granularitySpec == null) { log.warn("No granularitySpec has been specified. Using UniformGranularitySpec as default."); this.granularitySpec = new UniformGranularitySpec(null, null, null); diff --git a/server/src/main/java/org/apache/druid/segment/indexing/granularity/ArbitraryGranularitySpec.java b/server/src/main/java/org/apache/druid/segment/indexing/granularity/ArbitraryGranularitySpec.java index 5e95f9c55cb2..7112bf4ee12a 100644 --- a/server/src/main/java/org/apache/druid/segment/indexing/granularity/ArbitraryGranularitySpec.java +++ b/server/src/main/java/org/apache/druid/segment/indexing/granularity/ArbitraryGranularitySpec.java @@ -33,6 +33,7 @@ import org.joda.time.DateTime; import org.joda.time.Interval; +import javax.annotation.Nullable; import java.util.ArrayList; import java.util.List; import java.util.SortedSet; @@ -48,7 +49,7 @@ public class ArbitraryGranularitySpec implements GranularitySpec public ArbitraryGranularitySpec( @JsonProperty("queryGranularity") Granularity queryGranularity, @JsonProperty("rollup") Boolean rollup, - @JsonProperty("intervals") List inputIntervals + @JsonProperty("intervals") @Nullable List inputIntervals ) { this.queryGranularity = queryGranularity == null ? Granularities.NONE : queryGranularity; diff --git a/server/src/main/java/org/apache/druid/segment/realtime/firehose/InlineFirehose.java b/server/src/main/java/org/apache/druid/segment/realtime/firehose/InlineFirehose.java index d8d85179ec03..5125eaab791a 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/firehose/InlineFirehose.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/firehose/InlineFirehose.java @@ -25,7 +25,6 @@ import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.InputRowListPlusJson; import org.apache.druid.data.input.impl.StringInputRowParser; -import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.parsers.ParseException; import java.io.ByteArrayInputStream; @@ -77,10 +76,10 @@ public InputRowListPlusJson nextRowWithRaw() { String raw = nextRaw(); try { - return InputRowListPlusJson.of(parser.parse(raw), StringUtils.toUtf8(raw)); + return InputRowListPlusJson.of(parser.parse(raw), parser.parseString(raw)); } catch (ParseException e) { - return InputRowListPlusJson.of(StringUtils.toUtf8(raw), e); + return InputRowListPlusJson.of(parser.parseString(raw), e); } } diff --git a/server/src/main/java/org/apache/druid/segment/realtime/firehose/InlineFirehoseFactory.java b/server/src/main/java/org/apache/druid/segment/realtime/firehose/InlineFirehoseFactory.java index 31cb38c8a58f..24d125910cc4 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/firehose/InlineFirehoseFactory.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/firehose/InlineFirehoseFactory.java @@ -42,7 +42,7 @@ public class InlineFirehoseFactory implements FiniteFirehoseFactory raw = rowPlusRaw.getRawValues(); + Map expected = new HashMap<>(); + expected.put("timestamp", TIMESTAMP_0); + expected.put("value", VALUE_0); + Assert.assertEquals(expected, raw); Assert.assertNull(rowPlusRaw.getParseException()); } @@ -134,11 +140,14 @@ public void testNextRowWithRawNotParseable() InlineFirehose target = create(data); InputRowListPlusJson rowPlusRaw = target.nextRowWithRaw(); - InputRow row = rowPlusRaw.getInputRow(); - Assert.assertNull(row); + Assert.assertNull(rowPlusRaw.getInputRows()); + + Map raw = rowPlusRaw.getRawValues(); + Map expected = new HashMap<>(); + expected.put("timestamp", VALUE_0); + expected.put("value", TIMESTAMP_0); + Assert.assertEquals(expected, raw); - byte[] raw = rowPlusRaw.getRaw(); - assertRawValue(data, raw); Assert.assertNotNull(rowPlusRaw.getParseException()); } @@ -186,8 +195,12 @@ public void testMultiline() // Second line InputRowListPlusJson rowPlusRaw = target.nextRowWithRaw(); - assertRowValue(VALUE_1, rowPlusRaw.getInputRow()); - assertRawValue(LINE_1, rowPlusRaw.getRaw()); + assertRowValue(VALUE_1, Iterables.getOnlyElement(rowPlusRaw.getInputRows())); + Map raw = rowPlusRaw.getRawValues(); + Map expected = new HashMap<>(); + expected.put("timestamp", TIMESTAMP_1); + expected.put("value", VALUE_1); + Assert.assertEquals(expected, raw); Assert.assertNull(rowPlusRaw.getParseException()); Assert.assertFalse(target.hasMore()); @@ -211,12 +224,5 @@ private static void assertRowValue(String expected, InputRow row) Assert.assertEquals(1, values.size()); Assert.assertEquals(expected, values.get(0)); } - - private static void assertRawValue(String expected, byte[] raw) - { - Assert.assertNotNull(raw); - Assert.assertEquals(expected, new String(raw, CHARSET)); - } - } From b6973f7b09933180c00eecd7f5a1f1cf43bedc85 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Mon, 18 Nov 2019 15:40:49 -0800 Subject: [PATCH 2/8] Cleanup javadocs and names --- .../org/apache/druid/data/input/Firehose.java | 8 ++--- .../druid/data/input/InputEntityReader.java | 2 +- ...on.java => InputRowListPlusRawValues.java} | 34 +++++++++++++------ .../druid/data/input/InputSourceReader.java | 2 +- .../input/IntermediateRowParsingReader.java | 12 +++---- .../input/impl/FileIteratingFirehose.java | 8 ++--- .../FirehoseToInputSourceReaderAdaptor.java | 8 ++--- .../impl/InputEntityIteratingReader.java | 4 +-- .../input/impl/TimedShutoffInputSource.java | 4 +++ .../impl/TimedShutoffInputSourceReader.java | 6 ++-- .../impl/TimedShutoffInputSourceTest.java | 4 +-- .../overlord/sampler/InputSourceSampler.java | 21 ++++++------ .../overlord/sampler/SamplerConfig.java | 2 -- .../RecordSupplierInputSource.java | 10 +++++- .../SeekableStreamSamplerSpec.java | 10 +++--- .../indexing/overlord/TaskLifecycleTest.java | 6 ++-- .../druid/segment/transform/Transformer.java | 20 +++++------ .../TransformingInputEntityReader.java | 4 +-- .../TransformingInputSourceReader.java | 4 +-- .../realtime/firehose/InlineFirehose.java | 8 ++--- .../realtime/firehose/InlineFirehoseTest.java | 8 ++--- 21 files changed, 104 insertions(+), 81 deletions(-) rename core/src/main/java/org/apache/druid/data/input/{InputRowListPlusJson.java => InputRowListPlusRawValues.java} (57%) diff --git a/core/src/main/java/org/apache/druid/data/input/Firehose.java b/core/src/main/java/org/apache/druid/data/input/Firehose.java index a95c55df74d1..6c7605ca11d0 100644 --- a/core/src/main/java/org/apache/druid/data/input/Firehose.java +++ b/core/src/main/java/org/apache/druid/data/input/Firehose.java @@ -67,7 +67,7 @@ public interface Firehose extends Closeable InputRow nextRow() throws IOException; /** - * Returns an {@link InputRowListPlusJson} object containing the InputRow plus the raw, unparsed data corresponding to + * Returns an {@link InputRowListPlusRawValues} object containing the InputRow plus the raw, unparsed data corresponding to * the next row available. Used in the sampler to provide the caller with information to assist in configuring a parse * spec. If a ParseException is thrown by the parser, it should be caught and returned in the InputRowListPlusJson so * we will be able to provide information on the raw row which failed to be parsed. Should only be called if hasMore @@ -76,13 +76,13 @@ public interface Firehose extends Closeable * @return an InputRowListPlusJson which may contain any of: an InputRow, map of the raw data, or a ParseException */ @Deprecated - default InputRowListPlusJson nextRowWithRaw() throws IOException + default InputRowListPlusRawValues nextRowWithRaw() throws IOException { try { - return InputRowListPlusJson.of(nextRow(), null); + return InputRowListPlusRawValues.of(nextRow(), null); } catch (ParseException e) { - return InputRowListPlusJson.of(null, e); + return InputRowListPlusRawValues.of(null, e); } } diff --git a/core/src/main/java/org/apache/druid/data/input/InputEntityReader.java b/core/src/main/java/org/apache/druid/data/input/InputEntityReader.java index 7152a751924b..9470028d1373 100644 --- a/core/src/main/java/org/apache/druid/data/input/InputEntityReader.java +++ b/core/src/main/java/org/apache/druid/data/input/InputEntityReader.java @@ -36,5 +36,5 @@ public interface InputEntityReader { CloseableIterator read() throws IOException; - CloseableIterator sample() throws IOException; + CloseableIterator sample() throws IOException; } diff --git a/core/src/main/java/org/apache/druid/data/input/InputRowListPlusJson.java b/core/src/main/java/org/apache/druid/data/input/InputRowListPlusRawValues.java similarity index 57% rename from core/src/main/java/org/apache/druid/data/input/InputRowListPlusJson.java rename to core/src/main/java/org/apache/druid/data/input/InputRowListPlusRawValues.java index dc47098ae3c7..31a39044444c 100644 --- a/core/src/main/java/org/apache/druid/data/input/InputRowListPlusJson.java +++ b/core/src/main/java/org/apache/druid/data/input/InputRowListPlusRawValues.java @@ -27,40 +27,52 @@ import java.util.List; import java.util.Map; -public class InputRowListPlusJson +/** + * A triple of a list of {@link InputRow}s, a {@link Map} of raw values, and a {@link ParseException}. + * The rawValues map contains the raw values before being parsed into InputRows. Note that a single map can be parsed + * into multiple InputRows, for example, with explodeSpec. + * The ParseException is the exception thrown when parsing bytes into either the rawValues map or the list of InputRows. + * + * In any case, one of triple must not be null. + */ +public class InputRowListPlusRawValues { @Nullable private final List inputRows; @Nullable - private final Map rawColumns; + private final Map rawValues; @Nullable private final ParseException parseException; - public static InputRowListPlusJson of(@Nullable InputRow inputRow, Map rawColumns) + public static InputRowListPlusRawValues of(@Nullable InputRow inputRow, Map rawColumns) { return of(inputRow == null ? null : Collections.singletonList(inputRow), rawColumns); } - public static InputRowListPlusJson of(@Nullable List inputRows, Map rawColumns) + public static InputRowListPlusRawValues of(@Nullable List inputRows, Map rawColumns) { - return new InputRowListPlusJson(inputRows, Preconditions.checkNotNull(rawColumns, "rawColumns"), null); + return new InputRowListPlusRawValues(inputRows, Preconditions.checkNotNull(rawColumns, "rawColumns"), null); } - public static InputRowListPlusJson of(@Nullable Map rawColumns, ParseException parseException) + public static InputRowListPlusRawValues of(@Nullable Map rawColumns, ParseException parseException) { - return new InputRowListPlusJson(null, rawColumns, Preconditions.checkNotNull(parseException, "parseException")); + return new InputRowListPlusRawValues( + null, + rawColumns, + Preconditions.checkNotNull(parseException, "parseException") + ); } - private InputRowListPlusJson( + private InputRowListPlusRawValues( @Nullable List inputRows, - @Nullable Map rawColumns, + @Nullable Map rawValues, @Nullable ParseException parseException ) { this.inputRows = inputRows; - this.rawColumns = rawColumns; + this.rawValues = rawValues; this.parseException = parseException; } @@ -73,7 +85,7 @@ public List getInputRows() @Nullable public Map getRawValues() { - return rawColumns; + return rawValues; } @Nullable diff --git a/core/src/main/java/org/apache/druid/data/input/InputSourceReader.java b/core/src/main/java/org/apache/druid/data/input/InputSourceReader.java index 0a8921422c39..3e1d233fa3e6 100644 --- a/core/src/main/java/org/apache/druid/data/input/InputSourceReader.java +++ b/core/src/main/java/org/apache/druid/data/input/InputSourceReader.java @@ -39,5 +39,5 @@ public interface InputSourceReader { CloseableIterator read() throws IOException; - CloseableIterator sample() throws IOException; + CloseableIterator sample() throws IOException; } diff --git a/core/src/main/java/org/apache/druid/data/input/IntermediateRowParsingReader.java b/core/src/main/java/org/apache/druid/data/input/IntermediateRowParsingReader.java index 103992a99c02..aae448b8799b 100644 --- a/core/src/main/java/org/apache/druid/data/input/IntermediateRowParsingReader.java +++ b/core/src/main/java/org/apache/druid/data/input/IntermediateRowParsingReader.java @@ -56,7 +56,7 @@ public CloseableIterator read() throws IOException } @Override - public CloseableIterator sample() + public CloseableIterator sample() throws IOException { return intermediateRowIterator().map(row -> { @@ -65,16 +65,16 @@ public CloseableIterator sample() rawColumns = toMap(row); } catch (Exception e) { - return InputRowListPlusJson.of(null, new ParseException(e, "Unable to parse row [%s] into JSON", row)); + return InputRowListPlusRawValues.of(null, new ParseException(e, "Unable to parse row [%s] into JSON", row)); } try { - return InputRowListPlusJson.of(parseInputRows(row), rawColumns); + return InputRowListPlusRawValues.of(parseInputRows(row), rawColumns); } catch (ParseException e) { - return InputRowListPlusJson.of(rawColumns, e); + return InputRowListPlusRawValues.of(rawColumns, e); } catch (IOException e) { - return InputRowListPlusJson.of(rawColumns, new ParseException(e, "Unable to parse row [%s] into inputRow", row)); + return InputRowListPlusRawValues.of(rawColumns, new ParseException(e, "Unable to parse row [%s] into inputRow", row)); } }); } @@ -91,7 +91,7 @@ public CloseableIterator sample() protected abstract List parseInputRows(T intermediateRow) throws IOException, ParseException; /** - * Converts the given intermediate row into a {@link Map}. The returned JSON will be used by FirehoseSampler. + * Converts the given intermediate row into a {@link Map}. The returned JSON will be used by InputSourceSampler. * Implementations can use any method to convert the given row into a Map. */ protected abstract Map toMap(T intermediateRow) throws IOException; diff --git a/core/src/main/java/org/apache/druid/data/input/impl/FileIteratingFirehose.java b/core/src/main/java/org/apache/druid/data/input/impl/FileIteratingFirehose.java index 74f1d4c7392d..42c627df2138 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/FileIteratingFirehose.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/FileIteratingFirehose.java @@ -22,7 +22,7 @@ import org.apache.commons.io.LineIterator; import org.apache.druid.data.input.Firehose; import org.apache.druid.data.input.InputRow; -import org.apache.druid.data.input.InputRowListPlusJson; +import org.apache.druid.data.input.InputRowListPlusRawValues; import org.apache.druid.java.util.common.parsers.ParseException; import javax.annotation.Nullable; @@ -81,7 +81,7 @@ public InputRow nextRow() throws IOException } @Override - public InputRowListPlusJson nextRowWithRaw() throws IOException + public InputRowListPlusRawValues nextRowWithRaw() throws IOException { if (!hasMore()) { throw new NoSuchElementException(); @@ -89,10 +89,10 @@ public InputRowListPlusJson nextRowWithRaw() throws IOException String raw = lineIterator.next(); try { - return InputRowListPlusJson.of(parser.parse(raw), parser.parseString(raw)); + return InputRowListPlusRawValues.of(parser.parse(raw), parser.parseString(raw)); } catch (ParseException e) { - return InputRowListPlusJson.of(parser.parseString(raw), e); + return InputRowListPlusRawValues.of(parser.parseString(raw), e); } } diff --git a/core/src/main/java/org/apache/druid/data/input/impl/FirehoseToInputSourceReaderAdaptor.java b/core/src/main/java/org/apache/druid/data/input/impl/FirehoseToInputSourceReaderAdaptor.java index 8dd1ea3b8438..697c64fe7a37 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/FirehoseToInputSourceReaderAdaptor.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/FirehoseToInputSourceReaderAdaptor.java @@ -22,7 +22,7 @@ import org.apache.druid.data.input.Firehose; import org.apache.druid.data.input.FirehoseFactory; import org.apache.druid.data.input.InputRow; -import org.apache.druid.data.input.InputRowListPlusJson; +import org.apache.druid.data.input.InputRowListPlusRawValues; import org.apache.druid.data.input.InputSourceReader; import org.apache.druid.java.util.common.parsers.CloseableIterator; @@ -85,9 +85,9 @@ public void close() throws IOException } @Override - public CloseableIterator sample() throws IOException + public CloseableIterator sample() throws IOException { - return new CloseableIterator() + return new CloseableIterator() { final Firehose firehose = firehoseFactory.connectForSampler(inputRowParser, temporaryDirectory); @@ -103,7 +103,7 @@ public boolean hasNext() } @Override - public InputRowListPlusJson next() + public InputRowListPlusRawValues next() { try { return firehose.nextRowWithRaw(); diff --git a/core/src/main/java/org/apache/druid/data/input/impl/InputEntityIteratingReader.java b/core/src/main/java/org/apache/druid/data/input/impl/InputEntityIteratingReader.java index ec0090e2a284..3ca0412d8852 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/InputEntityIteratingReader.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/InputEntityIteratingReader.java @@ -23,7 +23,7 @@ import org.apache.druid.data.input.InputEntityReader; import org.apache.druid.data.input.InputFormat; import org.apache.druid.data.input.InputRow; -import org.apache.druid.data.input.InputRowListPlusJson; +import org.apache.druid.data.input.InputRowListPlusRawValues; import org.apache.druid.data.input.InputRowSchema; import org.apache.druid.data.input.InputSourceReader; import org.apache.druid.java.util.common.CloseableIterators; @@ -84,7 +84,7 @@ public CloseableIterator read() } @Override - public CloseableIterator sample() + public CloseableIterator sample() { return createIterator(entity -> { // InputEntityReader is stateful and so a new one should be created per entity. diff --git a/core/src/main/java/org/apache/druid/data/input/impl/TimedShutoffInputSource.java b/core/src/main/java/org/apache/druid/data/input/impl/TimedShutoffInputSource.java index bc72580b8668..fd944e5835c1 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/TimedShutoffInputSource.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/TimedShutoffInputSource.java @@ -29,6 +29,10 @@ import javax.annotation.Nullable; import java.io.File; +/** + * A wrapping InputSource that will close the underlying InputSource at {@link #shutoffTime}. + * This InputSource is supposed to be used only for InputSourceSampler. + */ public class TimedShutoffInputSource implements InputSource { private final InputSource delegate; diff --git a/core/src/main/java/org/apache/druid/data/input/impl/TimedShutoffInputSourceReader.java b/core/src/main/java/org/apache/druid/data/input/impl/TimedShutoffInputSourceReader.java index 58a3ace1e542..2094bf96512f 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/TimedShutoffInputSourceReader.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/TimedShutoffInputSourceReader.java @@ -20,7 +20,7 @@ package org.apache.druid.data.input.impl; import org.apache.druid.data.input.InputRow; -import org.apache.druid.data.input.InputRowListPlusJson; +import org.apache.druid.data.input.InputRowListPlusRawValues; import org.apache.druid.data.input.InputSourceReader; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.io.Closer; @@ -55,10 +55,10 @@ public CloseableIterator read() throws IOException } @Override - public CloseableIterator sample() throws IOException + public CloseableIterator sample() throws IOException { final ScheduledExecutorService shutdownExec = Execs.scheduledSingleThreaded("timed-shutoff-reader-%d"); - final CloseableIterator delegateIterator = delegate.sample(); + final CloseableIterator delegateIterator = delegate.sample(); return decorateShutdownTimeout(shutdownExec, delegateIterator); } diff --git a/core/src/test/java/org/apache/druid/data/input/impl/TimedShutoffInputSourceTest.java b/core/src/test/java/org/apache/druid/data/input/impl/TimedShutoffInputSourceTest.java index 205f85613674..4f04e880d9b1 100644 --- a/core/src/test/java/org/apache/druid/data/input/impl/TimedShutoffInputSourceTest.java +++ b/core/src/test/java/org/apache/druid/data/input/impl/TimedShutoffInputSourceTest.java @@ -21,7 +21,7 @@ import com.google.common.collect.ImmutableList; import org.apache.druid.data.input.InputFormat; -import org.apache.druid.data.input.InputRowListPlusJson; +import org.apache.druid.data.input.InputRowListPlusRawValues; import org.apache.druid.data.input.InputRowSchema; import org.apache.druid.data.input.InputSource; import org.apache.druid.data.input.InputSourceReader; @@ -49,7 +49,7 @@ public void testTimeoutShutoff() throws IOException, InterruptedException inputFormat, null ); - try (CloseableIterator iterator = reader.sample()) { + try (CloseableIterator iterator = reader.sample()) { Thread.sleep(timeoutMs + 1000); Assert.assertFalse(iterator.hasNext()); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/InputSourceSampler.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/InputSourceSampler.java index 6c44a3726157..21321bf4adaa 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/InputSourceSampler.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/InputSourceSampler.java @@ -25,7 +25,7 @@ import org.apache.commons.lang3.ArrayUtils; import org.apache.druid.data.input.InputFormat; import org.apache.druid.data.input.InputRow; -import org.apache.druid.data.input.InputRowListPlusJson; +import org.apache.druid.data.input.InputRowListPlusRawValues; import org.apache.druid.data.input.InputRowSchema; import org.apache.druid.data.input.InputSource; import org.apache.druid.data.input.InputSourceReader; @@ -83,7 +83,8 @@ public class InputSourceSampler public SamplerResponse sample( final InputSource inputSource, - @Nullable final InputFormat inputFormat, // can be null only if inputSource.needsFormat() = false + // inputFormat can be null only if inputSource.needsFormat() = false or parser is specified. + @Nullable final InputFormat inputFormat, @Nullable final DataSchema dataSchema, @Nullable final SamplerConfig samplerConfig ) @@ -110,7 +111,7 @@ public SamplerResponse sample( inputFormat, tempDir ); - try (final CloseableIterator iterator = reader.sample(); + try (final CloseableIterator iterator = reader.sample(); final IncrementalIndex index = buildIncrementalIndex(nonNullSamplerConfig, nonNullDataSchema); final Closer closer1 = closer) { SamplerResponseRow[] responseRows = new SamplerResponseRow[nonNullSamplerConfig.getNumRows()]; @@ -119,21 +120,21 @@ public SamplerResponse sample( while (counter < responseRows.length && iterator.hasNext()) { Map rawColumns = null; try { - final InputRowListPlusJson inputRowListPlusJson = iterator.next(); + final InputRowListPlusRawValues inputRowListPlusRawValues = iterator.next(); - if (inputRowListPlusJson.getRawValues() != null) { - rawColumns = inputRowListPlusJson.getRawValues(); + if (inputRowListPlusRawValues.getRawValues() != null) { + rawColumns = inputRowListPlusRawValues.getRawValues(); } - if (inputRowListPlusJson.getParseException() != null) { - throw inputRowListPlusJson.getParseException(); + if (inputRowListPlusRawValues.getParseException() != null) { + throw inputRowListPlusRawValues.getParseException(); } - if (inputRowListPlusJson.getInputRows() == null) { + if (inputRowListPlusRawValues.getInputRows() == null) { continue; } - for (InputRow row : inputRowListPlusJson.getInputRows()) { + for (InputRow row : inputRowListPlusRawValues.getInputRows()) { if (!Intervals.ETERNITY.contains(row.getTimestamp())) { throw new ParseException("Timestamp cannot be represented as a long: [%s]", row); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/SamplerConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/SamplerConfig.java index 3a907c026622..6799663cddb2 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/SamplerConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/SamplerConfig.java @@ -47,8 +47,6 @@ public SamplerConfig( /** * The maximum number of rows to return in a response. The actual number of returned rows may be less if: * - The sampled source contains less data. - * - We are reading from the cache ({@link SamplerConfig#cacheKey} is set and {@link SamplerConfig#isSkipCache()} - * is false) and the cache contains less data. * - {@link SamplerConfig#timeoutMs} elapses before this value is reached. * - {@link org.apache.druid.segment.indexing.granularity.GranularitySpec#isRollup()} is true and input rows get * rolled-up into fewer indexed rows. diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/RecordSupplierInputSource.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/RecordSupplierInputSource.java index f3d1b6122577..ef49ce3fcea0 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/RecordSupplierInputSource.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/RecordSupplierInputSource.java @@ -38,13 +38,17 @@ import java.util.Set; import java.util.stream.Collectors; +/** + * {@link org.apache.druid.data.input.InputSource} wrapping {@link RecordSupplier}. It will fetch data via + * RecordSupplier and convert it into {@link ByteEntity}. See {@link #createEntityIterator}. + */ public class RecordSupplierInputSource extends AbstractInputSource { private final String topic; private final RecordSupplier recordSupplier; private final boolean useEarliestOffset; - public RecordSupplierInputSource( + RecordSupplierInputSource( String topic, RecordSupplier recordSupplier, boolean useEarliestOffset @@ -106,6 +110,10 @@ protected InputSourceReader formattableReader( ); } + /** + * Returns an iterator converting each byte array from RecordSupplier into a ByteEntity. Note that the + * returned iterator will be blocked until the RecordSupplier gives any data. + */ CloseableIterator createEntityIterator() { return new CloseableIterator() diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamSamplerSpec.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamSamplerSpec.java index dadeee506666..3a9a92f3c9ff 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamSamplerSpec.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamSamplerSpec.java @@ -27,7 +27,7 @@ import org.apache.druid.data.input.InputEntity; import org.apache.druid.data.input.InputFormat; import org.apache.druid.data.input.InputRow; -import org.apache.druid.data.input.InputRowListPlusJson; +import org.apache.druid.data.input.InputRowListPlusRawValues; import org.apache.druid.data.input.InputSource; import org.apache.druid.data.input.InputSplit; import org.apache.druid.data.input.SplitHintSpec; @@ -180,7 +180,7 @@ public InputRow nextRow() } @Override - public InputRowListPlusJson nextRowWithRaw() + public InputRowListPlusRawValues nextRowWithRaw() { final ByteBuffer bb = ((ByteEntity) entityIterator.next()).getBuffer(); @@ -193,15 +193,15 @@ public InputRowListPlusJson nextRowWithRaw() } } catch (ParseException e) { - return InputRowListPlusJson.of(null, e); + return InputRowListPlusRawValues.of(null, e); } try { final List rows = parser.parseBatch(bb); - return InputRowListPlusJson.of(rows.isEmpty() ? null : rows, rawColumns); + return InputRowListPlusRawValues.of(rows.isEmpty() ? null : rows, rawColumns); } catch (ParseException e) { - return InputRowListPlusJson.of(rawColumns, e); + return InputRowListPlusRawValues.of(rawColumns, e); } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java index 38d25dd2b1da..35c3a8060e00 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java @@ -37,7 +37,7 @@ import org.apache.druid.data.input.Firehose; import org.apache.druid.data.input.FirehoseFactory; import org.apache.druid.data.input.InputRow; -import org.apache.druid.data.input.InputRowListPlusJson; +import org.apache.druid.data.input.InputRowListPlusRawValues; import org.apache.druid.data.input.InputRowSchema; import org.apache.druid.data.input.InputSourceReader; import org.apache.druid.data.input.MapBasedInputRow; @@ -313,7 +313,7 @@ public InputRow next() } @Override - public CloseableIterator sample() + public CloseableIterator sample() { throw new UnsupportedOperationException(); } @@ -348,7 +348,7 @@ public CloseableIterator read() } @Override - public CloseableIterator sample() + public CloseableIterator sample() { throw new UnsupportedOperationException(); } diff --git a/processing/src/main/java/org/apache/druid/segment/transform/Transformer.java b/processing/src/main/java/org/apache/druid/segment/transform/Transformer.java index cbde1d0e9c6b..6906022a9909 100644 --- a/processing/src/main/java/org/apache/druid/segment/transform/Transformer.java +++ b/processing/src/main/java/org/apache/druid/segment/transform/Transformer.java @@ -20,7 +20,7 @@ package org.apache.druid.segment.transform; import org.apache.druid.data.input.InputRow; -import org.apache.druid.data.input.InputRowListPlusJson; +import org.apache.druid.data.input.InputRowListPlusRawValues; import org.apache.druid.data.input.Row; import org.apache.druid.data.input.Rows; import org.apache.druid.java.util.common.DateTimes; @@ -95,39 +95,39 @@ public InputRow transform(@Nullable final InputRow row) } @Nullable - public InputRowListPlusJson transform(@Nullable final InputRowListPlusJson row) + public InputRowListPlusRawValues transform(@Nullable final InputRowListPlusRawValues row) { if (row == null) { return null; } - final InputRowListPlusJson inputRowListPlusJson; + final InputRowListPlusRawValues inputRowListPlusRawValues; if (transforms.isEmpty() || row.getInputRows() == null) { - inputRowListPlusJson = row; + inputRowListPlusRawValues = row; } else { final List originalRows = row.getInputRows(); final List transformedRows = new ArrayList<>(originalRows.size()); for (InputRow originalRow : originalRows) { transformedRows.add(new TransformedInputRow(originalRow, transforms)); } - inputRowListPlusJson = InputRowListPlusJson.of(transformedRows, row.getRawValues()); + inputRowListPlusRawValues = InputRowListPlusRawValues.of(transformedRows, row.getRawValues()); } if (valueMatcher != null) { - if (inputRowListPlusJson.getInputRows() != null) { - final List filteredRows = new ArrayList<>(inputRowListPlusJson.getInputRows().size()); - for (InputRow inputRow : inputRowListPlusJson.getInputRows()) { + if (inputRowListPlusRawValues.getInputRows() != null) { + final List filteredRows = new ArrayList<>(inputRowListPlusRawValues.getInputRows().size()); + for (InputRow inputRow : inputRowListPlusRawValues.getInputRows()) { rowSupplierForValueMatcher.set(inputRow); if (valueMatcher.matches()) { filteredRows.add(inputRow); } } - return InputRowListPlusJson.of(filteredRows, row.getRawValues()); + return InputRowListPlusRawValues.of(filteredRows, row.getRawValues()); } } - return inputRowListPlusJson; + return inputRowListPlusRawValues; } public static class TransformedInputRow implements InputRow diff --git a/processing/src/main/java/org/apache/druid/segment/transform/TransformingInputEntityReader.java b/processing/src/main/java/org/apache/druid/segment/transform/TransformingInputEntityReader.java index 7fd0101df9dc..33bed4658691 100644 --- a/processing/src/main/java/org/apache/druid/segment/transform/TransformingInputEntityReader.java +++ b/processing/src/main/java/org/apache/druid/segment/transform/TransformingInputEntityReader.java @@ -21,7 +21,7 @@ import org.apache.druid.data.input.InputEntityReader; import org.apache.druid.data.input.InputRow; -import org.apache.druid.data.input.InputRowListPlusJson; +import org.apache.druid.data.input.InputRowListPlusRawValues; import org.apache.druid.java.util.common.parsers.CloseableIterator; import java.io.IOException; @@ -44,7 +44,7 @@ public CloseableIterator read() throws IOException } @Override - public CloseableIterator sample() throws IOException + public CloseableIterator sample() throws IOException { return delegate.sample().map(transformer::transform); } diff --git a/processing/src/main/java/org/apache/druid/segment/transform/TransformingInputSourceReader.java b/processing/src/main/java/org/apache/druid/segment/transform/TransformingInputSourceReader.java index 5f9cfc0a885e..4039cbdb6e2d 100644 --- a/processing/src/main/java/org/apache/druid/segment/transform/TransformingInputSourceReader.java +++ b/processing/src/main/java/org/apache/druid/segment/transform/TransformingInputSourceReader.java @@ -20,7 +20,7 @@ package org.apache.druid.segment.transform; import org.apache.druid.data.input.InputRow; -import org.apache.druid.data.input.InputRowListPlusJson; +import org.apache.druid.data.input.InputRowListPlusRawValues; import org.apache.druid.data.input.InputSourceReader; import org.apache.druid.java.util.common.parsers.CloseableIterator; @@ -44,7 +44,7 @@ public CloseableIterator read() throws IOException } @Override - public CloseableIterator sample() throws IOException + public CloseableIterator sample() throws IOException { return delegate.sample().map(transformer::transform); } diff --git a/server/src/main/java/org/apache/druid/segment/realtime/firehose/InlineFirehose.java b/server/src/main/java/org/apache/druid/segment/realtime/firehose/InlineFirehose.java index 5125eaab791a..924ee92b4cb1 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/firehose/InlineFirehose.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/firehose/InlineFirehose.java @@ -23,7 +23,7 @@ import org.apache.commons.io.LineIterator; import org.apache.druid.data.input.Firehose; import org.apache.druid.data.input.InputRow; -import org.apache.druid.data.input.InputRowListPlusJson; +import org.apache.druid.data.input.InputRowListPlusRawValues; import org.apache.druid.data.input.impl.StringInputRowParser; import org.apache.druid.java.util.common.parsers.ParseException; @@ -72,14 +72,14 @@ private String nextRaw() } @Override - public InputRowListPlusJson nextRowWithRaw() + public InputRowListPlusRawValues nextRowWithRaw() { String raw = nextRaw(); try { - return InputRowListPlusJson.of(parser.parse(raw), parser.parseString(raw)); + return InputRowListPlusRawValues.of(parser.parse(raw), parser.parseString(raw)); } catch (ParseException e) { - return InputRowListPlusJson.of(parser.parseString(raw), e); + return InputRowListPlusRawValues.of(parser.parseString(raw), e); } } diff --git a/server/src/test/java/org/apache/druid/segment/realtime/firehose/InlineFirehoseTest.java b/server/src/test/java/org/apache/druid/segment/realtime/firehose/InlineFirehoseTest.java index 5b7279c35403..2a717c566829 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/firehose/InlineFirehoseTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/firehose/InlineFirehoseTest.java @@ -21,7 +21,7 @@ import com.google.common.collect.Iterables; import org.apache.druid.data.input.InputRow; -import org.apache.druid.data.input.InputRowListPlusJson; +import org.apache.druid.data.input.InputRowListPlusRawValues; import org.apache.druid.data.input.impl.CSVParseSpec; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.StringInputRowParser; @@ -119,7 +119,7 @@ public void testNextRowWithRawParseable() { final String data = PARSEABLE; InlineFirehose target = create(data); - InputRowListPlusJson rowPlusRaw = target.nextRowWithRaw(); + InputRowListPlusRawValues rowPlusRaw = target.nextRowWithRaw(); InputRow row = Iterables.getOnlyElement(rowPlusRaw.getInputRows()); assertRowValue(VALUE_0, row); @@ -138,7 +138,7 @@ public void testNextRowWithRawNotParseable() { final String data = NOT_PARSEABLE; InlineFirehose target = create(data); - InputRowListPlusJson rowPlusRaw = target.nextRowWithRaw(); + InputRowListPlusRawValues rowPlusRaw = target.nextRowWithRaw(); Assert.assertNull(rowPlusRaw.getInputRows()); @@ -194,7 +194,7 @@ public void testMultiline() assertRowValue(VALUE_0, row0); // Second line - InputRowListPlusJson rowPlusRaw = target.nextRowWithRaw(); + InputRowListPlusRawValues rowPlusRaw = target.nextRowWithRaw(); assertRowValue(VALUE_1, Iterables.getOnlyElement(rowPlusRaw.getInputRows())); Map raw = rowPlusRaw.getRawValues(); Map expected = new HashMap<>(); From bbd3e113e4bcdb1c4b5210020c20443e459b030d Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 19 Nov 2019 18:25:50 -0800 Subject: [PATCH 3/8] fix style --- .../apache/druid/indexing/overlord/sampler/FirehoseSampler.java | 0 .../druid/indexing/overlord/sampler/InputSourceSamplerTest.java | 1 - 2 files changed, 1 deletion(-) delete mode 100644 indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/FirehoseSampler.java diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/FirehoseSampler.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/FirehoseSampler.java deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/InputSourceSamplerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/InputSourceSamplerTest.java index b00e3820258b..3cb8dd41376c 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/InputSourceSamplerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/InputSourceSamplerTest.java @@ -21,7 +21,6 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; import org.apache.druid.common.config.NullHandling; import org.apache.druid.data.input.FirehoseFactoryToInputSourceAdaptor; import org.apache.druid.data.input.InputFormat; From 9a74d0ae1706169e44e3e5d3ea05fd838c2cf39b Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 19 Nov 2019 18:59:44 -0800 Subject: [PATCH 4/8] fix timed shutoff input source reader --- .../impl/TimedShutoffInputSourceReader.java | 28 +++++++++++++++++-- .../indexing/kafka/KafkaSamplerSpecTest.java | 3 +- 2 files changed, 26 insertions(+), 5 deletions(-) diff --git a/core/src/main/java/org/apache/druid/data/input/impl/TimedShutoffInputSourceReader.java b/core/src/main/java/org/apache/druid/data/input/impl/TimedShutoffInputSourceReader.java index 2094bf96512f..dd5f3ecc4f08 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/TimedShutoffInputSourceReader.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/TimedShutoffInputSourceReader.java @@ -72,21 +72,43 @@ private CloseableIterator decorateShutdownTimeout( closer.register(exec::shutdownNow); final CloseableIterator wrappingIterator = new CloseableIterator() { + /** + * Indicates this iterator has been closed or not. + * Volatile since there is a happens-before relationship between {@link #hasNext()} and {@link #close()}. + */ volatile boolean closed; + /** + * Indicates {@link #next} is valid or not. + * Not volatile since {@link #hasNext()} and {@link #next()} are supposed to be called by the same thread. + */ + boolean validNext; + /** + * Caching the next item. + * Not volatile since {@link #hasNext()} and {@link #next()} are supposed to be called by the same thread. + */ + T next; @Override public boolean hasNext() { - return !closed && delegateIterator.hasNext(); + if (!closed && delegateIterator.hasNext()) { + next = delegateIterator.next(); + validNext = true; + return true; + } else { + validNext = false; + return false; + } } @Override public T next() { - if (!hasNext()) { + if (validNext) { + return next; + } else { throw new NoSuchElementException(); } - return delegateIterator.next(); } @Override diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaSamplerSpecTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaSamplerSpecTest.java index 48f1811845d1..87a096d88373 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaSamplerSpecTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaSamplerSpecTest.java @@ -53,7 +53,6 @@ import org.junit.Test; import java.util.Arrays; -import java.util.HashMap; import java.util.Iterator; import java.util.List; @@ -125,7 +124,7 @@ public void testSample() null, new KafkaSupervisorIOConfig( TOPIC, - new JsonInputFormat(new JSONPathSpec(true, ImmutableList.of()), ImmutableMap.of()), + new JsonInputFormat(JSONPathSpec.DEFAULT, null), null, null, null, From 2ccf186dd8050cc711796055fa75dfe9a3e6b2b5 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 19 Nov 2019 20:58:09 -0800 Subject: [PATCH 5/8] fix timed shutoff input source reader again --- .../druid/data/input/impl/TimedShutoffInputSourceReader.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/org/apache/druid/data/input/impl/TimedShutoffInputSourceReader.java b/core/src/main/java/org/apache/druid/data/input/impl/TimedShutoffInputSourceReader.java index dd5f3ecc4f08..f76d2959c628 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/TimedShutoffInputSourceReader.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/TimedShutoffInputSourceReader.java @@ -91,12 +91,14 @@ private CloseableIterator decorateShutdownTimeout( @Override public boolean hasNext() { + if (validNext) { + return true; + } if (!closed && delegateIterator.hasNext()) { next = delegateIterator.next(); validNext = true; return true; } else { - validNext = false; return false; } } @@ -105,6 +107,7 @@ public boolean hasNext() public T next() { if (validNext) { + validNext = false; return next; } else { throw new NoSuchElementException(); From 630628d2465ed538a0d2bdae534f37250f70f6c0 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 19 Nov 2019 23:18:45 -0800 Subject: [PATCH 6/8] tidy up timed shutoff reader --- .../impl/TimedShutoffInputSourceReader.java | 20 ++++++++----------- 1 file changed, 8 insertions(+), 12 deletions(-) diff --git a/core/src/main/java/org/apache/druid/data/input/impl/TimedShutoffInputSourceReader.java b/core/src/main/java/org/apache/druid/data/input/impl/TimedShutoffInputSourceReader.java index f76d2959c628..632832b3ba99 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/TimedShutoffInputSourceReader.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/TimedShutoffInputSourceReader.java @@ -78,25 +78,20 @@ private CloseableIterator decorateShutdownTimeout( */ volatile boolean closed; /** - * Indicates {@link #next} is valid or not. + * Caching the next item. The item returned from the underling iterator is either a non-null {@link InputRow} + * or {@link InputRowListPlusRawValues}. * Not volatile since {@link #hasNext()} and {@link #next()} are supposed to be called by the same thread. */ - boolean validNext; - /** - * Caching the next item. - * Not volatile since {@link #hasNext()} and {@link #next()} are supposed to be called by the same thread. - */ - T next; + T next = null; @Override public boolean hasNext() { - if (validNext) { + if (next != null) { return true; } if (!closed && delegateIterator.hasNext()) { next = delegateIterator.next(); - validNext = true; return true; } else { return false; @@ -106,9 +101,10 @@ public boolean hasNext() @Override public T next() { - if (validNext) { - validNext = false; - return next; + if (next != null) { + final T returnValue = next; + next = null; + return returnValue; } else { throw new NoSuchElementException(); } From 40e0f1cfbddca22a7ec56a2d045a866a191f02d4 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 20 Nov 2019 09:57:07 -0800 Subject: [PATCH 7/8] unused imports --- .../org/apache/druid/java/util/common/parsers/CSVParser.java | 3 --- 1 file changed, 3 deletions(-) diff --git a/core/src/main/java/org/apache/druid/java/util/common/parsers/CSVParser.java b/core/src/main/java/org/apache/druid/java/util/common/parsers/CSVParser.java index 17538bf25a96..17eba147564b 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/parsers/CSVParser.java +++ b/core/src/main/java/org/apache/druid/java/util/common/parsers/CSVParser.java @@ -21,9 +21,6 @@ import com.google.common.annotations.VisibleForTesting; import com.opencsv.RFC4180Parser; -import com.opencsv.RFC4180ParserBuilder; -import com.opencsv.enums.CSVReaderNullFieldIndicator; -import org.apache.druid.common.config.NullHandling; import org.apache.druid.data.input.impl.CsvReader; import javax.annotation.Nullable; From e2f04362aabd14829932a6b54aaa9a0bc0e1cad8 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 20 Nov 2019 12:14:42 -0800 Subject: [PATCH 8/8] fix tc --- .../segment/incremental/IncrementalIndexSchema.java | 12 ------------ 1 file changed, 12 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexSchema.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexSchema.java index 4b96cc4257ee..ac9cc1d4d612 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexSchema.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexSchema.java @@ -127,18 +127,6 @@ public Builder withTimestampSpec(TimestampSpec timestampSpec) return this; } - public Builder withTimestampSpec(InputRowParser parser) - { - if (parser != null - && parser.getParseSpec() != null - && parser.getParseSpec().getTimestampSpec() != null) { - this.timestampSpec = parser.getParseSpec().getTimestampSpec(); - } else { - this.timestampSpec = new TimestampSpec(null, null, null); - } - return this; - } - public Builder withQueryGranularity(Granularity gran) { this.gran = gran;